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 05:57:41 UTC

svn commit: r1329052 [2/3] - in /hbase/branches/0.92: ./ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/io/hfile/ src/main/java/org/apache/hadoop/hbase/metrics/histogram/ src/main/java/org/apache/hadoop/hbase/region...

Modified: hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1329052&r1=1329051&r2=1329052&view=diff
==============================================================================
--- hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/0.92/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Mon Apr 23 03:57:40 2012
@@ -94,8 +94,8 @@ public class TestHRegion extends HBaseTe
   static final Log LOG = LogFactory.getLog(TestHRegion.class);
 
   HRegion region = null;
-  private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
 
   private final int MAX_VERSIONS = 2;
 
@@ -124,6 +124,64 @@ public class TestHRegion extends HBaseTe
     EnvironmentEdgeManagerTestHelper.reset();
   }
 
+  @Test public void testgetHDFSBlocksDistribution() throws Exception {
+    HBaseTestingUtility htu = new HBaseTestingUtility();
+    final int DEFAULT_BLOCK_SIZE = 1024;
+    htu.getConfiguration().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
+    htu.getConfiguration().setInt("dfs.replication", 2);
+
+
+    // set up a cluster with 3 nodes
+    MiniHBaseCluster cluster;
+    String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
+    int regionServersCount = 3;
+
+    try {
+      cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
+      byte [][] families = {fam1, fam2};
+      HTable ht = htu.createTable(Bytes.toBytes(this.getName()), families);
+
+      //Setting up region
+      byte row[] = Bytes.toBytes("row1");
+      byte col[] = Bytes.toBytes("col1");
+
+      Put put = new Put(row);
+      put.add(fam1, col, 1, Bytes.toBytes("test1"));
+      put.add(fam2, col, 1, Bytes.toBytes("test2"));
+      ht.put(put);
+
+      HRegion firstRegion = htu.getHBaseCluster().
+          getRegions(Bytes.toBytes(this.getName())).get(0);
+      firstRegion.flushcache();
+      HDFSBlocksDistribution blocksDistribution1 =
+          firstRegion.getHDFSBlocksDistribution();
+
+      // given the default replication factor is 2 and we have 2 HFiles,
+      // we will have total of 4 replica of blocks on 3 datanodes; thus there
+      // must be at least one host that have replica for 2 HFiles. That host's
+      // weight will be equal to the unique block weight.
+      long uniqueBlocksWeight1 =
+          blocksDistribution1.getUniqueBlocksTotalWeight();
+
+      String topHost = blocksDistribution1.getTopHosts().get(0);
+      long topHostWeight = blocksDistribution1.getWeight(topHost);
+      assertTrue(uniqueBlocksWeight1 == topHostWeight);
+
+      // use the static method to compute the value, it should be the same.
+      // static method is used by load balancer or other components
+      HDFSBlocksDistribution blocksDistribution2 =
+          HRegion.computeHDFSBlocksDistribution(htu.getConfiguration(),
+              firstRegion.getTableDesc(),
+              firstRegion.getRegionInfo().getEncodedName());
+      long uniqueBlocksWeight2 =
+          blocksDistribution2.getUniqueBlocksTotalWeight();
+
+      assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
+    } finally {
+      htu.shutdownMiniCluster();
+    }
+  }
+
   //////////////////////////////////////////////////////////////////////////////
   // New tests that doesn't spin up a mini cluster but rather just test the
   // individual code pieces in the HRegion. Putting files locally in
@@ -136,39 +194,44 @@ public class TestHRegion extends HBaseTe
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
     Configuration conf = HBaseConfiguration.create();
-    initHRegion(tableName, method, conf, family);
-    Path regiondir = region.getRegionDir();
-    FileSystem fs = region.getFilesystem();
-    byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
-
-    Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
-
-    long maxSeqId = 1050;
-    long minSeqId = 1000;
-
-    for (long i = minSeqId; i <= maxSeqId; i += 10) {
-      Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
-      fs.create(recoveredEdits);
-      HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
-
-      long time = System.nanoTime();
-      WALEdit edit = new WALEdit();
-      edit.add(new KeyValue(row, family, Bytes.toBytes(i),
-          time, KeyValue.Type.Put, Bytes.toBytes(i)));
-      writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
-          i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
-
-      writer.close();
-    }
-    MonitoredTask status = TaskMonitor.get().createStatus(method);
-    long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
-    assertEquals(maxSeqId, seqId);
-    Get get = new Get(row);
-    Result result = region.get(get, null);
-    for (long i = minSeqId; i <= maxSeqId; i += 10) {
-      List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
-      assertEquals(1, kvs.size());
-      assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+    this.region = initHRegion(tableName, method, conf, family);
+    try {
+      Path regiondir = region.getRegionDir();
+      FileSystem fs = region.getFilesystem();
+      byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
+
+      Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+
+      long maxSeqId = 1050;
+      long minSeqId = 1000;
+
+      for (long i = minSeqId; i <= maxSeqId; i += 10) {
+        Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
+        fs.create(recoveredEdits);
+        HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
+
+        long time = System.nanoTime();
+        WALEdit edit = new WALEdit();
+        edit.add(new KeyValue(row, family, Bytes.toBytes(i),
+            time, KeyValue.Type.Put, Bytes.toBytes(i)));
+        writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
+            i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
+
+        writer.close();
+      }
+      MonitoredTask status = TaskMonitor.get().createStatus(method);
+      long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
+      assertEquals(maxSeqId, seqId);
+      Get get = new Get(row);
+      Result result = region.get(get, null);
+      for (long i = minSeqId; i <= maxSeqId; i += 10) {
+        List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
+        assertEquals(1, kvs.size());
+        assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -176,44 +239,49 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplaySomeIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    initHRegion(tableName, method, HBaseConfiguration.create(), family);
-    Path regiondir = region.getRegionDir();
-    FileSystem fs = region.getFilesystem();
-    byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
-
-    Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
-
-    long maxSeqId = 1050;
-    long minSeqId = 1000;
-
-    for (long i = minSeqId; i <= maxSeqId; i += 10) {
-      Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
-      fs.create(recoveredEdits);
-      HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
-
-      long time = System.nanoTime();
-      WALEdit edit = new WALEdit();
-      edit.add(new KeyValue(row, family, Bytes.toBytes(i),
-          time, KeyValue.Type.Put, Bytes.toBytes(i)));
-      writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
-          i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
-
-      writer.close();
-    }
-    long recoverSeqId = 1030;
-    MonitoredTask status = TaskMonitor.get().createStatus(method);
-    long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
-    assertEquals(maxSeqId, seqId);
-    Get get = new Get(row);
-    Result result = region.get(get, null);
-    for (long i = minSeqId; i <= maxSeqId; i += 10) {
-      List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
-      if (i < recoverSeqId) {
-        assertEquals(0, kvs.size());
-      } else {
-        assertEquals(1, kvs.size());
-        assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
+    try {
+      Path regiondir = region.getRegionDir();
+      FileSystem fs = region.getFilesystem();
+      byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
+
+      Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+
+      long maxSeqId = 1050;
+      long minSeqId = 1000;
+
+      for (long i = minSeqId; i <= maxSeqId; i += 10) {
+        Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
+        fs.create(recoveredEdits);
+        HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
+
+        long time = System.nanoTime();
+        WALEdit edit = new WALEdit();
+        edit.add(new KeyValue(row, family, Bytes.toBytes(i),
+            time, KeyValue.Type.Put, Bytes.toBytes(i)));
+        writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
+            i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
+
+        writer.close();
+      }
+      long recoverSeqId = 1030;
+      MonitoredTask status = TaskMonitor.get().createStatus(method);
+      long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
+      assertEquals(maxSeqId, seqId);
+      Get get = new Get(row);
+      Result result = region.get(get, null);
+      for (long i = minSeqId; i <= maxSeqId; i += 10) {
+        List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
+        if (i < recoverSeqId) {
+          assertEquals(0, kvs.size());
+        } else {
+          assertEquals(1, kvs.size());
+          assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
+        }
       }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -221,25 +289,30 @@ public class TestHRegion extends HBaseTe
     String method = "testSkipRecoveredEditsReplayAllIgnored";
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
-    initHRegion(tableName, method, HBaseConfiguration.create(), family);
-    Path regiondir = region.getRegionDir();
-    FileSystem fs = region.getFilesystem();
+    this.region = initHRegion(tableName, method, HBaseConfiguration.create(), family);
+    try {
+      Path regiondir = region.getRegionDir();
+      FileSystem fs = region.getFilesystem();
 
-    Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
-    for (int i = 1000; i < 1050; i += 10) {
+      Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
+      for (int i = 1000; i < 1050; i += 10) {
+        Path recoveredEdits = new Path(
+            recoveredEditsDir, String.format("%019d", i));
+        FSDataOutputStream dos=  fs.create(recoveredEdits);
+        dos.writeInt(i);
+        dos.close();
+      }
+      long minSeqId = 2000;
       Path recoveredEdits = new Path(
-          recoveredEditsDir, String.format("%019d", i));
+          recoveredEditsDir, String.format("%019d", minSeqId-1));
       FSDataOutputStream dos=  fs.create(recoveredEdits);
-      dos.writeInt(i);
       dos.close();
+      long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
+      assertEquals(minSeqId, seqId);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    long minSeqId = 2000;
-    Path recoveredEdits = new Path(
-        recoveredEditsDir, String.format("%019d", minSeqId-1));
-    FSDataOutputStream dos=  fs.create(recoveredEdits);
-    dos.close();
-    long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
-    assertEquals(minSeqId, seqId);
   }
 
   public void testGetWhileRegionClose() throws IOException {
@@ -249,52 +322,56 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, hc, families);
-
-    // Put data in region
-    final int startRow = 100;
-    putData(startRow, numRows, qual1, families);
-    putData(startRow, numRows, qual2, families);
-    putData(startRow, numRows, qual3, families);
-    // this.region.flushcache();
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicInteger gets = new AtomicInteger(0);
-    GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
-    try {
-      // Set ten threads running concurrently getting from the region.
-      for (int i = 0; i < threads.length / 2; i++) {
-        threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
-          done, gets);
-        threads[i].setDaemon(true);
-        threads[i].start();
-      }
-      // Artificially make the condition by setting closing flag explicitly.
-      // I can't make the issue happen with a call to region.close().
-      this.region.closing.set(true);
-      for (int i = threads.length / 2; i < threads.length; i++) {
-        threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
-          done, gets);
-        threads[i].setDaemon(true);
-        threads[i].start();
-      }
-    } finally {
-      if (this.region != null) {
-        this.region.close();
-        this.region.getLog().closeAndDelete();
-      }
-    }
-    done.set(true);
-    for (GetTillDoneOrException t: threads) {
+    this.region = initHRegion(tableName, method, hc, families);
+    try {
+      // Put data in region
+      final int startRow = 100;
+      putData(startRow, numRows, qual1, families);
+      putData(startRow, numRows, qual2, families);
+      putData(startRow, numRows, qual3, families);
+      // this.region.flushcache();
+      final AtomicBoolean done = new AtomicBoolean(false);
+      final AtomicInteger gets = new AtomicInteger(0);
+      GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
       try {
-        t.join();
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-      if (t.e != null) {
-        LOG.info("Exception=" + t.e);
-        assertFalse("Found a NPE in " + t.getName(),
-          t.e instanceof NullPointerException);
+        // Set ten threads running concurrently getting from the region.
+        for (int i = 0; i < threads.length / 2; i++) {
+          threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
+              done, gets);
+          threads[i].setDaemon(true);
+          threads[i].start();
+        }
+        // Artificially make the condition by setting closing flag explicitly.
+        // I can't make the issue happen with a call to region.close().
+        this.region.closing.set(true);
+        for (int i = threads.length / 2; i < threads.length; i++) {
+          threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
+              done, gets);
+          threads[i].setDaemon(true);
+          threads[i].start();
+        }
+      } finally {
+        if (this.region != null) {
+          this.region.close();
+          this.region.getLog().closeAndDelete();
+        }
+      }
+      done.set(true);
+      for (GetTillDoneOrException t: threads) {
+        try {
+          t.join();
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+        if (t.e != null) {
+          LOG.info("Exception=" + t.e);
+          assertFalse("Found a NPE in " + t.getName(),
+              t.e instanceof NullPointerException);
+        }
       }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -338,38 +415,43 @@ public class TestHRegion extends HBaseTe
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
-    initHRegion(TABLE, getName(), FAMILIES);
-    String value = "this is the value";
-    String value2 = "this is some other value";
-    String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
-    String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
-    String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
-    putRows(this.region, 3, value, keyPrefix1);
-    putRows(this.region, 3, value, keyPrefix2);
-    putRows(this.region, 3, value, keyPrefix3);
-    // this.region.flushCommits();
-    putRows(this.region, 3, value2, keyPrefix1);
-    putRows(this.region, 3, value2, keyPrefix2);
-    putRows(this.region, 3, value2, keyPrefix3);
-    System.out.println("Checking values for key: " + keyPrefix1);
-    assertEquals("Got back incorrect number of rows from scan", 3,
-      getNumberOfRows(keyPrefix1, value2, this.region));
-    System.out.println("Checking values for key: " + keyPrefix2);
-    assertEquals("Got back incorrect number of rows from scan", 3,
-      getNumberOfRows(keyPrefix2, value2, this.region));
-    System.out.println("Checking values for key: " + keyPrefix3);
-    assertEquals("Got back incorrect number of rows from scan", 3,
-      getNumberOfRows(keyPrefix3, value2, this.region));
-    deleteColumns(this.region, value2, keyPrefix1);
-    deleteColumns(this.region, value2, keyPrefix2);
-    deleteColumns(this.region, value2, keyPrefix3);
-    System.out.println("Starting important checks.....");
-    assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
-      0, getNumberOfRows(keyPrefix1, value2, this.region));
-    assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
-      0, getNumberOfRows(keyPrefix2, value2, this.region));
-    assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
-      0, getNumberOfRows(keyPrefix3, value2, this.region));
+    this.region = initHRegion(TABLE, getName(), FAMILIES);
+    try {
+      String value = "this is the value";
+      String value2 = "this is some other value";
+      String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
+      String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
+      String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
+      putRows(this.region, 3, value, keyPrefix1);
+      putRows(this.region, 3, value, keyPrefix2);
+      putRows(this.region, 3, value, keyPrefix3);
+      // this.region.flushCommits();
+      putRows(this.region, 3, value2, keyPrefix1);
+      putRows(this.region, 3, value2, keyPrefix2);
+      putRows(this.region, 3, value2, keyPrefix3);
+      System.out.println("Checking values for key: " + keyPrefix1);
+      assertEquals("Got back incorrect number of rows from scan", 3,
+          getNumberOfRows(keyPrefix1, value2, this.region));
+      System.out.println("Checking values for key: " + keyPrefix2);
+      assertEquals("Got back incorrect number of rows from scan", 3,
+          getNumberOfRows(keyPrefix2, value2, this.region));
+      System.out.println("Checking values for key: " + keyPrefix3);
+      assertEquals("Got back incorrect number of rows from scan", 3,
+          getNumberOfRows(keyPrefix3, value2, this.region));
+      deleteColumns(this.region, value2, keyPrefix1);
+      deleteColumns(this.region, value2, keyPrefix2);
+      deleteColumns(this.region, value2, keyPrefix3);
+      System.out.println("Starting important checks.....");
+      assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
+          0, getNumberOfRows(keyPrefix1, value2, this.region));
+      assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
+          0, getNumberOfRows(keyPrefix2, value2, this.region));
+      assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
+          0, getNumberOfRows(keyPrefix3, value2, this.region));
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   private void deleteColumns(HRegion r, String value, String keyPrefix)
@@ -454,17 +536,22 @@ public class TestHRegion extends HBaseTe
   public void testFamilyWithAndWithoutColon() throws Exception {
     byte [] b = Bytes.toBytes(getName());
     byte [] cf = Bytes.toBytes("cf");
-    initHRegion(b, getName(), cf);
-    Put p = new Put(b);
-    byte [] cfwithcolon = Bytes.toBytes("cf:");
-    p.add(cfwithcolon, cfwithcolon, cfwithcolon);
-    boolean exception = false;
-    try {
-      this.region.put(p);
-    } catch (NoSuchColumnFamilyException e) {
-      exception = true;
+    this.region = initHRegion(b, getName(), cf);
+    try {
+      Put p = new Put(b);
+      byte [] cfwithcolon = Bytes.toBytes("cf:");
+      p.add(cfwithcolon, cfwithcolon, cfwithcolon);
+      boolean exception = false;
+      try {
+        this.region.put(p);
+      } catch (NoSuchColumnFamilyException e) {
+        exception = true;
+      }
+      assertTrue(exception);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    assertTrue(exception);
   }
 
   @SuppressWarnings("unchecked")
@@ -473,96 +560,100 @@ public class TestHRegion extends HBaseTe
     byte[] cf = Bytes.toBytes("cf");
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    initHRegion(b, getName(), cf);
+    this.region = initHRegion(b, getName(), cf);
+    try {
+      HLog.getSyncOps(); // clear counter from prior tests
+      assertEquals(0, HLog.getSyncOps());
+
+      LOG.info("First a batch put with all valid puts");
+      final Put[] puts = new Put[10];
+      for (int i = 0; i < 10; i++) {
+        puts[i] = new Put(Bytes.toBytes("row_" + i));
+        puts[i].add(cf, qual, val);
+      }
+
+      OperationStatus[] codes = this.region.put(puts);
+      assertEquals(10, codes.length);
+      for (int i = 0; i < 10; i++) {
+        assertEquals(OperationStatusCode.SUCCESS, codes[i]
+            .getOperationStatusCode());
+      }
+      assertEquals(1, HLog.getSyncOps());
+
+      LOG.info("Next a batch put with one invalid family");
+      puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
+      codes = this.region.put(puts);
+      assertEquals(10, codes.length);
+      for (int i = 0; i < 10; i++) {
+        assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+          OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+      }
+      assertEquals(1, HLog.getSyncOps());
+
+      LOG.info("Next a batch put that has to break into two batches to avoid a lock");
+      Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
+
+      MultithreadedTestUtil.TestContext ctx =
+          new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
+      final AtomicReference<OperationStatus[]> retFromThread =
+          new AtomicReference<OperationStatus[]>();
+          TestThread putter = new TestThread(ctx) {
+            @Override
+            public void doWork() throws IOException {
+              retFromThread.set(region.put(puts));
+            }
+          };
+          LOG.info("...starting put thread while holding lock");
+          ctx.addThread(putter);
+          ctx.startThreads();
+
+          LOG.info("...waiting for put thread to sync first time");
+          long startWait = System.currentTimeMillis();
+          while (HLog.getSyncOps() == 0) {
+            Thread.sleep(100);
+            if (System.currentTimeMillis() - startWait > 10000) {
+              fail("Timed out waiting for thread to sync first minibatch");
+            }
+          }
+          LOG.info("...releasing row lock, which should let put thread continue");
+          region.releaseRowLock(lockedRow);
+          LOG.info("...joining on thread");
+          ctx.stop();
+          LOG.info("...checking that next batch was synced");
+          assertEquals(1, HLog.getSyncOps());
+          codes = retFromThread.get();
+          for (int i = 0; i < 10; i++) {
+            assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+              OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+          }
+
+          LOG.info("Nexta, a batch put which uses an already-held lock");
+          lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
+          LOG.info("...obtained row lock");
+          List<Pair<Put, Integer>> putsAndLocks = Lists.newArrayList();
+          for (int i = 0; i < 10; i++) {
+            Pair<Put, Integer> pair = new Pair<Put, Integer>(puts[i], null);
+            if (i == 2) pair.setSecond(lockedRow);
+            putsAndLocks.add(pair);
+          }
 
-    HLog.getSyncOps(); // clear counter from prior tests
-    assertEquals(0, HLog.getSyncOps());
+          codes = region.put(putsAndLocks.toArray(new Pair[0]));
+          LOG.info("...performed put");
+          for (int i = 0; i < 10; i++) {
+            assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
+              OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
+          }
+          // Make sure we didn't do an extra batch
+          assertEquals(1, HLog.getSyncOps());
 
-    LOG.info("First a batch put with all valid puts");
-    final Put[] puts = new Put[10];
-    for (int i = 0; i < 10; i++) {
-      puts[i] = new Put(Bytes.toBytes("row_" + i));
-      puts[i].add(cf, qual, val);
-    }
-
-    OperationStatus[] codes = this.region.put(puts);
-    assertEquals(10, codes.length);
-    for (int i = 0; i < 10; i++) {
-      assertEquals(OperationStatusCode.SUCCESS, codes[i]
-          .getOperationStatusCode());
-    }
-    assertEquals(1, HLog.getSyncOps());
-
-    LOG.info("Next a batch put with one invalid family");
-    puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
-    codes = this.region.put(puts);
-    assertEquals(10, codes.length);
-    for (int i = 0; i < 10; i++) {
-      assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
-        OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
-    }
-    assertEquals(1, HLog.getSyncOps());
-
-    LOG.info("Next a batch put that has to break into two batches to avoid a lock");
-    Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
-
-    MultithreadedTestUtil.TestContext ctx =
-      new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
-    final AtomicReference<OperationStatus[]> retFromThread =
-      new AtomicReference<OperationStatus[]>();
-    TestThread putter = new TestThread(ctx) {
-      @Override
-      public void doWork() throws IOException {
-        retFromThread.set(region.put(puts));
-      }
-    };
-    LOG.info("...starting put thread while holding lock");
-    ctx.addThread(putter);
-    ctx.startThreads();
-
-    LOG.info("...waiting for put thread to sync first time");
-    long startWait = System.currentTimeMillis();
-    while (HLog.getSyncOps() == 0) {
-      Thread.sleep(100);
-      if (System.currentTimeMillis() - startWait > 10000) {
-        fail("Timed out waiting for thread to sync first minibatch");
-      }
-    }
-    LOG.info("...releasing row lock, which should let put thread continue");
-    region.releaseRowLock(lockedRow);
-    LOG.info("...joining on thread");
-    ctx.stop();
-    LOG.info("...checking that next batch was synced");
-    assertEquals(1, HLog.getSyncOps());
-    codes = retFromThread.get();
-    for (int i = 0; i < 10; i++) {
-      assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
-        OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
-    }
-
-    LOG.info("Nexta, a batch put which uses an already-held lock");
-    lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
-    LOG.info("...obtained row lock");
-    List<Pair<Put, Integer>> putsAndLocks = Lists.newArrayList();
-    for (int i = 0; i < 10; i++) {
-      Pair<Put, Integer> pair = new Pair<Put, Integer>(puts[i], null);
-      if (i == 2) pair.setSecond(lockedRow);
-      putsAndLocks.add(pair);
-    }
-
-    codes = region.put(putsAndLocks.toArray(new Pair[0]));
-    LOG.info("...performed put");
-    for (int i = 0; i < 10; i++) {
-      assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
-        OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
-    }
-    // Make sure we didn't do an extra batch
-    assertEquals(1, HLog.getSyncOps());
-
-    // Make sure we still hold lock
-    assertTrue(region.isRowLocked(lockedRow));
-    LOG.info("...releasing lock");
-    region.releaseRowLock(lockedRow);
+          // Make sure we still hold lock
+          assertTrue(region.isRowLocked(lockedRow));
+          LOG.info("...releasing lock");
+          region.releaseRowLock(lockedRow);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -580,65 +671,68 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
-
-    //Putting empty data in key
-    Put put = new Put(row1);
-    put.add(fam1, qf1, emptyVal);
-
-    //checkAndPut with empty value
-    boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(emptyVal), put, lockId, true);
-    assertTrue(res);
-    
-    //Putting data in key
-    put = new Put(row1);
-    put.add(fam1, qf1, val1);
-
-    //checkAndPut with correct value
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(emptyVal), put, lockId, true);
-    assertTrue(res);
-
-    // not empty anymore
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(emptyVal), put, lockId, true);
-    assertFalse(res);
-
-    Delete delete = new Delete(row1);
-    delete.deleteColumn(fam1, qf1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(emptyVal), delete, lockId, true);
-    assertFalse(res);
-
-    put = new Put(row1);
-    put.add(fam1, qf1, val2);
-    //checkAndPut with correct value
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(val1), put, lockId, true);
-    assertTrue(res);
-
-    //checkAndDelete with correct value
-    delete = new Delete(row1);
-    delete.deleteColumn(fam1, qf1);
-    delete.deleteColumn(fam1, qf1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(val2), delete, lockId, true);
-    assertTrue(res);
-
-    delete = new Delete(row1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new BinaryComparator(emptyVal), delete, lockId, true);
-    assertTrue(res);
-
-    //checkAndPut looking for a null value
-    put = new Put(row1);
-    put.add(fam1, qf1, val1);
-
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-        new NullComparator(), put, lockId, true);
-    assertTrue(res);
-
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      //Putting empty data in key
+      Put put = new Put(row1);
+      put.add(fam1, qf1, emptyVal);
+
+      //checkAndPut with empty value
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), put, lockId, true);
+      assertTrue(res);
+
+      //Putting data in key
+      put = new Put(row1);
+      put.add(fam1, qf1, val1);
+
+      //checkAndPut with correct value
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), put, lockId, true);
+      assertTrue(res);
+
+      // not empty anymore
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), put, lockId, true);
+      assertFalse(res);
+
+      Delete delete = new Delete(row1);
+      delete.deleteColumn(fam1, qf1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), delete, lockId, true);
+      assertFalse(res);
+
+      put = new Put(row1);
+      put.add(fam1, qf1, val2);
+      //checkAndPut with correct value
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val1), put, lockId, true);
+      assertTrue(res);
+
+      //checkAndDelete with correct value
+      delete = new Delete(row1);
+      delete.deleteColumn(fam1, qf1);
+      delete.deleteColumn(fam1, qf1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val2), delete, lockId, true);
+      assertTrue(res);
+
+      delete = new Delete(row1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), delete, lockId, true);
+      assertTrue(res);
+
+      //checkAndPut looking for a null value
+      put = new Put(row1);
+      put.add(fam1, qf1, val1);
+
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new NullComparator(), put, lockId, true);
+      assertTrue(res);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testCheckAndMutate_WithWrongValue() throws IOException{
@@ -652,24 +746,28 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
-
-    //Putting data in key
-    Put put = new Put(row1);
-    put.add(fam1, qf1, val1);
-    region.put(put);
-
-    //checkAndPut with wrong value
-    boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val2), put, lockId, true);
-    assertEquals(false, res);
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      //Putting data in key
+      Put put = new Put(row1);
+      put.add(fam1, qf1, val1);
+      region.put(put);
 
-    //checkAndDelete with wrong value
-    Delete delete = new Delete(row1);
-    delete.deleteFamily(fam1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val2), delete, lockId, true);
-    assertEquals(false, res);
+      //checkAndPut with wrong value
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val2), put, lockId, true);
+      assertEquals(false, res);
+
+      //checkAndDelete with wrong value
+      Delete delete = new Delete(row1);
+      delete.deleteFamily(fam1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val2), delete, lockId, true);
+      assertEquals(false, res);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testCheckAndMutate_WithCorrectValue() throws IOException{
@@ -682,24 +780,28 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
-
-    //Putting data in key
-    Put put = new Put(row1);
-    put.add(fam1, qf1, val1);
-    region.put(put);
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      //Putting data in key
+      Put put = new Put(row1);
+      put.add(fam1, qf1, val1);
+      region.put(put);
 
-    //checkAndPut with correct value
-    boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val1), put, lockId, true);
-    assertEquals(true, res);
-
-    //checkAndDelete with correct value
-    Delete delete = new Delete(row1);
-    delete.deleteColumn(fam1, qf1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val1), put, lockId, true);
-    assertEquals(true, res);
+      //checkAndPut with correct value
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val1), put, lockId, true);
+      assertEquals(true, res);
+
+      //checkAndDelete with correct value
+      Delete delete = new Delete(row1);
+      delete.deleteColumn(fam1, qf1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val1), put, lockId, true);
+      assertEquals(true, res);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testCheckAndPut_ThatPutWasWritten() throws IOException{
@@ -716,52 +818,59 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
-
-    //Putting data in the key to check
-    Put put = new Put(row1);
-    put.add(fam1, qf1, val1);
-    region.put(put);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      //Putting data in the key to check
+      Put put = new Put(row1);
+      put.add(fam1, qf1, val1);
+      region.put(put);
 
-    //Creating put to add
-    long ts = System.currentTimeMillis();
-    KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
-    put = new Put(row1);
-    put.add(kv);
-
-    //checkAndPut with wrong value
-    Store store = region.getStore(fam1);
-    store.memstore.kvset.size();
-
-    boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val1), put, lockId, true);
-    assertEquals(true, res);
-    store.memstore.kvset.size();
-
-    Get get = new Get(row1);
-    get.addColumn(fam2, qf1);
-    KeyValue [] actual = region.get(get, null).raw();
-
-    KeyValue [] expected = {kv};
-
-    assertEquals(expected.length, actual.length);
-    for(int i=0; i<actual.length; i++) {
-      assertEquals(expected[i], actual[i]);
+      //Creating put to add
+      long ts = System.currentTimeMillis();
+      KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
+      put = new Put(row1);
+      put.add(kv);
+
+      //checkAndPut with wrong value
+      Store store = region.getStore(fam1);
+      store.memstore.kvset.size();
+
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val1), put, lockId, true);
+      assertEquals(true, res);
+      store.memstore.kvset.size();
+
+      Get get = new Get(row1);
+      get.addColumn(fam2, qf1);
+      KeyValue [] actual = region.get(get, null).raw();
+
+      KeyValue [] expected = {kv};
+
+      assertEquals(expected.length, actual.length);
+      for(int i=0; i<actual.length; i++) {
+        assertEquals(expected[i], actual[i]);
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-
   }
 
   public void testCheckAndPut_wrongRowInPut() throws IOException {
-    initHRegion(tableName, this.getName(), COLUMNS);
-
-    Put put = new Put(row2);
-    put.add(fam1, qual1, value1);
+    this.region = initHRegion(tableName, this.getName(), COLUMNS);
     try {
-    boolean res = region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
-        new BinaryComparator(value2), put, null, false);
-      fail();
-    } catch (DoNotRetryIOException expected) {
-      // expected exception.
+      Put put = new Put(row2);
+      put.add(fam1, qual1, value1);
+      try {
+        boolean res = region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
+            new BinaryComparator(value2), put, null, false);
+        fail();
+      } catch (DoNotRetryIOException expected) {
+        // expected exception.
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -783,60 +892,64 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      //Put content
+      Put put = new Put(row1);
+      put.add(fam1, qf1, val1);
+      region.put(put);
+      Threads.sleep(2);
 
-    //Put content
-    Put put = new Put(row1);
-    put.add(fam1, qf1, val1);
-    region.put(put);
-    Threads.sleep(2);
-
-    put = new Put(row1);
-    put.add(fam1, qf1, val2);
-    put.add(fam2, qf1, val3);
-    put.add(fam2, qf2, val2);
-    put.add(fam2, qf3, val1);
-    put.add(fam1, qf3, val1);
-    region.put(put);
-
-    //Multi-column delete
-    Delete delete = new Delete(row1);
-    delete.deleteColumn(fam1, qf1);
-    delete.deleteColumn(fam2, qf1);
-    delete.deleteColumn(fam1, qf3);
-    boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val2), delete, lockId, true);
-    assertEquals(true, res);
-
-    Get get = new Get(row1);
-    get.addColumn(fam1, qf1);
-    get.addColumn(fam1, qf3);
-    get.addColumn(fam2, qf2);
-    Result r = region.get(get, null);
-    assertEquals(2, r.size());
-    assertEquals(val1, r.getValue(fam1, qf1));
-    assertEquals(val2, r.getValue(fam2, qf2));
-
-    //Family delete
-    delete = new Delete(row1);
-    delete.deleteFamily(fam2);
-    res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL,
-      new BinaryComparator(emptyVal), delete, lockId, true);
-    assertEquals(true, res);
-
-    get = new Get(row1);
-    r = region.get(get, null);
-    assertEquals(1, r.size());
-    assertEquals(val1, r.getValue(fam1, qf1));
-
-    //Row delete
-    delete = new Delete(row1);
-    res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
-      new BinaryComparator(val1), delete, lockId, true);
-    assertEquals(true, res);
-    get = new Get(row1);
-    r = region.get(get, null);
-    assertEquals(0, r.size());
+      put = new Put(row1);
+      put.add(fam1, qf1, val2);
+      put.add(fam2, qf1, val3);
+      put.add(fam2, qf2, val2);
+      put.add(fam2, qf3, val1);
+      put.add(fam1, qf3, val1);
+      region.put(put);
+
+      //Multi-column delete
+      Delete delete = new Delete(row1);
+      delete.deleteColumn(fam1, qf1);
+      delete.deleteColumn(fam2, qf1);
+      delete.deleteColumn(fam1, qf3);
+      boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val2), delete, lockId, true);
+      assertEquals(true, res);
+
+      Get get = new Get(row1);
+      get.addColumn(fam1, qf1);
+      get.addColumn(fam1, qf3);
+      get.addColumn(fam2, qf2);
+      Result r = region.get(get, null);
+      assertEquals(2, r.size());
+      assertEquals(val1, r.getValue(fam1, qf1));
+      assertEquals(val2, r.getValue(fam2, qf2));
+
+      //Family delete
+      delete = new Delete(row1);
+      delete.deleteFamily(fam2);
+      res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL,
+          new BinaryComparator(emptyVal), delete, lockId, true);
+      assertEquals(true, res);
+
+      get = new Get(row1);
+      r = region.get(get, null);
+      assertEquals(1, r.size());
+      assertEquals(val1, r.getValue(fam1, qf1));
+
+      //Row delete
+      delete = new Delete(row1);
+      res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
+          new BinaryComparator(val1), delete, lockId, true);
+      assertEquals(true, res);
+      get = new Get(row1);
+      r = region.get(get, null);
+      assertEquals(0, r.size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -854,20 +967,24 @@ public class TestHRegion extends HBaseTe
     put.add(fam1, qual, 2, value);
 
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
-
-    region.put(put);
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      region.put(put);
 
-    // We do support deleting more than 1 'latest' version
-    Delete delete = new Delete(row1);
-    delete.deleteColumn(fam1, qual);
-    delete.deleteColumn(fam1, qual);
-    region.delete(delete, null, false);
-
-    Get get = new Get(row1);
-    get.addFamily(fam1);
-    Result r = region.get(get, null);
-    assertEquals(0, r.size());
+      // We do support deleting more than 1 'latest' version
+      Delete delete = new Delete(row1);
+      delete.deleteColumn(fam1, qual);
+      delete.deleteColumn(fam1, qual);
+      region.delete(delete, null, false);
+
+      Get get = new Get(row1);
+      get.addFamily(fam1);
+      Result r = region.get(get, null);
+      assertEquals(0, r.size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testDelete_CheckFamily() throws IOException {
@@ -880,33 +997,37 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1, fam2, fam3);
-
-    List<KeyValue> kvs  = new ArrayList<KeyValue>();
-    kvs.add(new KeyValue(row1, fam4, null, null));
+    this.region = initHRegion(tableName, method, fam1, fam2, fam3);
+    try {
+      List<KeyValue> kvs  = new ArrayList<KeyValue>();
+      kvs.add(new KeyValue(row1, fam4, null, null));
 
 
-    //testing existing family
-    byte [] family = fam2;
-    try {
-      Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
-      deleteMap.put(family, kvs);
-      region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
-    } catch (Exception e) {
-      assertTrue("Family " +new String(family)+ " does not exist", false);
-    }
+      //testing existing family
+      byte [] family = fam2;
+      try {
+        Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+        deleteMap.put(family, kvs);
+        region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+      } catch (Exception e) {
+        assertTrue("Family " +new String(family)+ " does not exist", false);
+      }
 
-    //testing non existing family
-    boolean ok = false;
-    family = fam4;
-    try {
-      Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
-      deleteMap.put(family, kvs);
-      region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
-    } catch (Exception e) {
-      ok = true;
+      //testing non existing family
+      boolean ok = false;
+      family = fam4;
+      try {
+        Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+        deleteMap.put(family, kvs);
+        region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
+      } catch (Exception e) {
+        ok = true;
+      }
+      assertEquals("Family " +new String(family)+ " does exist", true, ok);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    assertEquals("Family " +new String(family)+ " does exist", true, ok);
   }
 
   public void testDelete_mixed() throws IOException, InterruptedException {
@@ -914,62 +1035,67 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    initHRegion(tableName, method, families);
-    EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
+    this.region = initHRegion(tableName, method, families);
+    try {
+      EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
-    byte [] row = Bytes.toBytes("table_name");
-    // column names
-    byte [] serverinfo = Bytes.toBytes("serverinfo");
-    byte [] splitA = Bytes.toBytes("splitA");
-    byte [] splitB = Bytes.toBytes("splitB");
-
-    // add some data:
-    Put put = new Put(row);
-    put.add(fam, splitA, Bytes.toBytes("reference_A"));
-    region.put(put);
-
-    put = new Put(row);
-    put.add(fam, splitB, Bytes.toBytes("reference_B"));
-    region.put(put);
-
-    put = new Put(row);
-    put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
-    region.put(put);
+      byte [] row = Bytes.toBytes("table_name");
+      // column names
+      byte [] serverinfo = Bytes.toBytes("serverinfo");
+      byte [] splitA = Bytes.toBytes("splitA");
+      byte [] splitB = Bytes.toBytes("splitB");
 
-    // ok now delete a split:
-    Delete delete = new Delete(row);
-    delete.deleteColumns(fam, splitA);
-    region.delete(delete, null, true);
+      // add some data:
+      Put put = new Put(row);
+      put.add(fam, splitA, Bytes.toBytes("reference_A"));
+      region.put(put);
 
-    // assert some things:
-    Get get = new Get(row).addColumn(fam, serverinfo);
-    Result result = region.get(get, null);
-    assertEquals(1, result.size());
+      put = new Put(row);
+      put.add(fam, splitB, Bytes.toBytes("reference_B"));
+      region.put(put);
 
-    get = new Get(row).addColumn(fam, splitA);
-    result = region.get(get, null);
-    assertEquals(0, result.size());
+      put = new Put(row);
+      put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
+      region.put(put);
 
-    get = new Get(row).addColumn(fam, splitB);
-    result = region.get(get, null);
-    assertEquals(1, result.size());
+      // ok now delete a split:
+      Delete delete = new Delete(row);
+      delete.deleteColumns(fam, splitA);
+      region.delete(delete, null, true);
+
+      // assert some things:
+      Get get = new Get(row).addColumn(fam, serverinfo);
+      Result result = region.get(get, null);
+      assertEquals(1, result.size());
 
-    // Assert that after a delete, I can put.
-    put = new Put(row);
-    put.add(fam, splitA, Bytes.toBytes("reference_A"));
-    region.put(put);
-    get = new Get(row);
-    result = region.get(get, null);
-    assertEquals(3, result.size());
-
-    // Now delete all... then test I can add stuff back
-    delete = new Delete(row);
-    region.delete(delete, null, false);
-    assertEquals(0, region.get(get, null).size());
+      get = new Get(row).addColumn(fam, splitA);
+      result = region.get(get, null);
+      assertEquals(0, result.size());
 
-    region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
-    result = region.get(get, null);
-    assertEquals(1, result.size());
+      get = new Get(row).addColumn(fam, splitB);
+      result = region.get(get, null);
+      assertEquals(1, result.size());
+
+      // Assert that after a delete, I can put.
+      put = new Put(row);
+      put.add(fam, splitA, Bytes.toBytes("reference_A"));
+      region.put(put);
+      get = new Get(row);
+      result = region.get(get, null);
+      assertEquals(3, result.size());
+
+      // Now delete all... then test I can add stuff back
+      delete = new Delete(row);
+      region.delete(delete, null, false);
+      assertEquals(0, region.get(get, null).size());
+
+      region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
+      result = region.get(get, null);
+      assertEquals(1, result.size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testDeleteRowWithFutureTs() throws IOException {
@@ -977,34 +1103,38 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    initHRegion(tableName, method, families);
-
-    byte [] row = Bytes.toBytes("table_name");
-    // column names
-    byte [] serverinfo = Bytes.toBytes("serverinfo");
-
-    // add data in the far future
-    Put put = new Put(row);
-    put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
-    region.put(put);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      byte [] row = Bytes.toBytes("table_name");
+      // column names
+      byte [] serverinfo = Bytes.toBytes("serverinfo");
 
-    // now delete something in the present
-    Delete delete = new Delete(row);
-    region.delete(delete, null, true);
+      // add data in the far future
+      Put put = new Put(row);
+      put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
+      region.put(put);
 
-    // make sure we still see our data
-    Get get = new Get(row).addColumn(fam, serverinfo);
-    Result result = region.get(get, null);
-    assertEquals(1, result.size());
+      // now delete something in the present
+      Delete delete = new Delete(row);
+      region.delete(delete, null, true);
+
+      // make sure we still see our data
+      Get get = new Get(row).addColumn(fam, serverinfo);
+      Result result = region.get(get, null);
+      assertEquals(1, result.size());
+
+      // delete the future row
+      delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
+      region.delete(delete, null, true);
 
-    // delete the future row
-    delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
-    region.delete(delete, null, true);
-
-    // make sure it is gone
-    get = new Get(row).addColumn(fam, serverinfo);
-    result = region.get(get, null);
-    assertEquals(0, result.size());
+      // make sure it is gone
+      get = new Get(row).addColumn(fam, serverinfo);
+      result = region.get(get, null);
+      assertEquals(0, result.size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   /**
@@ -1016,41 +1146,45 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("info");
     byte [][] families = {fam};
     String method = this.getName();
-    initHRegion(tableName, method, families);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      byte [] row = Bytes.toBytes("row1");
+      // column names
+      byte [] qual = Bytes.toBytes("qual");
 
-    byte [] row = Bytes.toBytes("row1");
-    // column names
-    byte [] qual = Bytes.toBytes("qual");
-
-    // add data with LATEST_TIMESTAMP, put without WAL
-    Put put = new Put(row);
-    put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
-    region.put(put, false);
+      // add data with LATEST_TIMESTAMP, put without WAL
+      Put put = new Put(row);
+      put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
+      region.put(put, false);
 
-    // Make sure it shows up with an actual timestamp
-    Get get = new Get(row).addColumn(fam, qual);
-    Result result = region.get(get, null);
-    assertEquals(1, result.size());
-    KeyValue kv = result.raw()[0];
-    LOG.info("Got: " + kv);
-    assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
-        kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
-
-    // Check same with WAL enabled (historically these took different
-    // code paths, so check both)
-    row = Bytes.toBytes("row2");
-    put = new Put(row);
-    put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
-    region.put(put, true);
-
-    // Make sure it shows up with an actual timestamp
-    get = new Get(row).addColumn(fam, qual);
-    result = region.get(get, null);
-    assertEquals(1, result.size());
-    kv = result.raw()[0];
-    LOG.info("Got: " + kv);
-    assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
-        kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+      // Make sure it shows up with an actual timestamp
+      Get get = new Get(row).addColumn(fam, qual);
+      Result result = region.get(get, null);
+      assertEquals(1, result.size());
+      KeyValue kv = result.raw()[0];
+      LOG.info("Got: " + kv);
+      assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
+          kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+
+      // Check same with WAL enabled (historically these took different
+      // code paths, so check both)
+      row = Bytes.toBytes("row2");
+      put = new Put(row);
+      put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
+      region.put(put, true);
+
+      // Make sure it shows up with an actual timestamp
+      get = new Get(row).addColumn(fam, qual);
+      result = region.get(get, null);
+      assertEquals(1, result.size());
+      kv = result.raw()[0];
+      LOG.info("Got: " + kv);
+      assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
+          kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
 
   }
 
@@ -1058,39 +1192,42 @@ public class TestHRegion extends HBaseTe
     byte [] tableName = Bytes.toBytes("test_table");
     byte [] fam1 = Bytes.toBytes("columnA");
     byte [] fam2 = Bytes.toBytes("columnB");
-    initHRegion(tableName, getName(), fam1, fam2);
-
-    byte [] rowA = Bytes.toBytes("rowA");
-    byte [] rowB = Bytes.toBytes("rowB");
+    this.region = initHRegion(tableName, getName(), fam1, fam2);
+    try {
+      byte [] rowA = Bytes.toBytes("rowA");
+      byte [] rowB = Bytes.toBytes("rowB");
 
-    byte [] value = Bytes.toBytes("value");
+      byte [] value = Bytes.toBytes("value");
 
-    Delete delete = new Delete(rowA);
-    delete.deleteFamily(fam1);
+      Delete delete = new Delete(rowA);
+      delete.deleteFamily(fam1);
 
-    region.delete(delete, null, true);
+      region.delete(delete, null, true);
 
-    // now create data.
-    Put put = new Put(rowA);
-    put.add(fam2, null, value);
-    region.put(put);
-
-    put = new Put(rowB);
-    put.add(fam1, null, value);
-    put.add(fam2, null, value);
-    region.put(put);
+      // now create data.
+      Put put = new Put(rowA);
+      put.add(fam2, null, value);
+      region.put(put);
 
-    Scan scan = new Scan();
-    scan.addFamily(fam1).addFamily(fam2);
-    InternalScanner s = region.getScanner(scan);
-    List<KeyValue> results = new ArrayList<KeyValue>();
-    s.next(results);
-    assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
+      put = new Put(rowB);
+      put.add(fam1, null, value);
+      put.add(fam2, null, value);
+      region.put(put);
 
-    results.clear();
-    s.next(results);
-    assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
+      Scan scan = new Scan();
+      scan.addFamily(fam1).addFamily(fam2);
+      InternalScanner s = region.getScanner(scan);
+      List<KeyValue> results = new ArrayList<KeyValue>();
+      s.next(results);
+      assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
 
+      results.clear();
+      s.next(results);
+      assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testDeleteColumns_PostInsert() throws IOException,
@@ -1108,47 +1245,50 @@ public class TestHRegion extends HBaseTe
 
   public void doTestDelete_AndPostInsert(Delete delete)
       throws IOException, InterruptedException {
-    initHRegion(tableName, getName(), fam1);
-    EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
-    Put put = new Put(row);
-    put.add(fam1, qual1, value1);
-    region.put(put);
-
-    // now delete the value:
-    region.delete(delete, null, true);
+    this.region = initHRegion(tableName, getName(), fam1);
+    try {
+      EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
+      Put put = new Put(row);
+      put.add(fam1, qual1, value1);
+      region.put(put);
 
+      // now delete the value:
+      region.delete(delete, null, true);
 
-    // ok put data:
-    put = new Put(row);
-    put.add(fam1, qual1, value2);
-    region.put(put);
 
-    // ok get:
-    Get get = new Get(row);
-    get.addColumn(fam1, qual1);
+      // ok put data:
+      put = new Put(row);
+      put.add(fam1, qual1, value2);
+      region.put(put);
 
-    Result r = region.get(get, null);
-    assertEquals(1, r.size());
-    assertByteEquals(value2, r.getValue(fam1, qual1));
-
-    // next:
-    Scan scan = new Scan(row);
-    scan.addColumn(fam1, qual1);
-    InternalScanner s = region.getScanner(scan);
+      // ok get:
+      Get get = new Get(row);
+      get.addColumn(fam1, qual1);
 
-    List<KeyValue> results = new ArrayList<KeyValue>();
-    assertEquals(false, s.next(results));
-    assertEquals(1, results.size());
-    KeyValue kv = results.get(0);
-
-    assertByteEquals(value2, kv.getValue());
-    assertByteEquals(fam1, kv.getFamily());
-    assertByteEquals(qual1, kv.getQualifier());
-    assertByteEquals(row, kv.getRow());
+      Result r = region.get(get, null);
+      assertEquals(1, r.size());
+      assertByteEquals(value2, r.getValue(fam1, qual1));
+
+      // next:
+      Scan scan = new Scan(row);
+      scan.addColumn(fam1, qual1);
+      InternalScanner s = region.getScanner(scan);
+
+      List<KeyValue> results = new ArrayList<KeyValue>();
+      assertEquals(false, s.next(results));
+      assertEquals(1, results.size());
+      KeyValue kv = results.get(0);
+
+      assertByteEquals(value2, kv.getValue());
+      assertByteEquals(fam1, kv.getFamily());
+      assertByteEquals(qual1, kv.getQualifier());
+      assertByteEquals(row, kv.getRow());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
-
-
   public void testDelete_CheckTimestampUpdated()
   throws IOException {
     byte [] row1 = Bytes.toBytes("row1");
@@ -1158,27 +1298,31 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      //Building checkerList
+      List<KeyValue> kvs  = new ArrayList<KeyValue>();
+      kvs.add(new KeyValue(row1, fam1, col1, null));
+      kvs.add(new KeyValue(row1, fam1, col2, null));
+      kvs.add(new KeyValue(row1, fam1, col3, null));
+
+      Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
+      deleteMap.put(fam1, kvs);
+      region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
 
-    //Building checkerList
-    List<KeyValue> kvs  = new ArrayList<KeyValue>();
-    kvs.add(new KeyValue(row1, fam1, col1, null));
-    kvs.add(new KeyValue(row1, fam1, col2, null));
-    kvs.add(new KeyValue(row1, fam1, col3, null));
-
-    Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
-    deleteMap.put(fam1, kvs);
-    region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
-
-    // extract the key values out the memstore:
-    // This is kinda hacky, but better than nothing...
-    long now = System.currentTimeMillis();
-    KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
-    assertTrue(firstKv.getTimestamp() <= now);
-    now = firstKv.getTimestamp();
-    for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
-      assertTrue(kv.getTimestamp() <= now);
-      now = kv.getTimestamp();
+      // extract the key values out the memstore:
+      // This is kinda hacky, but better than nothing...
+      long now = System.currentTimeMillis();
+      KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
+      assertTrue(firstKv.getTimestamp() <= now);
+      now = firstKv.getTimestamp();
+      for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
+        assertTrue(kv.getTimestamp() <= now);
+        now = kv.getTimestamp();
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -1194,19 +1338,23 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
-
-    Get get = new Get(row1);
-    get.addColumn(fam2, col1);
-
-    //Test
+    this.region = initHRegion(tableName, method, fam1);
     try {
-      region.get(get, null);
-    } catch (NoSuchColumnFamilyException e){
-      assertFalse(false);
-      return;
+      Get get = new Get(row1);
+      get.addColumn(fam2, col1);
+
+      //Test
+      try {
+        region.get(get, null);
+      } catch (NoSuchColumnFamilyException e){
+        assertFalse(false);
+        return;
+      }
+      assertFalse(true);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    assertFalse(true);
   }
 
   public void testGet_Basic() throws IOException {
@@ -1221,44 +1369,48 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, fam1);
+    this.region = initHRegion(tableName, method, fam1);
+    try {
+      //Add to memstore
+      Put put = new Put(row1);
+      put.add(fam1, col1, null);
+      put.add(fam1, col2, null);
+      put.add(fam1, col3, null);
+      put.add(fam1, col4, null);
+      put.add(fam1, col5, null);
+      region.put(put);
 
-    //Add to memstore
-    Put put = new Put(row1);
-    put.add(fam1, col1, null);
-    put.add(fam1, col2, null);
-    put.add(fam1, col3, null);
-    put.add(fam1, col4, null);
-    put.add(fam1, col5, null);
-    region.put(put);
-
-    Get get = new Get(row1);
-    get.addColumn(fam1, col2);
-    get.addColumn(fam1, col4);
-    //Expected result
-    KeyValue kv1 = new KeyValue(row1, fam1, col2);
-    KeyValue kv2 = new KeyValue(row1, fam1, col4);
-    KeyValue [] expected = {kv1, kv2};
-
-    //Test
-    Result res = region.get(get, null);
-    assertEquals(expected.length, res.size());
-    for(int i=0; i<res.size(); i++){
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
-      assertEquals(0,
-          Bytes.compareTo(
-              expected[i].getQualifier(), res.raw()[i].getQualifier()));
-    }
-
-    // Test using a filter on a Get
-    Get g = new Get(row1);
-    final int count = 2;
-    g.setFilter(new ColumnCountGetFilter(count));
-    res = region.get(g, null);
-    assertEquals(count, res.size());
+      Get get = new Get(row1);
+      get.addColumn(fam1, col2);
+      get.addColumn(fam1, col4);
+      //Expected result
+      KeyValue kv1 = new KeyValue(row1, fam1, col2);
+      KeyValue kv2 = new KeyValue(row1, fam1, col4);
+      KeyValue [] expected = {kv1, kv2};
+
+      //Test
+      Result res = region.get(get, null);
+      assertEquals(expected.length, res.size());
+      for(int i=0; i<res.size(); i++){
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
+        assertEquals(0,
+            Bytes.compareTo(
+                expected[i].getQualifier(), res.raw()[i].getQualifier()));
+      }
+
+      // Test using a filter on a Get
+      Get g = new Get(row1);
+      final int count = 2;
+      g.setFilter(new ColumnCountGetFilter(count));
+      res = region.get(g, null);
+      assertEquals(count, res.size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   public void testGet_Empty() throws IOException {
@@ -1267,13 +1419,17 @@ public class TestHRegion extends HBaseTe
     byte [] fam = Bytes.toBytes("fam");
 
     String method = this.getName();
-    initHRegion(tableName, method, fam);
-
-    Get get = new Get(row);
-    get.addFamily(fam);
-    Result r = region.get(get, null);
+    this.region = initHRegion(tableName, method, fam);
+    try {
+      Get get = new Get(row);
+      get.addFamily(fam);
+      Result r = region.get(get, null);
 
-    assertTrue(r.isEmpty());
+      assertTrue(r.isEmpty());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   //Test that checked if there was anything special when reading from the ROOT
@@ -1283,68 +1439,72 @@ public class TestHRegion extends HBaseTe
   public void stestGet_Root() throws IOException {
     //Setting up region
     String method = this.getName();
-    initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
+    this.region = initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
+    try {
+      //Add to memstore
+      Put put = new Put(HConstants.EMPTY_START_ROW);
+      put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
+      region.put(put);
 
-    //Add to memstore
-    Put put = new Put(HConstants.EMPTY_START_ROW);
-    put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
-    region.put(put);
-
-    Get get = new Get(HConstants.EMPTY_START_ROW);
-    get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-
-    //Expected result
-    KeyValue kv1 = new KeyValue(HConstants.EMPTY_START_ROW,
-        HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
-    KeyValue [] expected = {kv1};
-
-    //Test from memstore
-    Result res = region.get(get, null);
-
-    assertEquals(expected.length, res.size());
-    for(int i=0; i<res.size(); i++){
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
-      assertEquals(0,
-          Bytes.compareTo(
-              expected[i].getQualifier(), res.raw()[i].getQualifier()));
-    }
+      Get get = new Get(HConstants.EMPTY_START_ROW);
+      get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
 
-    //flush
-    region.flushcache();
+      //Expected result
+      KeyValue kv1 = new KeyValue(HConstants.EMPTY_START_ROW,
+          HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
+      KeyValue [] expected = {kv1};
+
+      //Test from memstore
+      Result res = region.get(get, null);
+
+      assertEquals(expected.length, res.size());
+      for(int i=0; i<res.size(); i++){
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
+        assertEquals(0,
+            Bytes.compareTo(
+                expected[i].getQualifier(), res.raw()[i].getQualifier()));
+      }
 
-    //test2
-    res = region.get(get, null);
+      //flush
+      region.flushcache();
 
-    assertEquals(expected.length, res.size());
-    for(int i=0; i<res.size(); i++){
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
-      assertEquals(0,
-          Bytes.compareTo(
-              expected[i].getQualifier(), res.raw()[i].getQualifier()));
-    }
+      //test2
+      res = region.get(get, null);
 
-    //Scan
-    Scan scan = new Scan();
-    scan.addFamily(HConstants.CATALOG_FAMILY);
-    InternalScanner s = region.getScanner(scan);
-    List<KeyValue> result = new ArrayList<KeyValue>();
-    s.next(result);
-
-    assertEquals(expected.length, result.size());
-    for(int i=0; i<res.size(); i++){
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getRow(), result.get(i).getRow()));
-      assertEquals(0,
-          Bytes.compareTo(expected[i].getFamily(), result.get(i).getFamily()));
-      assertEquals(0,
-          Bytes.compareTo(
-              expected[i].getQualifier(), result.get(i).getQualifier()));
+      assertEquals(expected.length, res.size());
+      for(int i=0; i<res.size(); i++){
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
+        assertEquals(0,
+            Bytes.compareTo(
+                expected[i].getQualifier(), res.raw()[i].getQualifier()));
+      }
+
+      //Scan
+      Scan scan = new Scan();
+      scan.addFamily(HConstants.CATALOG_FAMILY);
+      InternalScanner s = region.getScanner(scan);
+      List<KeyValue> result = new ArrayList<KeyValue>();
+      s.next(result);
+
+      assertEquals(expected.length, result.size());
+      for(int i=0; i<res.size(); i++){
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getRow(), result.get(i).getRow()));
+        assertEquals(0,
+            Bytes.compareTo(expected[i].getFamily(), result.get(i).getFamily()));
+        assertEquals(0,
+            Bytes.compareTo(
+                expected[i].getQualifier(), result.get(i).getQualifier()));
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -1358,58 +1518,62 @@ public class TestHRegion extends HBaseTe
     Configuration hc = initSplit();
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, hc, families);
-
-    final int threadCount = 10;
-    final int lockCount = 10;
+    this.region = initHRegion(tableName, method, hc, families);
+    try {
+      final int threadCount = 10;
+      final int lockCount = 10;
 
-    List<Thread>threads = new ArrayList<Thread>(threadCount);
-    for (int i = 0; i < threadCount; i++) {
-      threads.add(new Thread(Integer.toString(i)) {
-        @Override
-        public void run() {
-          Integer [] lockids = new Integer[lockCount];
-          // Get locks.
-          for (int i = 0; i < lockCount; i++) {
-            try {
-              byte [] rowid = Bytes.toBytes(Integer.toString(i));
-              lockids[i] = region.obtainRowLock(rowid);
-              assertEquals(rowid, region.getRowFromLock(lockids[i]));
-              LOG.debug(getName() + " locked " + Bytes.toString(rowid));
-            } catch (IOException e) {
-              e.printStackTrace();
+      List<Thread>threads = new ArrayList<Thread>(threadCount);
+      for (int i = 0; i < threadCount; i++) {
+        threads.add(new Thread(Integer.toString(i)) {
+          @Override
+          public void run() {
+            Integer [] lockids = new Integer[lockCount];
+            // Get locks.
+            for (int i = 0; i < lockCount; i++) {
+              try {
+                byte [] rowid = Bytes.toBytes(Integer.toString(i));
+                lockids[i] = region.obtainRowLock(rowid);
+                assertEquals(rowid, region.getRowFromLock(lockids[i]));
+                LOG.debug(getName() + " locked " + Bytes.toString(rowid));
+              } catch (IOException e) {
+                e.printStackTrace();
+              }
             }
-          }
-          LOG.debug(getName() + " set " +
-              Integer.toString(lockCount) + " locks");
+            LOG.debug(getName() + " set " +
+                Integer.toString(lockCount) + " locks");
 
-          // Abort outstanding locks.
-          for (int i = lockCount - 1; i >= 0; i--) {
-            region.releaseRowLock(lockids[i]);
-            LOG.debug(getName() + " unlocked " + i);
+            // Abort outstanding locks.
+            for (int i = lockCount - 1; i >= 0; i--) {
+              region.releaseRowLock(lockids[i]);
+              LOG.debug(getName() + " unlocked " + i);
+            }
+            LOG.debug(getName() + " released " +
+                Integer.toString(lockCount) + " locks");
           }
-          LOG.debug(getName() + " released " +
-              Integer.toString(lockCount) + " locks");
-        }
-      });
-    }
+        });
+      }
 
-    // Startup all our threads.
-    for (Thread t : threads) {
-      t.start();
-    }
+      // Startup all our threads.
+      for (Thread t : threads) {
+        t.start();
+      }
 
-    // Now wait around till all are done.
-    for (Thread t: threads) {
-      while (t.isAlive()) {
-        try {
-          Thread.sleep(1);
-        } catch (InterruptedException e) {
-          // Go around again.
+      // Now wait around till all are done.
+      for (Thread t: threads) {
+        while (t.isAlive()) {
+          try {
+            Thread.sleep(1);
+          } catch (InterruptedException e) {
+            // Go around again.
+          }
         }
       }
+      LOG.info("locks completed.");
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    LOG.info("locks completed.");
   }
 
   //////////////////////////////////////////////////////////////////////////////
@@ -1421,46 +1585,51 @@ public class TestHRegion extends HBaseTe
     Configuration hc = initSplit();
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, hc, families);
+    this.region = initHRegion(tableName, method, hc, families);
     try {
-      LOG.info("" + addContent(region, fam3));
-      region.flushcache();
-      region.compactStores();
-      byte [] splitRow = region.checkSplit();
-      assertNotNull(splitRow);
-      LOG.info("SplitRow: " + Bytes.toString(splitRow));
-      HRegion [] subregions = splitRegion(region, splitRow);
       try {
-        // Need to open the regions.
-        for (int i = 0; i < subregions.length; i++) {
-          openClosedRegion(subregions[i]);
-          subregions[i].compactStores();
-        }
-        Path oldRegionPath = region.getRegionDir();
-        Path oldRegion1 = subregions[0].getRegionDir();
-        Path oldRegion2 = subregions[1].getRegionDir();
-        long startTime = System.currentTimeMillis();
-        region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
-        LOG.info("Merge regions elapsed time: " +
-            ((System.currentTimeMillis() - startTime) / 1000.0));
-        fs.delete(oldRegion1, true);
-        fs.delete(oldRegion2, true);
-        fs.delete(oldRegionPath, true);
-        LOG.info("splitAndMerge completed.");
-      } finally {
-        for (int i = 0; i < subregions.length; i++) {
-          try {
-            subregions[i].close();
-          } catch (IOException e) {
-            // Ignore.
+        LOG.info("" + addContent(region, fam3));
+        region.flushcache();
+        region.compactStores();
+        byte [] splitRow = region.checkSplit();
+        assertNotNull(splitRow);
+        LOG.info("SplitRow: " + Bytes.toString(splitRow));
+        HRegion [] subregions = splitRegion(region, splitRow);
+        try {
+          // Need to open the regions.
+          for (int i = 0; i < subregions.length; i++) {
+            openClosedRegion(subregions[i]);
+            subregions[i].compactStores();
           }
+          Path oldRegionPath = region.getRegionDir();
+          Path oldRegion1 = subregions[0].getRegionDir();
+          Path oldRegion2 = subregions[1].getRegionDir();
+          long startTime = System.currentTimeMillis();
+          region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
+          LOG.info("Merge regions elapsed time: " +
+              ((System.currentTimeMillis() - startTime) / 1000.0));
+          fs.delete(oldRegion1, true);
+          fs.delete(oldRegion2, true);
+          fs.delete(oldRegionPath, true);
+          LOG.info("splitAndMerge completed.");
+        } finally {
+          for (int i = 0; i < subregions.length; i++) {
+            try {
+              subregions[i].close();
+            } catch (IOException e) {
+              // Ignore.
+            }
+          }
+        }
+      } finally {
+        if (region != null) {
+          region.close();
+          region.getLog().closeAndDelete();
         }
       }
     } finally {
-      if (region != null) {
-        region.close();
-        region.getLog().closeAndDelete();
-      }
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -1508,15 +1677,19 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
-
-    Scan scan = new Scan();
-    scan.addFamily(fam1);
-    scan.addFamily(fam2);
+    this.region = initHRegion(tableName, method, families);
     try {
-      region.getScanner(scan);
-    } catch (Exception e) {
-      assertTrue("Families could not be found in Region", false);
+      Scan scan = new Scan();
+      scan.addFamily(fam1);
+      scan.addFamily(fam2);
+      try {
+        region.getScanner(scan);
+      } catch (Exception e) {
+        assertTrue("Families could not be found in Region", false);
+      }
+    } finally {
+      this.region.close();
+      this.region.getLog().closeAndDelete();
     }
   }
 
@@ -1529,17 +1702,21 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
-
-    Scan scan = new Scan();
-    scan.addFamily(fam2);
-    boolean ok = false;
+    this.region = initHRegion(tableName, method, families);
     try {
-      region.getScanner(scan);
-    } catch (Exception e) {
-      ok = true;
+      Scan scan = new Scan();
+      scan.addFamily(fam2);
+      boolean ok = false;
+      try {
+        region.getScanner(scan);
+      } catch (Exception e) {
+        ok = true;
+      }
+      assertTrue("Families could not be found in Region", ok);
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-    assertTrue("Families could not be found in Region", ok);
   }
 
   public void testGetScanner_WithNoFamilies() throws IOException {
@@ -1554,40 +1731,45 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
+    this.region = initHRegion(tableName, method, families);
+    try {
 
+      //Putting data in Region
+      Put put = new Put(row1);
+      put.add(fam1, null, null);
+      put.add(fam2, null, null);
+      put.add(fam3, null, null);
+      put.add(fam4, null, null);
+      region.put(put);
 
-    //Putting data in Region
-    Put put = new Put(row1);
-    put.add(fam1, null, null);
-    put.add(fam2, null, null);
-    put.add(fam3, null, null);
-    put.add(fam4, null, null);
-    region.put(put);
-
-    Scan scan = null;
-    HRegion.RegionScannerImpl is = null;
-
-    //Testing to see how many scanners that is produced by getScanner, starting
-    //with known number, 2 - current = 1
-    scan = new Scan();
-    scan.addFamily(fam2);
-    scan.addFamily(fam4);
-    is = (RegionScannerImpl) region.getScanner(scan);
-    MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
-    assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
-
-    scan = new Scan();
-    is = (RegionScannerImpl) region.getScanner(scan);
-    MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
-    assertEquals(families.length -1,
-        ((RegionScannerImpl)is).storeHeap.getHeap().size());
+      Scan scan = null;
+      HRegion.RegionScannerImpl is = null;
+
+      //Testing to see how many scanners that is produced by getScanner, starting
+      //with known number, 2 - current = 1
+      scan = new Scan();
+      scan.addFamily(fam2);
+      scan.addFamily(fam4);
+      is = (RegionScannerImpl) region.getScanner(scan);
+      MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
+      assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
+
+      scan = new Scan();
+      is = (RegionScannerImpl) region.getScanner(scan);
+      MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
+      assertEquals(families.length -1,
+          ((RegionScannerImpl)is).storeHeap.getHeap().size());
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
   }
 
   /**
    * This method tests https://issues.apache.org/jira/browse/HBASE-2516.
+   * @throws IOException 
    */
-  public void testGetScanner_WithRegionClosed() {
+  public void testGetScanner_WithRegionClosed() throws IOException {
     byte[] tableName = Bytes.toBytes("testtable");
     byte[] fam1 = Bytes.toBytes("fam1");
     byte[] fam2 = Bytes.toBytes("fam2");
@@ -1597,20 +1779,25 @@ public class TestHRegion extends HBaseTe
     //Setting up region
     String method = this.getName();
     try {
-      initHRegion(tableName, method, families);
+      this.region = initHRegion(tableName, method, families);
     } catch (IOException e) {
       e.printStackTrace();
       fail("Got IOException during initHRegion, " + e.getMessage());
     }
-    region.closed.set(true);
     try {
-      region.getScanner(null);
-      fail("Expected to get an exception during getScanner on a region that is closed");
-    } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
-      //this is the correct exception that is expected
-    } catch (IOException e) {
-      fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
-              + e.getMessage());
+      region.closed.set(true);
+      try {
+        region.getScanner(null);
+        fail("Expected to get an exception during getScanner on a region that is closed");
+      } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
+        //this is the correct exception that is expected
+      } catch (IOException e) {
+        fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
+            + e.getMessage());
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
   }
 
@@ -1628,53 +1815,56 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      //Putting data in Region
+      Put put = null;
+      put = new Put(row1);
+      put.add(fam1, null, ts, null);
+      put.add(fam2, null, ts, null);
+      put.add(fam3, null, ts, null);
+      put.add(fam4, null, ts, null);
+      region.put(put);
 
-    //Putting data in Region
-    Put put = null;
-    put = new Put(row1);
-    put.add(fam1, null, ts, null);
-    put.add(fam2, null, ts, null);
-    put.add(fam3, null, ts, null);
-    put.add(fam4, null, ts, null);
-    region.put(put);
-
-    put = new Put(row2);
-    put.add(fam1, null, ts, null);
-    put.add(fam2, null, ts, null);
-    put.add(fam3, null, ts, null);
-    put.add(fam4, null, ts, null);
-    region.put(put);
+      put = new Put(row2);
+      put.add(fam1, null, ts, null);
+      put.add(fam2, null, ts, null);
+      put.add(fam3, null, ts, null);
+      put.add(fam4, null, ts, null);
+      region.put(put);
 
-    Scan scan = new Scan();
-    scan.addFamily(fam2);
-    scan.addFamily(fam4);
-    InternalScanner is = region.getScanner(scan);
-
-    List<KeyValue> res = null;
-
-    //Result 1
-    List<KeyValue> expected1 = new ArrayList<KeyValue>();
-    expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
-    expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
-
-    res = new ArrayList<KeyValue>();
-    is.next(res);
-    for(int i=0; i<res.size(); i++) {
-      assertEquals(expected1.get(i), res.get(i));
-    }
-
-    //Result 2
-    List<KeyValue> expected2 = new ArrayList<KeyValue>();
-    expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
-    expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
-
-    res = new ArrayList<KeyValue>();
-    is.next(res);
-    for(int i=0; i<res.size(); i++) {
-      assertEquals(expected2.get(i), res.get(i));
+      Scan scan = new Scan();
+      scan.addFamily(fam2);
+      scan.addFamily(fam4);
+      InternalScanner is = region.getScanner(scan);
+
+      List<KeyValue> res = null;
+
+      //Result 1
+      List<KeyValue> expected1 = new ArrayList<KeyValue>();
+      expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
+      expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
+
+      res = new ArrayList<KeyValue>();
+      is.next(res);
+      for(int i=0; i<res.size(); i++) {
+        assertEquals(expected1.get(i), res.get(i));
+      }
+
+      //Result 2
+      List<KeyValue> expected2 = new ArrayList<KeyValue>();
+      expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
+      expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
+
+      res = new ArrayList<KeyValue>();
+      is.next(res);
+      for(int i=0; i<res.size(); i++) {
+        assertEquals(expected2.get(i), res.get(i));
+      }
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
     }
-
   }
 
   public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions()
@@ -1692,44 +1882,48 @@ public class TestHRegion extends HBaseTe
 
     //Setting up region
     String method = this.getName();
-    initHRegion(tableName, method, families);
+    this.region = initHRegion(tableName, method, families);
+    try {
+      //Putting data in Region
+      Put put = null;
+      KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
+      KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);

[... 1979 lines stripped ...]