Browse Source

HADOOP-2579 initializing a new HTable object against a nonexistent table
throws a NoServerForRegionException instead of a TableNotFoundException
when a different table has been created previously


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@612141 13f79535-47bb-0310-9956-ffa450edef68

Michael Stack 17 years ago
parent
commit
99978b6272

+ 4 - 0
src/contrib/hbase/CHANGES.txt

@@ -126,6 +126,10 @@ Trunk (unreleased changes)
    HADOOP-2587 Splits blocked by compactions cause region to be offline for
                duration of compaction. 
                (Bryan Duxbury via Stack)
+   HADOOP-2579 Initializing a new HTable object against a nonexistent table
+               throws a NoServerForRegionException instead of a
+               TableNotFoundException when a different table has been created
+               previously (Bryan Duxbury via Stack)
    
   IMPROVEMENTS
    HADOOP-2401 Add convenience put method that takes writable

+ 5 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java

@@ -409,6 +409,11 @@ public class HConnectionManager implements HConstants {
             metaLocation.getRegionInfo().getRegionName(), 
             metaKey, HConstants.LATEST_TIMESTAMP);
 
+          if (regionInfoRow == null) {
+            throw new TableNotFoundException("Table '" + tableName + 
+              "' does not exist.");
+          }
+
           // convert the MapWritable into a Map we can use
           SortedMap<Text, byte[]> results = 
             sortedMapFromMapWritable(regionInfoRow);

+ 4 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java

@@ -1074,6 +1074,10 @@ public class HRegion implements HConstants {
           key = new HStoreKey(closestKey, ts);
         }
       }
+
+      if (key == null) {
+        return null;
+      }
           
       // now that we've found our key, get the values
       TreeMap<Text, byte []> result = new TreeMap<Text, byte[]>();

+ 3 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java

@@ -1430,6 +1430,9 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
       // ask the region for all the data 
       Map<Text, byte[]> map = region.getClosestRowBefore(row, ts);
       // convert to a MapWritable
+      if (map == null) {
+        return null;
+      }
       for (Map.Entry<Text, byte []> es: map.entrySet()) {
         result.put(new HStoreKey(row, es.getKey()),
             new ImmutableBytesWritable(es.getValue()));

+ 5 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStore.java

@@ -191,7 +191,7 @@ public class HStore implements HConstants {
     private void internalGetFull(SortedMap<HStoreKey, byte []> map, HStoreKey key, 
         SortedMap<Text, byte []> results) {
 
-      if (map.isEmpty()) {
+      if (map.isEmpty() || key == null) {
         return;
       }
 
@@ -1596,6 +1596,10 @@ public class HStore implements HConstants {
     throws IOException {
     Map<Text, List<Long>> deletes = new HashMap<Text, List<Long>>();
     
+    if (key == null) {
+      return;
+    }
+    
     this.lock.readLock().lock();
     memcache.getFull(key, results);
     try {

+ 1 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTable.java

@@ -89,6 +89,7 @@ public class HTable implements HConstants {
     this.numRetries = conf.getInt("hbase.client.retries.number", 5);
     this.rand = new Random();
     this.batch = new AtomicReference<BatchUpdate>();
+    this.connection.locateRegion(tableName, EMPTY_START_ROW);
     tableDoesNotExist = false;
     closed = false;
   }

+ 48 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHTable.java

@@ -25,10 +25,15 @@ import java.util.TreeMap;
 
 import org.apache.hadoop.io.Text;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+
 /**
  * Tests HTable
  */
 public class TestHTable extends HBaseClusterTestCase implements HConstants {
+  private static final Log LOG = LogFactory.getLog(TestHTable.class);
   private static final HColumnDescriptor column =
     new HColumnDescriptor(COLUMN_FAMILY.toString());
 
@@ -134,4 +139,47 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants {
     // to be reloaded.
     
   }
+  
+  /**
+    * For HADOOP-2579
+    */
+  public void testTableNotFoundExceptionWithoutAnyTables() {
+    try {
+      new HTable(conf, new Text("notATable"));
+      fail("Should have thrown a TableNotFoundException");
+    } catch (TableNotFoundException e) {
+      // expected
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail("Should have thrown a TableNotFoundException instead of a " +
+        e.getClass());
+    }
+  }
+  
+  /**
+    * For HADOOP-2579
+    */
+  public void testTableNotFoundExceptionWithATable() {
+    try {
+      HColumnDescriptor column =
+        new HColumnDescriptor(COLUMN_FAMILY.toString());
+      HBaseAdmin admin = new HBaseAdmin(conf);
+      HTableDescriptor testTableADesc =
+        new HTableDescriptor("table");
+      testTableADesc.addFamily(column);
+      admin.createTable(testTableADesc);
+
+      // This should throw a TableNotFoundException, it has not been created
+      new HTable(conf, new Text("notATable"));
+      
+      fail("Should have thrown a TableNotFoundException");
+    } catch (TableNotFoundException e) {
+      // expected
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail("Should have thrown a TableNotFoundException instead of a " +
+        e.getClass());
+    }
+  }
+  
 }