You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2008/02/08 20:59:35 UTC

svn commit: r619985 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/ src/test/org/apache/hadoop/hbase/mapred/

Author: stack
Date: Fri Feb  8 11:59:21 2008
New Revision: 619985

URL: http://svn.apache.org/viewvc?rev=619985&view=rev
Log:
HBASE-406 Remove HTable and HConnection close methods

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnection.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnectionManager.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHTable.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri Feb  8 11:59:21 2008
@@ -32,6 +32,8 @@
                (Bryan Duxbury via Stack)
    HBASE-416   Add apache-style logging to REST server and add setting log
                level, etc.
+   HBASE-406   Remove HTable and HConnection close methods
+               (Bryan Duxbury via Stack)
 
 
 Branch 0.1

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnection.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnection.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnection.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnection.java Fri Feb  8 11:59:21 2008
@@ -86,10 +86,4 @@
    */
   public HRegionInterface getHRegionConnection(HServerAddress regionServer)
   throws IOException;
-  
-  /**
-   * Discard all the information about this table
-   * @param tableName the name of the table to close
-   */
-  public void close(Text tableName);
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnectionManager.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnectionManager.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HConnectionManager.java Fri Feb  8 11:59:21 2008
@@ -89,9 +89,6 @@
     synchronized (HBASE_INSTANCES) {
       TableServers instance =
         HBASE_INSTANCES.remove(conf.get(HBASE_DIR, DEFAULT_HBASE_DIR));
-      if (instance != null) {
-        instance.closeAll();
-      }
     }    
   }
   
@@ -112,9 +109,6 @@
     private final Integer userRegionLock = new Integer(0);
         
     private volatile HBaseConfiguration conf;
-
-    // Set of closed tables
-    private Set<Text> closedTables;
     
     // Known region HServerAddress.toString() -> HRegionInterface 
     private Map<String, HRegionInterface> servers;
@@ -153,7 +147,6 @@
 
       this.cachedRegionLocations = 
         new ConcurrentHashMap<Text, SortedMap<Text, HRegionLocation>>();
-      this.closedTables = Collections.synchronizedSet(new HashSet<Text>());
       this.servers = new ConcurrentHashMap<String, HRegionInterface>();
     }
     
@@ -658,43 +651,6 @@
       return server;
     }
 
-    /** {@inheritDoc} */
-    public void close(Text tableName) {
-      if (tableName == null || tableName.getLength() == 0) {
-        throw new IllegalArgumentException(
-            "table name cannot be null or zero length");
-      }
-            
-      if (closedTables.contains(tableName)) {
-        // Table already closed. Ignore it.
-        return;
-      }
-
-      closedTables.add(tableName);
-
-      if (cachedRegionLocations.containsKey(tableName)) {
-        SortedMap<Text, HRegionLocation> tableServers = 
-          cachedRegionLocations.remove(tableName);
-
-        // Shut down connections to the HRegionServers
-        synchronized (this.servers) {
-          for (HRegionLocation r: tableServers.values()) {
-            this.servers.remove(r.getServerAddress().toString());
-          }
-        }
-      }
-    }
-    
-    /** Convenience method for closing all open tables.*/
-    void closeAll() {
-      this.closed = true;
-      ArrayList<Text> tables = 
-        new ArrayList<Text>(cachedRegionLocations.keySet());
-      for (Text tableName: tables) {
-        close(tableName);
-      }
-    }
-    
     /*
      * Repeatedly try to find the root region by asking the master for where it is
      * @return HRegionLocation for root region if found

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/HTable.java Fri Feb  8 11:59:21 2008
@@ -61,17 +61,6 @@
   
   // For row mutation operations
   
-  protected volatile boolean closed;
-
-  protected void checkClosed() {
-    if (tableDoesNotExist) {
-      throw new IllegalStateException("table does not exist: " + tableName);
-    }
-    if (closed) {
-      throw new IllegalStateException("table is closed");
-    }
-  }
-  
   /**
    * Creates an object to access a HBase table
    * 
@@ -80,8 +69,6 @@
    * @throws IOException
    */
   public HTable(HBaseConfiguration conf, Text tableName) throws IOException {
-    closed = true;
-    tableDoesNotExist = true;
     this.connection = HConnectionManager.getConnection(conf);
     this.tableName = tableName;
     this.pause = conf.getLong("hbase.client.pause", 10 * 1000);
@@ -89,8 +76,6 @@
     this.rand = new Random();
     this.batch = new AtomicReference<BatchUpdate>();
     this.connection.locateRegion(tableName, EMPTY_START_ROW);
-    tableDoesNotExist = false;
-    closed = false;
   }
 
   /**
@@ -99,7 +84,6 @@
    * @return Location of row.
    */
   HRegionLocation getRegionLocation(Text row) throws IOException {
-    checkClosed();
     return this.connection.locateRegion(this.tableName, row);
   }
 
@@ -110,7 +94,6 @@
    * @return Location of row.
    */
   HRegionLocation getRegionLocation(Text row, boolean reload) throws IOException {
-    checkClosed();
     return reload?
       this.connection.relocateRegion(this.tableName, row):
       this.connection.locateRegion(tableName, row);
@@ -119,21 +102,8 @@
 
   /** @return the connection */
   public HConnection getConnection() {
-    checkClosed();
     return connection;
   }
-
-  /**
-   * Releases resources associated with this table. After calling close(), all
-   * other methods will throw an IllegalStateException
-   */
-  public synchronized void close() {
-    if (!closed) {
-      closed = true;
-      batch.set(null);
-      connection.close(tableName);
-    }
-  }
   
   /**
    * Verifies that no update is in progress
@@ -193,7 +163,6 @@
    */
   @SuppressWarnings("null")
   public Text[] getStartKeys() throws IOException {
-    checkClosed();
     List<Text> keyList = new ArrayList<Text>();
 
     long scannerId = -1L;
@@ -277,8 +246,6 @@
    * @throws IOException
    */
    public byte[] get(Text row, final Text column) throws IOException {
-     checkClosed();
-     
      return getRegionServerWithRetries(new ServerCallable<byte[]>(row){
        public byte[] call() throws IOException {
          return server.get(location.getRegionInfo().getRegionName(), row, column);
@@ -297,7 +264,6 @@
    */
   public byte[][] get(final Text row, final Text column, final int numVersions) 
   throws IOException {
-    checkClosed();
     byte [][] values = null;
 
     values = getRegionServerWithRetries(new ServerCallable<byte[][]>(row) {
@@ -331,7 +297,6 @@
   public byte[][] get(final Text row, final Text column, final long timestamp, 
     final int numVersions)
   throws IOException {
-    checkClosed();
     byte [][] values = null;
 
     values = getRegionServerWithRetries(new ServerCallable<byte[][]>(row) {
@@ -372,7 +337,6 @@
    */
   public SortedMap<Text, byte[]> getRow(final Text row, final long ts) 
   throws IOException {
-    checkClosed();
     HbaseMapWritable value = null;
          
     value = getRegionServerWithRetries(new ServerCallable<HbaseMapWritable>(row) {
@@ -426,7 +390,7 @@
    * @throws IOException
    */
   public HScannerInterface obtainScanner(Text[] columns, Text startRow,
-      long timestamp)
+    long timestamp)
   throws IOException {
     return obtainScanner(columns, startRow, timestamp, null);
   }
@@ -446,7 +410,7 @@
    * @throws IOException
    */
   public HScannerInterface obtainScanner(Text[] columns, Text startRow,
-      RowFilterInterface filter)
+    RowFilterInterface filter)
   throws IOException { 
     return obtainScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter);
   }
@@ -469,7 +433,7 @@
    * @throws IOException
    */
   public HScannerInterface obtainScanner(final Text[] columns,
-      final Text startRow, final Text stopRow)
+    final Text startRow, final Text stopRow)
   throws IOException {
     return obtainScanner(columns, startRow, stopRow,
       HConstants.LATEST_TIMESTAMP);
@@ -494,7 +458,7 @@
    * @throws IOException
    */
   public HScannerInterface obtainScanner(final Text[] columns,
-      final Text startRow, final Text stopRow, final long timestamp)
+    final Text startRow, final Text stopRow, final long timestamp)
   throws IOException {
     return obtainScanner(columns, startRow, timestamp,
       new WhileMatchRowFilter(new StopRowFilter(stopRow)));
@@ -516,9 +480,8 @@
    * @throws IOException
    */
   public HScannerInterface obtainScanner(Text[] columns,
-      Text startRow, long timestamp, RowFilterInterface filter)
+    Text startRow, long timestamp, RowFilterInterface filter)
   throws IOException {
-    checkClosed();
     return new ClientScanner(columns, startRow, timestamp, filter);
   }
 
@@ -566,7 +529,6 @@
    */
   @Deprecated
   public synchronized long startUpdate(final Text row) {
-    checkClosed();
     updateInProgress(false);
     batch.set(new BatchUpdate(row));
     return 1;
@@ -582,7 +544,6 @@
    */
   @Deprecated
   public void put(long lockid, Text column, byte val[]) {
-    checkClosed();
     if (lockid != 1) {
       throw new IllegalArgumentException("Invalid lock id!");
     }
@@ -617,7 +578,6 @@
    */
   @Deprecated
   public void delete(long lockid, Text column) {
-    checkClosed();
     if (lockid != 1) {
       throw new IllegalArgumentException("Invalid lock id!");
     }
@@ -645,8 +605,6 @@
    */
   public void deleteAll(final Text row, final Text column, final long ts)
   throws IOException {
-    checkClosed();
-          
     getRegionServerWithRetries(new ServerCallable<Boolean>(row) {
       public Boolean call() throws IOException {
         server.deleteAll(location.getRegionInfo().getRegionName(), row, 
@@ -664,8 +622,6 @@
    * @throws IOException
    */
   public void deleteAll(final Text row, final long ts) throws IOException {
-    checkClosed();
-    
     getRegionServerWithRetries(new ServerCallable<Boolean>(row){
       public Boolean call() throws IOException {
         server.deleteAll(location.getRegionInfo().getRegionName(), row, ts);
@@ -696,8 +652,6 @@
   public void deleteFamily(final Text row, final Text family, 
     final long timestamp)
   throws IOException {
-    checkClosed();
-    
     getRegionServerWithRetries(new ServerCallable<Boolean>(row){
       public Boolean call() throws IOException {
         server.deleteFamily(location.getRegionInfo().getRegionName(), row, 
@@ -730,7 +684,6 @@
    */
   @Deprecated
   public synchronized void abort(long lockid) {
-    checkClosed();
     if (lockid != 1) {
       throw new IllegalArgumentException("Invalid lock id!");
     }
@@ -783,7 +736,6 @@
    */ 
   public synchronized void commit(final BatchUpdate batchUpdate) 
   throws IOException {
-    checkClosed();
     getRegionServerWithRetries(
       new ServerCallable<Boolean>(batchUpdate.getRow()){
         public Boolean call() throws IOException {
@@ -843,8 +795,6 @@
      * Returns false if there are no more scanners.
      */
     private boolean nextScanner() throws IOException {
-      checkClosed();
-      
       // close the previous scanner if it's open
       if (this.scannerId != -1L) {
         this.server.close(this.scannerId);
@@ -923,7 +873,6 @@
     /** {@inheritDoc} */
     public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
     throws IOException {
-      checkClosed();
       if (this.closed) {
         return false;
       }
@@ -942,7 +891,7 @@
           key.setVersion(k.getTimestamp());
           key.setColumn(EMPTY_COLUMN);
           results.put(k.getColumn(),
-              ((ImmutableBytesWritable) e.getValue()).get());
+            ((ImmutableBytesWritable) e.getValue()).get());
         }
       }
       return values == null ? false : values.size() != 0;
@@ -952,7 +901,6 @@
      * {@inheritDoc}
      */
     public void close() throws IOException {
-      checkClosed();
       if (scannerId != -1L) {
         try {
           server.close(scannerId);
@@ -1063,5 +1011,10 @@
       }
     }
     return null;    
+  }
+  
+  @Deprecated
+  public void close() {
+    // do nothing...
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/DisabledTestScanner2.java Fri Feb  8 11:59:21 2008
@@ -117,7 +117,6 @@
       assertEquals(count, 3);
     } finally {
       scanner.close();
-      table.close();
     }
   }
 
@@ -128,20 +127,16 @@
     Text tableName = new Text(getName());
     createTable(new HBaseAdmin(this.conf), tableName);
     HTable table = new HTable(this.conf, tableName);
-    try {
-      final String lastKey = "aac";
-      addContent(new HTableIncommon(table), FIRST_COLKEY + ":");
-      HScannerInterface scanner =
-        table.obtainScanner(new Text [] {new Text(FIRST_COLKEY + ":")},
-            HConstants.EMPTY_START_ROW, new Text(lastKey));
-      for (Map.Entry<HStoreKey, SortedMap<Text, byte []>> e: scanner) {
-        if(e.getKey().getRow().toString().compareTo(lastKey) >= 0) {
-          LOG.info(e.getKey());
-          fail();
-        }
+    final String lastKey = "aac";
+    addContent(new HTableIncommon(table), FIRST_COLKEY + ":");
+    HScannerInterface scanner =
+      table.obtainScanner(new Text [] {new Text(FIRST_COLKEY + ":")},
+          HConstants.EMPTY_START_ROW, new Text(lastKey));
+    for (Map.Entry<HStoreKey, SortedMap<Text, byte []>> e: scanner) {
+      if(e.getKey().getRow().toString().compareTo(lastKey) >= 0) {
+        LOG.info(e.getKey());
+        fail();
       }
-    } finally {
-      table.close();
     }
   }
   
@@ -150,16 +145,12 @@
    */
   public void testIterator() throws Exception {
     HTable table = new HTable(this.conf, HConstants.ROOT_TABLE_NAME);
-    try {
-      HScannerInterface scanner =
-        table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY,
-            HConstants.EMPTY_START_ROW);
-      for (Map.Entry<HStoreKey, SortedMap<Text, byte []>> e: scanner) {
-        assertNotNull(e.getKey());
-        assertNotNull(e.getValue());
-      }
-    } finally {
-      table.close();
+    HScannerInterface scanner =
+      table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY,
+          HConstants.EMPTY_START_ROW);
+    for (Map.Entry<HStoreKey, SortedMap<Text, byte []>> e: scanner) {
+      assertNotNull(e.getKey());
+      assertNotNull(e.getValue());
     }
   }
 
@@ -174,22 +165,18 @@
     Text tableName = new Text(getName());
     createTable(admin, tableName);
     HTable table = new HTable(this.conf, tableName);
-    try {
-      // Add a row to columns without qualifiers and then two with.  Make one
-      // numbers only so easy to find w/ a regex.
-      long id = table.startUpdate(new Text(getName()));
-      final String firstColkeyFamily = Character.toString(FIRST_COLKEY) + ":";
-      table.put(id, new Text(firstColkeyFamily + getName()), GOOD_BYTES);
-      table.put(id, new Text(firstColkeyFamily + "22222"), GOOD_BYTES);
-      table.put(id, new Text(firstColkeyFamily), GOOD_BYTES);
-      table.commit(id);
-      // Now do a scan using a regex for a column name.
-      checkRegexingScanner(table, firstColkeyFamily + "\\d+");
-      // Do a new scan that only matches on column family.
-      checkRegexingScanner(table, firstColkeyFamily + "$");
-    } finally {
-      table.close();
-    }
+    // Add a row to columns without qualifiers and then two with.  Make one
+    // numbers only so easy to find w/ a regex.
+    long id = table.startUpdate(new Text(getName()));
+    final String firstColkeyFamily = Character.toString(FIRST_COLKEY) + ":";
+    table.put(id, new Text(firstColkeyFamily + getName()), GOOD_BYTES);
+    table.put(id, new Text(firstColkeyFamily + "22222"), GOOD_BYTES);
+    table.put(id, new Text(firstColkeyFamily), GOOD_BYTES);
+    table.commit(id);
+    // Now do a scan using a regex for a column name.
+    checkRegexingScanner(table, firstColkeyFamily + "\\d+");
+    // Do a new scan that only matches on column family.
+    checkRegexingScanner(table, firstColkeyFamily + "$");
   }
   
   /*
@@ -199,8 +186,9 @@
    * @param regexColumnname
    * @throws IOException
    */
-  private void checkRegexingScanner(final HTable table,
-      final String regexColumnname) throws IOException {
+  private void checkRegexingScanner(final HTable table, 
+    final String regexColumnname) 
+  throws IOException {
     Text [] regexCol = new Text [] {new Text(regexColumnname)};
     HScannerInterface scanner =
       table.obtainScanner(regexCol, HConstants.EMPTY_START_ROW);
@@ -234,22 +222,18 @@
     
     // Enter data
     HTable table = new HTable(conf, tableName);
-    try {
-      for (char i = FIRST_ROWKEY; i <= LAST_ROWKEY; i++) {
-        Text rowKey = new Text(new String(new char[] { i }));
-        long lockID = table.startUpdate(rowKey);
-        for (char j = 0; j < colKeys.length; j++) {
-          table.put(lockID, colKeys[j], (i >= FIRST_BAD_RANGE_ROWKEY && 
-              i <= LAST_BAD_RANGE_ROWKEY)? BAD_BYTES : GOOD_BYTES);
-        }
-        table.commit(lockID);
+    for (char i = FIRST_ROWKEY; i <= LAST_ROWKEY; i++) {
+      Text rowKey = new Text(new String(new char[] { i }));
+      long lockID = table.startUpdate(rowKey);
+      for (char j = 0; j < colKeys.length; j++) {
+        table.put(lockID, colKeys[j], (i >= FIRST_BAD_RANGE_ROWKEY && 
+            i <= LAST_BAD_RANGE_ROWKEY)? BAD_BYTES : GOOD_BYTES);
       }
-
-      regExpFilterTest(table, colKeys);
-      rowFilterSetTest(table, colKeys);
-    } finally {
-      table.close();
+      table.commit(lockID);
     }
+
+    regExpFilterTest(table, colKeys);
+    rowFilterSetTest(table, colKeys);
   }
   
   /**
@@ -292,7 +276,7 @@
   }
   
   private void rowFilterSetTest(HTable table, Text[] colKeys) 
-    throws Exception {
+  throws Exception {
     // Get the filter.  The RegExpRowFilter used should filter out vowels and 
     // the WhileMatchRowFilter(StopRowFilter) should filter out all rows 
     // greater than or equal to 'r'.
@@ -312,21 +296,21 @@
   
   private void iterateOnScanner(HScannerInterface scanner, String regexToMatch)
   throws Exception {
-      // A pattern that will only match rows that should not have been filtered.
-      Pattern p = Pattern.compile(regexToMatch);
-      
-      try {
-        // Use the scanner to ensure all results match the above pattern.
-        HStoreKey rowKey = new HStoreKey();
-        TreeMap<Text, byte[]> columns = new TreeMap<Text, byte[]>();
-        while (scanner.next(rowKey, columns)) {
-          String key = rowKey.getRow().toString();
-          assertTrue("Shouldn't have extracted '" + key + "'", 
-            p.matcher(key).matches());
-        }
-      } finally {
-        scanner.close();
+    // A pattern that will only match rows that should not have been filtered.
+    Pattern p = Pattern.compile(regexToMatch);
+    
+    try {
+      // Use the scanner to ensure all results match the above pattern.
+      HStoreKey rowKey = new HStoreKey();
+      TreeMap<Text, byte[]> columns = new TreeMap<Text, byte[]>();
+      while (scanner.next(rowKey, columns)) {
+        String key = rowKey.getRow().toString();
+        assertTrue("Shouldn't have extracted '" + key + "'", 
+          p.matcher(key).matches());
       }
+    } finally {
+      scanner.close();
+    }
   }
   
   /**
@@ -337,43 +321,39 @@
    */
   public void testSplitDeleteOneAddTwoRegions() throws IOException {
     HTable metaTable = new HTable(conf, HConstants.META_TABLE_NAME);
+    // First add a new table.  Its intial region will be added to META region.
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    Text tableName = new Text(getName());
+    admin.createTable(new HTableDescriptor(tableName.toString()));
+    List<HRegionInfo> regions = scan(metaTable);
+    assertEquals("Expected one region", 1, regions.size());
+    HRegionInfo region = regions.get(0);
+    assertTrue("Expected region named for test",
+        region.getRegionName().toString().startsWith(getName()));
+    // Now do what happens at split time; remove old region and then add two
+    // new ones in its place.
+    removeRegionFromMETA(metaTable, region.getRegionName());
+    HTableDescriptor desc = region.getTableDesc();
+    Path homedir = new Path(getName());
+    List<HRegion> newRegions = new ArrayList<HRegion>(2);
+    newRegions.add(HRegion.createHRegion(
+        new HRegionInfo(desc, null, new Text("midway")),
+        homedir, this.conf));
+    newRegions.add(HRegion.createHRegion(
+        new HRegionInfo(desc, new Text("midway"), null),
+        homedir, this.conf));
     try {
-      // First add a new table.  Its intial region will be added to META region.
-      HBaseAdmin admin = new HBaseAdmin(conf);
-      Text tableName = new Text(getName());
-      admin.createTable(new HTableDescriptor(tableName.toString()));
-      List<HRegionInfo> regions = scan(metaTable);
-      assertEquals("Expected one region", 1, regions.size());
-      HRegionInfo region = regions.get(0);
-      assertTrue("Expected region named for test",
-          region.getRegionName().toString().startsWith(getName()));
-      // Now do what happens at split time; remove old region and then add two
-      // new ones in its place.
-      removeRegionFromMETA(metaTable, region.getRegionName());
-      HTableDescriptor desc = region.getTableDesc();
-      Path homedir = new Path(getName());
-      List<HRegion> newRegions = new ArrayList<HRegion>(2);
-      newRegions.add(HRegion.createHRegion(
-          new HRegionInfo(desc, null, new Text("midway")),
-          homedir, this.conf));
-      newRegions.add(HRegion.createHRegion(
-          new HRegionInfo(desc, new Text("midway"), null),
-          homedir, this.conf));
-      try {
-        for (HRegion r : newRegions) {
-          addRegionToMETA(metaTable, r, this.cluster.getHMasterAddress(),
-              -1L);
-        }
-        regions = scan(metaTable);
-        assertEquals("Should be two regions only", 2, regions.size());
-      } finally {
-        for (HRegion r : newRegions) {
-          r.close();
-          r.getLog().closeAndDelete();
-        }
+      for (HRegion r : newRegions) {
+        addRegionToMETA(metaTable, r, this.cluster.getHMasterAddress(),
+            -1L);
       }
+      regions = scan(metaTable);
+      assertEquals("Should be two regions only", 2, regions.size());
     } finally {
-      metaTable.close();
+      for (HRegion r : newRegions) {
+        r.close();
+        r.getLog().closeAndDelete();
+      }
     }
   }
   

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHTable.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHTable.java Fri Feb  8 11:59:21 2008
@@ -110,20 +110,6 @@
       s.close();
     }
     
-    // Close table A and note how A becomes inaccessable
-    
-    a.close();
-    
-    try {
-      a.get(row, COLUMN_FAMILY);
-      fail();
-    } catch (IllegalStateException e) {
-      // expected
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail();
-    }
-    
     // Opening a new connection to A will cause the tables to be reloaded
 
     try {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestLogRolling.java Fri Feb  8 11:59:21 2008
@@ -131,37 +131,28 @@
     // When the META table can be opened, the region servers are running
     HTable meta = new HTable(conf, HConstants.META_TABLE_NAME);
     
-    try {
+    // Create the test table and open it
+    HTableDescriptor desc = new HTableDescriptor(tableName);
+    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
+    HBaseAdmin admin = new HBaseAdmin(conf);
+    admin.createTable(desc);
+    HTable table = new HTable(conf, new Text(tableName));
 
-      // Create the test table and open it
-      HTableDescriptor desc = new HTableDescriptor(tableName);
-      desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
-      HBaseAdmin admin = new HBaseAdmin(conf);
-      admin.createTable(desc);
-      HTable table = new HTable(conf, new Text(tableName));
+    for (int i = 1; i <= 2048; i++) {    // 2048 writes should cause 8 log rolls
+      long lockid =
+        table.startUpdate(new Text("row" + String.format("%1$04d", i)));
+      table.put(lockid, HConstants.COLUMN_FAMILY, value);
+      table.commit(lockid);
 
-      try {
-        for (int i = 1; i <= 2048; i++) {    // 2048 writes should cause 8 log rolls
-          long lockid =
-            table.startUpdate(new Text("row" + String.format("%1$04d", i)));
-          table.put(lockid, HConstants.COLUMN_FAMILY, value);
-          table.commit(lockid);
+      if (i % 256 == 0) {
+        // After every 256 writes sleep to let the log roller run
 
-          if (i % 256 == 0) {
-            // After every 256 writes sleep to let the log roller run
-
-            try {
-              Thread.sleep(2000);
-            } catch (InterruptedException e) {
-              // continue
-            }
-          }
+        try {
+          Thread.sleep(2000);
+        } catch (InterruptedException e) {
+          // continue
         }
-      } finally {
-        table.close();
       }
-    } finally {
-      meta.close();
     }
   }
   

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java?rev=619985&r1=619984&r2=619985&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java Fri Feb  8 11:59:21 2008
@@ -218,53 +218,49 @@
     // insert some data into the test table
     HTable table = new HTable(conf, new Text(SINGLE_REGION_TABLE_NAME));
 
-    try {
-      for(int i = 0; i < values.length; i++) {
-        long lockid = table.startUpdate(new Text("row_"
-            + String.format("%1$05d", i)));
-
-        try {
-          table.put(lockid, TEXT_INPUT_COLUMN, values[i]);
-          table.commit(lockid, System.currentTimeMillis());
-          lockid = -1;
-        } finally {
-          if (lockid != -1)
-            table.abort(lockid);
-        }
+    for(int i = 0; i < values.length; i++) {
+      long lockid = table.startUpdate(new Text("row_"
+          + String.format("%1$05d", i)));
+
+      try {
+        table.put(lockid, TEXT_INPUT_COLUMN, values[i]);
+        table.commit(lockid, System.currentTimeMillis());
+        lockid = -1;
+      } finally {
+        if (lockid != -1)
+          table.abort(lockid);
       }
+    }
 
-      LOG.info("Print table contents before map/reduce for " +
-        SINGLE_REGION_TABLE_NAME);
-      scanTable(SINGLE_REGION_TABLE_NAME, true);
+    LOG.info("Print table contents before map/reduce for " +
+      SINGLE_REGION_TABLE_NAME);
+    scanTable(SINGLE_REGION_TABLE_NAME, true);
 
-      @SuppressWarnings("deprecation")
-      MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
+    @SuppressWarnings("deprecation")
+    MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
 
-      try {
-        JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
-        jobConf.setJobName("process column contents");
-        jobConf.setNumMapTasks(1);
-        jobConf.setNumReduceTasks(1);
-
-        TableMap.initJob(SINGLE_REGION_TABLE_NAME, INPUT_COLUMN, 
-            ProcessContentsMapper.class, jobConf);
-
-        TableReduce.initJob(SINGLE_REGION_TABLE_NAME,
-            IdentityTableReduce.class, jobConf);
-        LOG.info("Started " + SINGLE_REGION_TABLE_NAME);
-        JobClient.runJob(jobConf);
-
-        LOG.info("Print table contents after map/reduce for " +
-          SINGLE_REGION_TABLE_NAME);
-      scanTable(SINGLE_REGION_TABLE_NAME, true);
+    try {
+      JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
+      jobConf.setJobName("process column contents");
+      jobConf.setNumMapTasks(1);
+      jobConf.setNumReduceTasks(1);
+
+      TableMap.initJob(SINGLE_REGION_TABLE_NAME, INPUT_COLUMN, 
+          ProcessContentsMapper.class, jobConf);
+
+      TableReduce.initJob(SINGLE_REGION_TABLE_NAME,
+          IdentityTableReduce.class, jobConf);
+      LOG.info("Started " + SINGLE_REGION_TABLE_NAME);
+      JobClient.runJob(jobConf);
 
-      // verify map-reduce results
-      verify(SINGLE_REGION_TABLE_NAME);
-      } finally {
-        mrCluster.shutdown();
-      }
+      LOG.info("Print table contents after map/reduce for " +
+        SINGLE_REGION_TABLE_NAME);
+    scanTable(SINGLE_REGION_TABLE_NAME, true);
+
+    // verify map-reduce results
+    verify(SINGLE_REGION_TABLE_NAME);
     } finally {
-      table.close();
+      mrCluster.shutdown();
     }
   }
   
@@ -287,34 +283,31 @@
     
     // Verify table indeed has multiple regions
     HTable table = new HTable(conf, new Text(MULTI_REGION_TABLE_NAME));
-    try {
-      Text[] startKeys = table.getStartKeys();
-      assertTrue(startKeys.length > 1);
 
-      @SuppressWarnings("deprecation")
-      MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
+    Text[] startKeys = table.getStartKeys();
+    assertTrue(startKeys.length > 1);
 
-      try {
-        JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
-        jobConf.setJobName("process column contents");
-        jobConf.setNumMapTasks(2);
-        jobConf.setNumReduceTasks(1);
-
-        TableMap.initJob(MULTI_REGION_TABLE_NAME, INPUT_COLUMN, 
-            ProcessContentsMapper.class, jobConf);
-
-        TableReduce.initJob(MULTI_REGION_TABLE_NAME,
-            IdentityTableReduce.class, jobConf);
-        LOG.info("Started " + MULTI_REGION_TABLE_NAME);
-        JobClient.runJob(jobConf);
+    @SuppressWarnings("deprecation")
+    MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
 
-        // verify map-reduce results
-        verify(MULTI_REGION_TABLE_NAME);
-      } finally {
-        mrCluster.shutdown();
-      }
+    try {
+      JobConf jobConf = new JobConf(conf, TestTableMapReduce.class);
+      jobConf.setJobName("process column contents");
+      jobConf.setNumMapTasks(2);
+      jobConf.setNumReduceTasks(1);
+
+      TableMap.initJob(MULTI_REGION_TABLE_NAME, INPUT_COLUMN, 
+          ProcessContentsMapper.class, jobConf);
+
+      TableReduce.initJob(MULTI_REGION_TABLE_NAME,
+          IdentityTableReduce.class, jobConf);
+      LOG.info("Started " + MULTI_REGION_TABLE_NAME);
+      JobClient.runJob(jobConf);
+
+      // verify map-reduce results
+      verify(MULTI_REGION_TABLE_NAME);
     } finally {
-      table.close();
+      mrCluster.shutdown();
     }
   }