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 2012/04/23 22:13:29 UTC

svn commit: r1329418 [1/3] - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/metrics/histogram/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ test/java...

Author: stack
Date: Mon Apr 23 20:13:28 2012
New Revision: 1329418

URL: http://svn.apache.org/viewvc?rev=1329418&view=rev
Log:
HBASE-5833 0.92 build has been failing pretty consistently on TestMasterFailover

Added:
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestMinVersions.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestResettingCounters.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestCloseRegionHandler.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestMergeTool.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java Mon Apr 23 20:13:28 2012
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -577,16 +578,17 @@ public class LruBlockCache implements Bl
    */
   private static class EvictionThread extends HasThread {
     private WeakReference<LruBlockCache> cache;
+    private boolean go = true;
 
     public EvictionThread(LruBlockCache cache) {
-      super("LruBlockCache.EvictionThread");
+      super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");
       setDaemon(true);
       this.cache = new WeakReference<LruBlockCache>(cache);
     }
 
     @Override
     public void run() {
-      while(true) {
+      while (this.go) {
         synchronized(this) {
           try {
             this.wait();
@@ -597,11 +599,17 @@ public class LruBlockCache implements Bl
         cache.evict();
       }
     }
+
     public void evict() {
       synchronized(this) {
         this.notify(); // FindBugs NN_NAKED_NOTIFY
       }
     }
+
+    void shutdown() {
+      this.go = false;
+      interrupt();
+    }
   }
 
   /*
@@ -725,6 +733,14 @@ public class LruBlockCache implements Bl
 
   public void shutdown() {
     this.scheduleThreadPool.shutdown();
+    for (int i = 0; i < 10; i++) {
+      if (!this.scheduleThreadPool.isShutdown()) Threads.sleep(10);
+    }
+    if (!this.scheduleThreadPool.isShutdown()) {
+      List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
+      LOG.debug("Still running " + runnables);
+    }
+    this.evictionThread.shutdown();
   }
 
   /** Clears the cache. Used in tests. */

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/metrics/histogram/ExponentiallyDecayingSample.java Mon Apr 23 20:13:28 2012
@@ -43,8 +43,8 @@ public class ExponentiallyDecayingSample
   private static final long RESCALE_THRESHOLD = TimeUnit.HOURS.toNanos(1);
 
   private static final ScheduledExecutorService TICK_SERVICE = 
-      Executors.newScheduledThreadPool(1, 
-          getNamedDaemonThreadFactory("decayingSampleTick"));
+    Executors.newScheduledThreadPool(1, 
+      getNamedDaemonThreadFactory(Thread.currentThread().getName() + ".decayingSampleTick."));
 
   private static volatile long CURRENT_TICK = 
       TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis());

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Mon Apr 23 20:13:28 2012
@@ -3548,7 +3548,11 @@ public class HRegion implements HeapSize
    * bootstrap code in the HMaster constructor.
    * Note, this method creates an {@link HLog} for the created region. It
    * needs to be closed explicitly.  Use {@link HRegion#getLog()} to get
-   * access.
+   * access.  <b>When done with a region created using this method, you will
+   * need to explicitly close the {@link HLog} it created too; it will not be
+   * done for you.  Not closing the log will leave at least a daemon thread
+   * running.</b>  Call {@link #closeHRegion(HRegion)} and it will do
+   * necessary cleanup for you.
    * @param info Info for region to create.
    * @param rootDir Root directory for HBase instance
    * @param conf
@@ -3564,6 +3568,23 @@ public class HRegion implements HeapSize
   }
 
   /**
+   * This will do the necessary cleanup a call to {@link #createHRegion(HRegionInfo, Path, Configuration, HTableDescriptor)}
+   * requires.  This method will close the region and then close its
+   * associated {@link HLog} file.  You use it if you call the other createHRegion,
+   * the one that takes an {@link HLog} instance but don't be surprised by the
+   * call to the {@link HLog#closeAndDelete()} on the {@link HLog} the
+   * HRegion was carrying.
+   * @param r
+   * @throws IOException
+   */
+  public static void closeHRegion(final HRegion r) throws IOException {
+    if (r == null) return;
+    r.close();
+    if (r.getLog() == null) return;
+    r.getLog().closeAndDelete();
+  }
+
+  /**
    * Convenience method creating new HRegions. Used by createTable.
    * The {@link HLog} for the created region needs to be closed explicitly.
    * Use {@link HRegion#getLog()} to get access.

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java Mon Apr 23 20:13:28 2012
@@ -221,13 +221,17 @@ public class JVMClusterUtil {
       final List<RegionServerThread> regionservers) {
     LOG.debug("Shutting down HBase Cluster");
     if (masters != null) {
+      // Do backups first.
+      JVMClusterUtil.MasterThread activeMaster = null;
       for (JVMClusterUtil.MasterThread t : masters) {
-        if (t.master.isActiveMaster()) {
-          t.master.shutdown();
-        } else {
+        if (!t.master.isActiveMaster()) {
           t.master.stopMaster();
+        } else {
+          activeMaster = t;
         }
       }
+      // Do active after.
+      if (activeMaster != null) activeMaster.master.shutdown();
     }
     // regionServerThreads can never be null because they are initialized when
     // the class is constructed.

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java Mon Apr 23 20:13:28 2012
@@ -170,6 +170,16 @@ public abstract class HBaseTestCase exte
       );
     }
 
+  /**
+   * You must call close on the returned region and then close on the log file
+   * it created. Do {@link HRegion#close()} followed by {@link HRegion#getLog()}
+   * and on it call close.
+   * @param desc
+   * @param startKey
+   * @param endKey
+   * @return An {@link HRegion}
+   * @throws IOException
+   */
   protected HRegion createNewHRegion(HTableDescriptor desc, byte [] startKey,
       byte [] endKey)
   throws IOException {
@@ -673,6 +683,11 @@ public abstract class HBaseTestCase exte
     }
   }
 
+  /**
+   * You must call {@link #closeRootAndMeta()} when done after calling this
+   * method. It does cleanup.
+   * @throws IOException
+   */
   protected void createRootAndMetaRegions() throws IOException {
     root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO, testDir,
         conf, HTableDescriptor.ROOT_TABLEDESC);

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPrefixFilter.java Mon Apr 23 20:13:28 2012
@@ -50,52 +50,56 @@ public class TestColumnPrefixFilter {
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
     HRegion region = HRegion.createHRegion(info, TEST_UTIL.
       getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
-
-    List<String> rows = generateRandomWords(100, "row");
-    List<String> columns = generateRandomWords(10000, "column");
-    long maxTimestamp = 2;
-
-    List<KeyValue> kvList = new ArrayList<KeyValue>();
-
-    Map<String, List<KeyValue>> prefixMap = new HashMap<String,
-        List<KeyValue>>();
-
-    prefixMap.put("p", new ArrayList<KeyValue>());
-    prefixMap.put("s", new ArrayList<KeyValue>());
-
-    String valueString = "ValueString";
-
-    for (String row: rows) {
-      Put p = new Put(Bytes.toBytes(row));
-      p.setWriteToWAL(false);
-      for (String column: columns) {
-        for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
-          KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
-              valueString);
-          p.add(kv);
-          kvList.add(kv);
-          for (String s: prefixMap.keySet()) {
-            if (column.startsWith(s)) {
-              prefixMap.get(s).add(kv);
+    try {
+      List<String> rows = generateRandomWords(100, "row");
+      List<String> columns = generateRandomWords(10000, "column");
+      long maxTimestamp = 2;
+
+      List<KeyValue> kvList = new ArrayList<KeyValue>();
+
+      Map<String, List<KeyValue>> prefixMap = new HashMap<String,
+          List<KeyValue>>();
+
+      prefixMap.put("p", new ArrayList<KeyValue>());
+      prefixMap.put("s", new ArrayList<KeyValue>());
+
+      String valueString = "ValueString";
+
+      for (String row: rows) {
+        Put p = new Put(Bytes.toBytes(row));
+        p.setWriteToWAL(false);
+        for (String column: columns) {
+          for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+            KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+                valueString);
+            p.add(kv);
+            kvList.add(kv);
+            for (String s: prefixMap.keySet()) {
+              if (column.startsWith(s)) {
+                prefixMap.get(s).add(kv);
+              }
             }
           }
         }
+        region.put(p);
       }
-      region.put(p);
-    }
 
-    ColumnPrefixFilter filter;
-    Scan scan = new Scan();
-    scan.setMaxVersions();
-    for (String s: prefixMap.keySet()) {
-      filter = new ColumnPrefixFilter(Bytes.toBytes(s));
-
-      scan.setFilter(filter);
-
-      InternalScanner scanner = region.getScanner(scan);
-      List<KeyValue> results = new ArrayList<KeyValue>();
-      while(scanner.next(results));
-      assertEquals(prefixMap.get(s).size(), results.size());
+      ColumnPrefixFilter filter;
+      Scan scan = new Scan();
+      scan.setMaxVersions();
+      for (String s: prefixMap.keySet()) {
+        filter = new ColumnPrefixFilter(Bytes.toBytes(s));
+
+        scan.setFilter(filter);
+
+        InternalScanner scanner = region.getScanner(scan);
+        List<KeyValue> results = new ArrayList<KeyValue>();
+        while(scanner.next(results));
+        assertEquals(prefixMap.get(s).size(), results.size());
+      }
+    } finally {
+      region.close();
+      region.getLog().closeAndDelete();
     }
 
     region.close();
@@ -110,55 +114,59 @@ public class TestColumnPrefixFilter {
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
     HRegion region = HRegion.createHRegion(info, TEST_UTIL.
       getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
-
-    List<String> rows = generateRandomWords(100, "row");
-    List<String> columns = generateRandomWords(10000, "column");
-    long maxTimestamp = 2;
-
-    List<KeyValue> kvList = new ArrayList<KeyValue>();
-
-    Map<String, List<KeyValue>> prefixMap = new HashMap<String,
-        List<KeyValue>>();
-
-    prefixMap.put("p", new ArrayList<KeyValue>());
-    prefixMap.put("s", new ArrayList<KeyValue>());
-
-    String valueString = "ValueString";
-
-    for (String row: rows) {
-      Put p = new Put(Bytes.toBytes(row));
-      p.setWriteToWAL(false);
-      for (String column: columns) {
-        for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
-          KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
-              valueString);
-          p.add(kv);
-          kvList.add(kv);
-          for (String s: prefixMap.keySet()) {
-            if (column.startsWith(s)) {
-              prefixMap.get(s).add(kv);
+    try {
+      List<String> rows = generateRandomWords(100, "row");
+      List<String> columns = generateRandomWords(10000, "column");
+      long maxTimestamp = 2;
+
+      List<KeyValue> kvList = new ArrayList<KeyValue>();
+
+      Map<String, List<KeyValue>> prefixMap = new HashMap<String,
+          List<KeyValue>>();
+
+      prefixMap.put("p", new ArrayList<KeyValue>());
+      prefixMap.put("s", new ArrayList<KeyValue>());
+
+      String valueString = "ValueString";
+
+      for (String row: rows) {
+        Put p = new Put(Bytes.toBytes(row));
+        p.setWriteToWAL(false);
+        for (String column: columns) {
+          for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+            KeyValue kv = KeyValueTestUtil.create(row, family, column, timestamp,
+                valueString);
+            p.add(kv);
+            kvList.add(kv);
+            for (String s: prefixMap.keySet()) {
+              if (column.startsWith(s)) {
+                prefixMap.get(s).add(kv);
+              }
             }
           }
         }
+        region.put(p);
       }
-      region.put(p);
-    }
 
-    ColumnPrefixFilter filter;
-    Scan scan = new Scan();
-    scan.setMaxVersions();
-    for (String s: prefixMap.keySet()) {
-      filter = new ColumnPrefixFilter(Bytes.toBytes(s));
-
-      //this is how this test differs from the one above
-      FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
-      filterList.addFilter(filter);
-      scan.setFilter(filterList);
-
-      InternalScanner scanner = region.getScanner(scan);
-      List<KeyValue> results = new ArrayList<KeyValue>();
-      while(scanner.next(results));
-      assertEquals(prefixMap.get(s).size(), results.size());
+      ColumnPrefixFilter filter;
+      Scan scan = new Scan();
+      scan.setMaxVersions();
+      for (String s: prefixMap.keySet()) {
+        filter = new ColumnPrefixFilter(Bytes.toBytes(s));
+
+        //this is how this test differs from the one above
+        FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
+        filterList.addFilter(filter);
+        scan.setFilter(filterList);
+
+        InternalScanner scanner = region.getScanner(scan);
+        List<KeyValue> results = new ArrayList<KeyValue>();
+        while(scanner.next(results));
+        assertEquals(prefixMap.get(s).size(), results.size());
+      }
+    } finally {
+      region.close();
+      region.getLog().closeAndDelete();
     }
 
     region.close();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java Mon Apr 23 20:13:28 2012
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertFal
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -56,6 +57,55 @@ import org.junit.experimental.categories
 public class TestMasterFailover {
   private static final Log LOG = LogFactory.getLog(TestMasterFailover.class);
 
+  @Test (timeout=180000)
+  public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
+      throws Exception {
+    LOG.info("Starting testShouldCheckMasterFailOverWhenMETAIsInOpenedState");
+    final int NUM_MASTERS = 1;
+    final int NUM_RS = 2;
+
+    Configuration conf = HBaseConfiguration.create();
+    conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
+    conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
+    // Start the cluster
+    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
+
+    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
+    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+
+    // Find regionserver carrying meta.
+    List<RegionServerThread> regionServerThreads =
+      cluster.getRegionServerThreads();
+    int count = -1;
+    HRegion metaRegion = null;
+    for (RegionServerThread regionServerThread : regionServerThreads) {
+      HRegionServer regionServer = regionServerThread.getRegionServer();
+      metaRegion = regionServer.getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
+      count++;
+      regionServer.abort("");
+      if (null != metaRegion) break;
+    }
+    HRegionServer regionServer = cluster.getRegionServer(count);
+
+    TEST_UTIL.shutdownMiniHBaseCluster();
+
+    // Create a ZKW to use in the test
+    ZooKeeperWatcher zkw = 
+      HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, 
+          metaRegion, regionServer.getServerName());
+
+    LOG.info("Staring cluster for second time");
+    TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, NUM_RS);
+
+    // Failover should be completed, now wait for no RIT
+    log("Waiting for no more RIT");
+    ZKAssign.blockUntilNoRIT(zkw);
+
+    zkw.close();
+    // Stop the cluster
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
   /**
    * Simple test of master failover.
    * <p>
@@ -101,6 +151,7 @@ public class TestMasterFailover {
     }
     assertEquals(1, numActive);
     assertEquals(NUM_MASTERS, masterThreads.size());
+    LOG.info("Active master " + activeName);
 
     // Check that ClusterStatus reports the correct active and backup masters
     assertNotNull(active);
@@ -110,16 +161,16 @@ public class TestMasterFailover {
     assertEquals(2, status.getBackupMasters().size());
 
     // attempt to stop one of the inactive masters
-    LOG.debug("\n\nStopping a backup master\n");
     int backupIndex = (activeIndex == 0 ? 1 : activeIndex - 1);
+    HMaster master = cluster.getMaster(backupIndex);
+    LOG.debug("\n\nStopping a backup master: " + master.getServerName() + "\n");
     cluster.stopMaster(backupIndex, false);
     cluster.waitOnMaster(backupIndex);
 
-    // verify still one active master and it's the same
+    // Verify still one active master and it's the same
     for (int i = 0; i < masterThreads.size(); i++) {
       if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        assertTrue(activeName.equals(
-            masterThreads.get(i).getMaster().getServerName()));
+        assertTrue(activeName.equals(masterThreads.get(i).getMaster().getServerName()));
         activeIndex = i;
         active = masterThreads.get(activeIndex).getMaster();
       }
@@ -127,7 +178,7 @@ public class TestMasterFailover {
     assertEquals(1, numActive);
     assertEquals(2, masterThreads.size());
     int rsCount = masterThreads.get(activeIndex).getMaster().getClusterStatus().getServersSize();
-    LOG.info("Active master managing " + rsCount +  " regions servers");
+    LOG.info("Active master " + active.getServerName() + " managing " + rsCount +  " regions servers");
     assertEquals(3, rsCount);
 
     // Check that ClusterStatus reports the correct active and backup masters
@@ -138,7 +189,7 @@ public class TestMasterFailover {
     assertEquals(1, status.getBackupMasters().size());
 
     // kill the active master
-    LOG.debug("\n\nStopping the active master\n");
+    LOG.debug("\n\nStopping the active master " + active.getServerName() + "\n");
     cluster.stopMaster(activeIndex, false);
     cluster.waitOnMaster(activeIndex);
 
@@ -159,7 +210,7 @@ public class TestMasterFailover {
     assertEquals(0, status.getBackupMastersSize());
     assertEquals(0, status.getBackupMasters().size());
     int rss = status.getServersSize();
-    LOG.info("Active master " + mastername.getHostname() + " managing " +
+    LOG.info("Active master " + mastername.getServerName() + " managing " +
       rss +  " region servers");
     assertEquals(3, rss);
 
@@ -167,83 +218,6 @@ public class TestMasterFailover {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test
-  public void testShouldCheckMasterFailOverWhenMETAIsInOpenedState()
-      throws Exception {
-    final int NUM_MASTERS = 1;
-    final int NUM_RS = 2;
-
-    Configuration conf = HBaseConfiguration.create();
-    conf.setInt("hbase.master.assignment.timeoutmonitor.period", 2000);
-    conf.setInt("hbase.master.assignment.timeoutmonitor.timeout", 8000);
-    // Start the cluster
-    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
-    TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-
-    // get all the master threads
-    List<MasterThread> masterThreads = cluster.getMasterThreads();
-
-    // wait for each to come online
-    for (MasterThread mt : masterThreads) {
-      assertTrue(mt.isAlive());
-    }
-
-    // verify only one is the active master and we have right number
-    int numActive = 0;
-    ServerName activeName = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        numActive++;
-        activeName = masterThreads.get(i).getMaster().getServerName();
-      }
-    }
-    assertEquals(1, numActive);
-    assertEquals(NUM_MASTERS, masterThreads.size());
-
-    // verify still one active master and it's the same
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        assertTrue(activeName.equals(masterThreads.get(i).getMaster()
-            .getServerName()));
-      }
-    }
-    assertEquals(1, numActive);
-    assertEquals(1, masterThreads.size());
-
-    List<RegionServerThread> regionServerThreads = cluster
-        .getRegionServerThreads();
-    int count = -1;
-    HRegion metaRegion = null;
-    for (RegionServerThread regionServerThread : regionServerThreads) {
-      HRegionServer regionServer = regionServerThread.getRegionServer();
-      metaRegion = regionServer
-          .getOnlineRegion(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
-      count++;
-      regionServer.abort("");
-      if (null != metaRegion) {
-        break;
-      }
-    }
-    HRegionServer regionServer = cluster.getRegionServer(count);
-
-    cluster.shutdown();
-    // Create a ZKW to use in the test
-    ZooKeeperWatcher zkw = 
-      HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL, 
-          metaRegion, regionServer.getServerName());
-
-    TEST_UTIL.startMiniHBaseCluster(1, 1);
-
-    // Failover should be completed, now wait for no RIT
-    log("Waiting for no more RIT");
-    ZKAssign.blockUntilNoRIT(zkw);
-
-    // Stop the cluster
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-
   /**
    * Complex test of master failover that tests as many permutations of the
    * different possible states that regions in transition could be in within ZK.
@@ -379,7 +353,7 @@ public class TestMasterFailover {
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
 
     HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(), null, null);
-    HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
+    createRegion(hriEnabled, rootdir, conf, htdEnabled);
 
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@@ -390,7 +364,7 @@ public class TestMasterFailover {
     // Write the .tableinfo
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
     HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
-    HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
+    createRegion(hriDisabled, rootdir, conf, htdDisabled);
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
 
@@ -692,7 +666,7 @@ public class TestMasterFailover {
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdEnabled);
     HRegionInfo hriEnabled = new HRegionInfo(htdEnabled.getName(),
         null, null);
-    HRegion.createHRegion(hriEnabled, rootdir, conf, htdEnabled);
+    createRegion(hriEnabled, rootdir, conf, htdEnabled);
 
     List<HRegionInfo> enabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdEnabled, SPLIT_KEYS);
@@ -703,7 +677,7 @@ public class TestMasterFailover {
     // Write the .tableinfo
     FSTableDescriptors.createTableDescriptor(filesystem, rootdir, htdDisabled);
     HRegionInfo hriDisabled = new HRegionInfo(htdDisabled.getName(), null, null);
-    HRegion.createHRegion(hriDisabled, rootdir, conf, htdDisabled);
+    createRegion(hriDisabled, rootdir, conf, htdDisabled);
 
     List<HRegionInfo> disabledRegions = TEST_UTIL.createMultiRegionsInMeta(
         TEST_UTIL.getConfiguration(), htdDisabled, SPLIT_KEYS);
@@ -1024,6 +998,19 @@ public class TestMasterFailover {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  HRegion createRegion(final HRegionInfo  hri, final Path rootdir, final Configuration c,
+      final HTableDescriptor htd)
+  throws IOException {
+    HRegion r = HRegion.createHRegion(hri, rootdir, c, htd);
+    // The above call to create a region will create an hlog file.  Each
+    // log file create will also create a running thread to do syncing.  We need
+    // to close out this log else we will have a running thread trying to sync
+    // the file system continuously which is ugly when dfs is taken away at the
+    // end of the test.
+    HRegion.closeHRegion(r);
+    return r;
+  }
+
   // TODO: Next test to add is with testing permutations of the RIT or the RS
   //       killed are hosting ROOT and META regions.
 

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestOpenedRegionHandler.java Mon Apr 23 20:13:28 2012
@@ -112,6 +112,7 @@ public class TestOpenedRegionHandler {
   @Test
   public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
       throws Exception {
+    HRegion region = null;
     try {
       int testIndex = 0;
       TEST_UTIL.startMiniZKCluster();
@@ -120,8 +121,7 @@ public class TestOpenedRegionHandler {
           "testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches");
       HRegionInfo hri = new HRegionInfo(htd.getName(),
           Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
-      HRegion region = HRegion.createHRegion(hri, TEST_UTIL
-          .getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+      region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
       assertNotNull(region);
       AssignmentManager am = Mockito.mock(AssignmentManager.class);
       when(am.isRegionInTransition(hri)).thenReturn(
@@ -160,6 +160,8 @@ public class TestOpenedRegionHandler {
       assertEquals("The region should not be opened successfully.", regionName,
           region.getRegionInfo().getEncodedName());
     } finally {
+      region.close();
+      region.getLog().closeAndDelete();
       TEST_UTIL.shutdownMiniZKCluster();
     }
   }

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java Mon Apr 23 20:13:28 2012
@@ -85,7 +85,16 @@ public class TestBlocksRead extends HBas
     EnvironmentEdgeManagerTestHelper.reset();
   }
 
-  private void initHRegion(byte[] tableName, String callingMethod,
+  /**
+   * Callers must afterward call {@link HRegion#closeHRegion(HRegion)}
+   * @param tableName
+   * @param callingMethod
+   * @param conf
+   * @param families
+   * @throws IOException
+   * @return created and initialized region.
+   */
+  private HRegion initHRegion(byte[] tableName, String callingMethod,
       HBaseConfiguration conf, String family) throws IOException {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     HColumnDescriptor familyDesc;
@@ -99,8 +108,9 @@ public class TestBlocksRead extends HBas
 
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
-    region = HRegion.createHRegion(info, path, conf, htd);
+    HRegion r = HRegion.createHRegion(info, path, conf, htd);
     blockCache = new CacheConfig(conf).getBlockCache();
+    return r;
   }
 
   private void putData(String family, String row, String col, long version)
@@ -212,45 +222,50 @@ public class TestBlocksRead extends HBas
     String FAMILY = "cf1";
     KeyValue kvs[];
     HBaseConfiguration conf = getConf();
-    initHRegion(TABLE, getName(), conf, FAMILY);
+    this.region = initHRegion(TABLE, getName(), conf, FAMILY);
 
-    putData(FAMILY, "row", "col1", 1);
-    putData(FAMILY, "row", "col2", 2);
-    putData(FAMILY, "row", "col3", 3);
-    putData(FAMILY, "row", "col4", 4);
-    putData(FAMILY, "row", "col5", 5);
-    putData(FAMILY, "row", "col6", 6);
-    putData(FAMILY, "row", "col7", 7);
-    region.flushcache();
-
-    // Expected block reads: 1
-    // The top block has the KV we are
-    // interested. So only 1 seek is needed.
-    kvs = getData(FAMILY, "row", "col1", 1);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col1", 1);
-
-    // Expected block reads: 2
-    // The top block and next block has the KVs we are
-    // interested. So only 2 seek is needed.
-    kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
-    assertEquals(2, kvs.length);
-    verifyData(kvs[0], "row", "col1", 1);
-    verifyData(kvs[1], "row", "col2", 2);
-
-    // Expected block reads: 3
-    // The first 2 seeks is to find out col2. [HBASE-4443]
-    // One additional seek for col3
-    // So 3 seeks are needed.
-    kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
-    assertEquals(2, kvs.length);
-    verifyData(kvs[0], "row", "col2", 2);
-    verifyData(kvs[1], "row", "col3", 3);
-
-    // Expected block reads: 2. [HBASE-4443]
-    kvs = getData(FAMILY, "row", Arrays.asList("col5"), 2);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col5", 5);
+    try {
+      putData(FAMILY, "row", "col1", 1);
+      putData(FAMILY, "row", "col2", 2);
+      putData(FAMILY, "row", "col3", 3);
+      putData(FAMILY, "row", "col4", 4);
+      putData(FAMILY, "row", "col5", 5);
+      putData(FAMILY, "row", "col6", 6);
+      putData(FAMILY, "row", "col7", 7);
+      region.flushcache();
+
+      // Expected block reads: 1
+      // The top block has the KV we are
+      // interested. So only 1 seek is needed.
+      kvs = getData(FAMILY, "row", "col1", 1);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col1", 1);
+
+      // Expected block reads: 2
+      // The top block and next block has the KVs we are
+      // interested. So only 2 seek is needed.
+      kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
+      assertEquals(2, kvs.length);
+      verifyData(kvs[0], "row", "col1", 1);
+      verifyData(kvs[1], "row", "col2", 2);
+
+      // Expected block reads: 3
+      // The first 2 seeks is to find out col2. [HBASE-4443]
+      // One additional seek for col3
+      // So 3 seeks are needed.
+      kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
+      assertEquals(2, kvs.length);
+      verifyData(kvs[0], "row", "col2", 2);
+      verifyData(kvs[1], "row", "col3", 3);
+
+      // Expected block reads: 2. [HBASE-4443]
+      kvs = getData(FAMILY, "row", Arrays.asList("col5"), 2);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col5", 5);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   /**
@@ -264,97 +279,102 @@ public class TestBlocksRead extends HBas
     String FAMILY = "cf1";
     KeyValue kvs[];
     HBaseConfiguration conf = getConf();
-    initHRegion(TABLE, getName(), conf, FAMILY);
+    this.region = initHRegion(TABLE, getName(), conf, FAMILY);
 
-    // File 1
-    putData(FAMILY, "row", "col1", 1);
-    putData(FAMILY, "row", "col2", 2);
-    region.flushcache();
-
-    // File 2
-    putData(FAMILY, "row", "col1", 3);
-    putData(FAMILY, "row", "col2", 4);
-    region.flushcache();
-
-    // Expected blocks read: 1.
-    // File 2's top block is also the KV we are
-    // interested. So only 1 seek is needed.
-    kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col1", 3);
-
-    // Expected blocks read: 2
-    // File 2's top block has the "col1" KV we are
-    // interested. We also need "col2" which is in a block
-    // of its own. So, we need that block as well.
-    kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
-    assertEquals(2, kvs.length);
-    verifyData(kvs[0], "row", "col1", 3);
-    verifyData(kvs[1], "row", "col2", 4);
-
-    // File 3: Add another column
-    putData(FAMILY, "row", "col3", 5);
-    region.flushcache();
-
-    // Expected blocks read: 1
-    // File 3's top block has the "col3" KV we are
-    // interested. So only 1 seek is needed.
-    kvs = getData(FAMILY, "row", "col3", 1);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col3", 5);
-
-    // Get a column from older file.
-    // For ROWCOL Bloom filter: Expected blocks read: 1.
-    // For ROW Bloom filter: Expected blocks read: 2.
-    // For NONE Bloom filter: Expected blocks read: 2.
-    kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1, 2, 2);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col1", 3);
-
-    // File 4: Delete the entire row.
-    deleteFamily(FAMILY, "row", 6);
-    region.flushcache();
-
-    // For ROWCOL Bloom filter: Expected blocks read: 2.
-    // For ROW Bloom filter: Expected blocks read: 3.
-    // For NONE Bloom filter: Expected blocks read: 3.
-    kvs = getData(FAMILY, "row", "col1", 2, 3, 3);
-    assertEquals(0, kvs.length);
-    kvs = getData(FAMILY, "row", "col2", 3, 4, 4);
-    assertEquals(0, kvs.length);
-    kvs = getData(FAMILY, "row", "col3", 2);
-    assertEquals(0, kvs.length);
-    kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 4);
-    assertEquals(0, kvs.length);
-
-    // File 5: Delete
-    deleteFamily(FAMILY, "row", 10);
-    region.flushcache();
-
-    // File 6: some more puts, but with timestamps older than the
-    // previous delete.
-    putData(FAMILY, "row", "col1", 7);
-    putData(FAMILY, "row", "col2", 8);
-    putData(FAMILY, "row", "col3", 9);
-    region.flushcache();
-
-    // Baseline expected blocks read: 8. [HBASE-4532]
-    kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
-    assertEquals(0, kvs.length);
-
-    // File 7: Put back new data
-    putData(FAMILY, "row", "col1", 11);
-    putData(FAMILY, "row", "col2", 12);
-    putData(FAMILY, "row", "col3", 13);
-    region.flushcache();
-
-
-    // Expected blocks read: 5. [HBASE-4585]
-    kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
-    assertEquals(3, kvs.length);
-    verifyData(kvs[0], "row", "col1", 11);
-    verifyData(kvs[1], "row", "col2", 12);
-    verifyData(kvs[2], "row", "col3", 13);
+    try {
+      // File 1
+      putData(FAMILY, "row", "col1", 1);
+      putData(FAMILY, "row", "col2", 2);
+      region.flushcache();
+
+      // File 2
+      putData(FAMILY, "row", "col1", 3);
+      putData(FAMILY, "row", "col2", 4);
+      region.flushcache();
+
+      // Expected blocks read: 1.
+      // File 2's top block is also the KV we are
+      // interested. So only 1 seek is needed.
+      kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col1", 3);
+
+      // Expected blocks read: 2
+      // File 2's top block has the "col1" KV we are
+      // interested. We also need "col2" which is in a block
+      // of its own. So, we need that block as well.
+      kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
+      assertEquals(2, kvs.length);
+      verifyData(kvs[0], "row", "col1", 3);
+      verifyData(kvs[1], "row", "col2", 4);
+
+      // File 3: Add another column
+      putData(FAMILY, "row", "col3", 5);
+      region.flushcache();
+
+      // Expected blocks read: 1
+      // File 3's top block has the "col3" KV we are
+      // interested. So only 1 seek is needed.
+      kvs = getData(FAMILY, "row", "col3", 1);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col3", 5);
+
+      // Get a column from older file.
+      // For ROWCOL Bloom filter: Expected blocks read: 1.
+      // For ROW Bloom filter: Expected blocks read: 2.
+      // For NONE Bloom filter: Expected blocks read: 2.
+      kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1, 2, 2);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col1", 3);
+
+      // File 4: Delete the entire row.
+      deleteFamily(FAMILY, "row", 6);
+      region.flushcache();
+
+      // For ROWCOL Bloom filter: Expected blocks read: 2.
+      // For ROW Bloom filter: Expected blocks read: 3.
+      // For NONE Bloom filter: Expected blocks read: 3.
+      kvs = getData(FAMILY, "row", "col1", 2, 3, 3);
+      assertEquals(0, kvs.length);
+      kvs = getData(FAMILY, "row", "col2", 3, 4, 4);
+      assertEquals(0, kvs.length);
+      kvs = getData(FAMILY, "row", "col3", 2);
+      assertEquals(0, kvs.length);
+      kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 4);
+      assertEquals(0, kvs.length);
+
+      // File 5: Delete
+      deleteFamily(FAMILY, "row", 10);
+      region.flushcache();
+
+      // File 6: some more puts, but with timestamps older than the
+      // previous delete.
+      putData(FAMILY, "row", "col1", 7);
+      putData(FAMILY, "row", "col2", 8);
+      putData(FAMILY, "row", "col3", 9);
+      region.flushcache();
+
+      // Baseline expected blocks read: 8. [HBASE-4532]
+      kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
+      assertEquals(0, kvs.length);
+ 
+      // File 7: Put back new data
+      putData(FAMILY, "row", "col1", 11);
+      putData(FAMILY, "row", "col2", 12);
+      putData(FAMILY, "row", "col3", 13);
+      region.flushcache();
+
+
+      // Expected blocks read: 5. [HBASE-4585]
+      kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
+      assertEquals(3, kvs.length);
+      verifyData(kvs[0], "row", "col1", 11);
+      verifyData(kvs[1], "row", "col2", 12);
+      verifyData(kvs[2], "row", "col3", 13);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   /**
@@ -367,62 +387,71 @@ public class TestBlocksRead extends HBas
     String FAMILY = "cf1";
 
     HBaseConfiguration conf = getConf();
-    initHRegion(TABLE, getName(), conf, FAMILY);
+    this.region = initHRegion(TABLE, getName(), conf, FAMILY);
 
-    putData(FAMILY, "row", "col1", 1);
-    putData(FAMILY, "row", "col2", 2);
-    region.flushcache();
-
-    // Execute a scan with caching turned off
-    // Expected blocks stored: 0
-    long blocksStart = getBlkCount();
-    Scan scan = new Scan();
-    scan.setCacheBlocks(false);
-    RegionScanner rs = region.getScanner(scan);
-    List<KeyValue> result = new ArrayList<KeyValue>(2);
-    rs.next(result);
-    assertEquals(2 * BLOOM_TYPE.length, result.size());
-    rs.close();
-    long blocksEnd = getBlkCount();
-
-    assertEquals(blocksStart, blocksEnd);
-
-    // Execute with caching turned on
-    // Expected blocks stored: 2
-    blocksStart = blocksEnd;
-    scan.setCacheBlocks(true);
-    rs = region.getScanner(scan);
-    result = new ArrayList<KeyValue>(2);
-    rs.next(result);
-    assertEquals(2 * BLOOM_TYPE.length, result.size());
-    rs.close();
-    blocksEnd = getBlkCount();
+    try {
+      putData(FAMILY, "row", "col1", 1);
+      putData(FAMILY, "row", "col2", 2);
+      region.flushcache();
+
+      // Execute a scan with caching turned off
+      // Expected blocks stored: 0
+      long blocksStart = getBlkCount();
+      Scan scan = new Scan();
+      scan.setCacheBlocks(false);
+      RegionScanner rs = region.getScanner(scan);
+      List<KeyValue> result = new ArrayList<KeyValue>(2);
+      rs.next(result);
+      assertEquals(2 * BLOOM_TYPE.length, result.size());
+      rs.close();
+      long blocksEnd = getBlkCount();
+
+      assertEquals(blocksStart, blocksEnd);
+
+      // Execute with caching turned on
+      // Expected blocks stored: 2
+      blocksStart = blocksEnd;
+      scan.setCacheBlocks(true);
+      rs = region.getScanner(scan);
+      result = new ArrayList<KeyValue>(2);
+      rs.next(result);
+      assertEquals(2 * BLOOM_TYPE.length, result.size());
+      rs.close();
+      blocksEnd = getBlkCount();
     
-    assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
-	}
+      assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
+  }
 
-	@Test
+  @Test
   public void testLazySeekBlocksReadWithDelete() throws Exception {
     byte[] TABLE = Bytes.toBytes("testLazySeekBlocksReadWithDelete");
     String FAMILY = "cf1";
     KeyValue kvs[];
     HBaseConfiguration conf = getConf();
-    initHRegion(TABLE, getName(), conf, FAMILY);
-
-    deleteFamily(FAMILY, "row", 200);
-    for (int i = 0; i < 100; i++) {
-      putData(FAMILY, "row", "col" + i, i);
-    }
-    putData(FAMILY, "row", "col99", 201);
-    region.flushcache();
+    this.region = initHRegion(TABLE, getName(), conf, FAMILY);
+    try {
+      deleteFamily(FAMILY, "row", 200);
+      for (int i = 0; i < 100; i++) {
+        putData(FAMILY, "row", "col" + i, i);
+      }
+      putData(FAMILY, "row", "col99", 201);
+      region.flushcache();
 
-    kvs = getData(FAMILY, "row", Arrays.asList("col0"), 2);
-    assertEquals(0, kvs.length);
+      kvs = getData(FAMILY, "row", Arrays.asList("col0"), 2);
+      assertEquals(0, kvs.length);
 
-    kvs = getData(FAMILY, "row", Arrays.asList("col99"), 2);
-    assertEquals(1, kvs.length);
-    verifyData(kvs[0], "row", "col99", 201);
-	}
+      kvs = getData(FAMILY, "row", Arrays.asList("col99"), 2);
+      assertEquals(1, kvs.length);
+      verifyData(kvs[0], "row", "col99", 201);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
+  }
 
   @org.junit.Rule
   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java Mon Apr 23 20:13:28 2012
@@ -62,95 +62,98 @@ public class TestColumnSeeking {
     HRegion region =
         HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(), TEST_UTIL
             .getConfiguration(), htd);
-
-    List<String> rows = generateRandomWords(10, "row");
-    List<String> allColumns = generateRandomWords(10, "column");
-    List<String> values = generateRandomWords(100, "value");
-
-    long maxTimestamp = 2;
-    double selectPercent = 0.5;
-    int numberOfTests = 5;
-    double flushPercentage = 0.2;
-    double minorPercentage = 0.2;
-    double majorPercentage = 0.2;
-    double putPercentage = 0.2;
-
-    HashMap<String, KeyValue> allKVMap = new HashMap<String, KeyValue>();
-
-    HashMap<String, KeyValue>[] kvMaps = new HashMap[numberOfTests];
-    ArrayList<String>[] columnLists = new ArrayList[numberOfTests];
-
-    for (int i = 0; i < numberOfTests; i++) {
-      kvMaps[i] = new HashMap<String, KeyValue>();
-      columnLists[i] = new ArrayList<String>();
-      for (String column : allColumns) {
-        if (Math.random() < selectPercent) {
-          columnLists[i].add(column);
+    try {
+      List<String> rows = generateRandomWords(10, "row");
+      List<String> allColumns = generateRandomWords(10, "column");
+      List<String> values = generateRandomWords(100, "value");
+
+      long maxTimestamp = 2;
+      double selectPercent = 0.5;
+      int numberOfTests = 5;
+      double flushPercentage = 0.2;
+      double minorPercentage = 0.2;
+      double majorPercentage = 0.2;
+      double putPercentage = 0.2;
+
+      HashMap<String, KeyValue> allKVMap = new HashMap<String, KeyValue>();
+
+      HashMap<String, KeyValue>[] kvMaps = new HashMap[numberOfTests];
+      ArrayList<String>[] columnLists = new ArrayList[numberOfTests];
+
+      for (int i = 0; i < numberOfTests; i++) {
+        kvMaps[i] = new HashMap<String, KeyValue>();
+        columnLists[i] = new ArrayList<String>();
+        for (String column : allColumns) {
+          if (Math.random() < selectPercent) {
+            columnLists[i].add(column);
+          }
         }
       }
-    }
 
-    for (String value : values) {
-      for (String row : rows) {
-        Put p = new Put(Bytes.toBytes(row));
-        p.setWriteToWAL(false);
-        for (String column : allColumns) {
-          for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
-            KeyValue kv =
-                KeyValueTestUtil.create(row, family, column, timestamp, value);
-            if (Math.random() < putPercentage) {
-              p.add(kv);
-              allKVMap.put(kv.getKeyString(), kv);
-              for (int i = 0; i < numberOfTests; i++) {
-                if (columnLists[i].contains(column)) {
-                  kvMaps[i].put(kv.getKeyString(), kv);
+      for (String value : values) {
+        for (String row : rows) {
+          Put p = new Put(Bytes.toBytes(row));
+          p.setWriteToWAL(false);
+          for (String column : allColumns) {
+            for (long timestamp = 1; timestamp <= maxTimestamp; timestamp++) {
+              KeyValue kv =
+                  KeyValueTestUtil.create(row, family, column, timestamp, value);
+              if (Math.random() < putPercentage) {
+                p.add(kv);
+                allKVMap.put(kv.getKeyString(), kv);
+                for (int i = 0; i < numberOfTests; i++) {
+                  if (columnLists[i].contains(column)) {
+                    kvMaps[i].put(kv.getKeyString(), kv);
+                  }
                 }
               }
             }
           }
-        }
-        region.put(p);
-        if (Math.random() < flushPercentage) {
-          LOG.info("Flushing... ");
-          region.flushcache();
-        }
+          region.put(p);
+          if (Math.random() < flushPercentage) {
+            LOG.info("Flushing... ");
+            region.flushcache();
+          }
 
-        if (Math.random() < minorPercentage) {
-          LOG.info("Minor compacting... ");
-          region.compactStores(false);
-        }
+          if (Math.random() < minorPercentage) {
+            LOG.info("Minor compacting... ");
+            region.compactStores(false);
+          }
 
-        if (Math.random() < majorPercentage) {
-          LOG.info("Major compacting... ");
-          region.compactStores(true);
+          if (Math.random() < majorPercentage) {
+            LOG.info("Major compacting... ");
+            region.compactStores(true);
+          }
         }
       }
-    }
-
-    for (int i = 0; i < numberOfTests + 1; i++) {
-      Collection<KeyValue> kvSet;
-      Scan scan = new Scan();
-      scan.setMaxVersions();
-      if (i < numberOfTests) {
-        kvSet = kvMaps[i].values();
-        for (String column : columnLists[i]) {
-          scan.addColumn(familyBytes, Bytes.toBytes(column));
-        }
-        LOG.info("ExplicitColumns scanner");
-        LOG.info("Columns: " + columnLists[i].size() + "  Keys: "
-            + kvSet.size());
-      } else {
-        kvSet = allKVMap.values();
-        LOG.info("Wildcard scanner");
-        LOG.info("Columns: " + allColumns.size() + "  Keys: " + kvSet.size());
 
+      for (int i = 0; i < numberOfTests + 1; i++) {
+        Collection<KeyValue> kvSet;
+        Scan scan = new Scan();
+        scan.setMaxVersions();
+        if (i < numberOfTests) {
+          kvSet = kvMaps[i].values();
+          for (String column : columnLists[i]) {
+            scan.addColumn(familyBytes, Bytes.toBytes(column));
+          }
+          LOG.info("ExplicitColumns scanner");
+          LOG.info("Columns: " + columnLists[i].size() + "  Keys: "
+              + kvSet.size());
+        } else {
+          kvSet = allKVMap.values();
+          LOG.info("Wildcard scanner");
+          LOG.info("Columns: " + allColumns.size() + "  Keys: " + kvSet.size());
+
+        }
+        InternalScanner scanner = region.getScanner(scan);
+        List<KeyValue> results = new ArrayList<KeyValue>();
+        while (scanner.next(results))
+          ;
+        assertEquals(kvSet.size(), results.size());
+        assertTrue(results.containsAll(kvSet));
       }
-      InternalScanner scanner = region.getScanner(scan);
-      List<KeyValue> results = new ArrayList<KeyValue>();
-      while (scanner.next(results))
-        ;
-      assertEquals(kvSet.size(), results.size());
-      assertTrue(results.containsAll(kvSet));
+    } finally {
+      HRegion.closeHRegion(region);
     }
 
     region.close();

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java?rev=1329418&r1=1329417&r2=1329418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java Mon Apr 23 20:13:28 2012
@@ -86,9 +86,10 @@ public class TestCompactSelection extend
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
 
     HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
-    HRegion.createHRegion(info, basedir, conf, htd);
+    HRegion region = HRegion.createHRegion(info, basedir, conf, htd);
+    HRegion.closeHRegion(region);
     Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
-    HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
+    region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
 
     store = new Store(basedir, region, hcd, fs, conf);
     TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());