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 2015/06/25 23:42:49 UTC

[1/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Repository: hbase
Updated Branches:
  refs/heads/master e6ed79219 -> b5b585304


http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
index 63154a8..7b6e25b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitter.java
@@ -34,9 +34,12 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.RegionSplitter.HexStringSplit;
@@ -109,22 +112,22 @@ public class TestRegionSplitter {
       expectedBounds.add(new byte[] {      0x20, 0, 0, 0, 0, 0, 0, 0});
       expectedBounds.add(new byte[] {      0x30, 0, 0, 0, 0, 0, 0, 0});
       expectedBounds.add(new byte[] {      0x40, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x50, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x60, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {      0x70, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0x80, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0x90, 0, 0, 0, 0, 0, 0, 0});
+      expectedBounds.add(new byte[] { 0x50, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { 0x60, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { 0x70, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { (byte) 0x90, 0, 0, 0, 0, 0, 0, 0 });
       expectedBounds.add(new byte[] {(byte)0xa0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0xb0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0xc0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0xd0, 0, 0, 0, 0, 0, 0, 0});
+      expectedBounds.add(new byte[] { (byte) 0xb0, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { (byte) 0xc0, 0, 0, 0, 0, 0, 0, 0 });
+      expectedBounds.add(new byte[] { (byte) 0xd0, 0, 0, 0, 0, 0, 0, 0 });
       expectedBounds.add(new byte[] {(byte)0xe0, 0, 0, 0, 0, 0, 0, 0});
-      expectedBounds.add(new byte[] {(byte)0xf0, 0, 0, 0, 0, 0, 0, 0});
+      expectedBounds.add(new byte[] { (byte) 0xf0, 0, 0, 0, 0, 0, 0, 0 });
       expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
 
       // Do table creation/pre-splitting and verification of region boundaries
       preSplitTableAndVerify(expectedBounds, UniformSplit.class.getSimpleName(),
-        TableName.valueOf("NewUniformPresplitTable"));
+          TableName.valueOf("NewUniformPresplitTable"));
     }
 
     /**
@@ -181,8 +184,7 @@ public class TestRegionSplitter {
 
         byte[][] twoRegionsSplits = splitter.split(2);
         assertEquals(1, twoRegionsSplits.length);
-        assertArrayEquals(twoRegionsSplits[0],
-            new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
+        assertArrayEquals(twoRegionsSplits[0], new byte[] { (byte) 0x80, 0, 0, 0, 0, 0, 0, 0 });
 
         byte[][] threeRegionsSplits = splitter.split(3);
         assertEquals(2, threeRegionsSplits.length);
@@ -194,7 +196,7 @@ public class TestRegionSplitter {
 
         // Check splitting existing regions that have start and end points
         byte[] splitPoint = splitter.split(new byte[] {0x10}, new byte[] {0x30});
-        assertArrayEquals(new byte[] {0x20}, splitPoint);
+        assertArrayEquals(new byte[] { 0x20 }, splitPoint);
 
         byte[] lastRow = new byte[] {xFF, xFF, xFF, xFF, xFF, xFF, xFF, xFF};
         assertArrayEquals(lastRow, splitter.lastRow());
@@ -202,15 +204,15 @@ public class TestRegionSplitter {
         assertArrayEquals(firstRow, splitter.firstRow());
 
         splitPoint = splitter.split(firstRow, new byte[] {0x20});
-        assertArrayEquals(splitPoint, new byte[] {0x10});
+        assertArrayEquals(splitPoint, new byte[] { 0x10 });
 
         splitPoint = splitter.split(new byte[] {(byte)0xdf, xFF, xFF, xFF, xFF,
                 xFF, xFF, xFF}, lastRow);
-        assertArrayEquals(splitPoint,
-                new byte[] {(byte)0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF});
+        assertArrayEquals(splitPoint, new byte[] { (byte) 0xef, xFF, xFF, xFF, xFF, xFF, xFF, xFF
+        });
 
         splitPoint = splitter.split(new byte[] {'a', 'a', 'a'}, new byte[] {'a', 'a', 'b'});
-        assertArrayEquals(splitPoint, new byte[] {'a', 'a', 'a', (byte)0x80 });
+        assertArrayEquals(splitPoint, new byte[] { 'a', 'a', 'a', (byte) 0x80 });
     }
 
   @Test
@@ -283,7 +285,7 @@ public class TestRegionSplitter {
         final Configuration conf = UTIL.getConfiguration();
         conf.setInt("split.count", numRegions);
         SplitAlgorithm splitAlgo = RegionSplitter.newSplitAlgoInstance(conf, splitClass);
-        RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] {CF_NAME}, conf);
+        RegionSplitter.createPresplitTable(tableName, splitAlgo, new String[] { CF_NAME }, conf);
         verifyBounds(expectedBounds, tableName);
     }
 
@@ -291,8 +293,7 @@ public class TestRegionSplitter {
   public void noopRollingSplit() throws Exception {
     final List<byte[]> expectedBounds = new ArrayList<byte[]>();
     expectedBounds.add(ArrayUtils.EMPTY_BYTE_ARRAY);
-    rollingSplitAndVerify(
-        TableName.valueOf(TestRegionSplitter.class.getSimpleName()),
+    rollingSplitAndVerify(TableName.valueOf(TestRegionSplitter.class.getSimpleName()),
         "UniformSplit", expectedBounds);
   }
 
@@ -309,27 +310,27 @@ public class TestRegionSplitter {
 
     private void verifyBounds(List<byte[]> expectedBounds, TableName tableName)
             throws Exception {
-        // Get region boundaries from the cluster and verify their endpoints
-        final int numRegions = expectedBounds.size()-1;
-        final HTable hTable = (HTable) UTIL.getConnection().getTable(tableName);
-        final Map<HRegionInfo, ServerName> regionInfoMap = hTable.getRegionLocations();
+      // Get region boundaries from the cluster and verify their endpoints
+      final int numRegions = expectedBounds.size()-1;
+      try (Table table = UTIL.getConnection().getTable(tableName);
+          RegionLocator locator = UTIL.getConnection().getRegionLocator(tableName)) {
+        final List<HRegionLocation> regionInfoMap = locator.getAllRegionLocations();
         assertEquals(numRegions, regionInfoMap.size());
-        for (Map.Entry<HRegionInfo, ServerName> entry: regionInfoMap.entrySet()) {
-            final HRegionInfo regionInfo = entry.getKey();
-            byte[] regionStart = regionInfo.getStartKey();
-            byte[] regionEnd = regionInfo.getEndKey();
-
-            // This region's start key should be one of the region boundaries
-            int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
-            assertNotSame(-1, startBoundaryIndex);
-
-            // This region's end key should be the region boundary that comes
-            // after the starting boundary.
-            byte[] expectedRegionEnd = expectedBounds.get(
-                    startBoundaryIndex+1);
-            assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
+        for (HRegionLocation entry : regionInfoMap) {
+          final HRegionInfo regionInfo = entry.getRegionInfo();
+          byte[] regionStart = regionInfo.getStartKey();
+          byte[] regionEnd = regionInfo.getEndKey();
+
+          // This region's start key should be one of the region boundaries
+          int startBoundaryIndex = indexOfBytes(expectedBounds, regionStart);
+          assertNotSame(-1, startBoundaryIndex);
+
+          // This region's end key should be the region boundary that comes
+          // after the starting boundary.
+          byte[] expectedRegionEnd = expectedBounds.get(startBoundaryIndex + 1);
+          assertEquals(0, Bytes.compareTo(regionEnd, expectedRegionEnd));
         }
-        hTable.close();
+      }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index f7f1f43..578ec85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -182,10 +184,14 @@ public class OfflineMetaRebuildTestCore {
     HTableDescriptor htd = tbl.getTableDescriptor();
     dumpMeta(htd);
 
-    Map<HRegionInfo, ServerName> hris = ((HTable)tbl).getRegionLocations();
-    for (Entry<HRegionInfo, ServerName> e : hris.entrySet()) {
-      HRegionInfo hri = e.getKey();
-      ServerName hsa = e.getValue();
+    List<HRegionLocation> regions;
+    try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+      regions = rl.getAllRegionLocations();
+    }
+
+    for (HRegionLocation e : regions) {
+      HRegionInfo hri = e.getRegionInfo();
+      ServerName hsa = e.getServerName();
       if (Bytes.compareTo(hri.getStartKey(), startKey) == 0
           && Bytes.compareTo(hri.getEndKey(), endKey) == 0) {
 


[3/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
index b3c29b7..9acc5ac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
@@ -389,55 +389,57 @@ public class TestHFileOutputFormat  {
       util.startMiniCluster();
       Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");
       HBaseAdmin admin = util.getHBaseAdmin();
-      HTable table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
+      Table table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
       assertEquals("Should start with empty table",
           0, util.countRows(table));
       int numRegions = -1;
-      try(RegionLocator r = table.getRegionLocator()) {
+      try(RegionLocator r = util.getConnection().getRegionLocator(TABLE_NAME)) {
         numRegions = r.getStartKeys().length;
-      }
-      assertEquals("Should make 5 regions", numRegions, 5);
 
-      // Generate the bulk load files
-      util.startMiniMapReduceCluster();
-      runIncrementalPELoad(conf, table, testDir);
-      // This doesn't write into the table, just makes files
-      assertEquals("HFOF should not touch actual table",
-          0, util.countRows(table));
+        assertEquals("Should make 5 regions", numRegions, 5);
 
+        // Generate the bulk load files
+        util.startMiniMapReduceCluster();
+        runIncrementalPELoad(conf, table, r, testDir);
+        // This doesn't write into the table, just makes files
+        assertEquals("HFOF should not touch actual table",
+            0, util.countRows(table));
 
-      // Make sure that a directory was created for every CF
-      int dir = 0;
-      for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
-        for (byte[] family : FAMILIES) {
-          if (Bytes.toString(family).equals(f.getPath().getName())) {
-            ++dir;
+
+
+        // Make sure that a directory was created for every CF
+        int dir = 0;
+        for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
+          for (byte[] family : FAMILIES) {
+            if (Bytes.toString(family).equals(f.getPath().getName())) {
+              ++dir;
+            }
           }
         }
-      }
-      assertEquals("Column family not found in FS.", FAMILIES.length, dir);
-
-      // handle the split case
-      if (shouldChangeRegions) {
-        LOG.info("Changing regions in table");
-        admin.disableTable(table.getName());
-        while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
-            getRegionStates().isRegionsInTransition()) {
-          Threads.sleep(200);
-          LOG.info("Waiting on table to finish disabling");
-        }
-        util.deleteTable(table.getName());
-        byte[][] newSplitKeys = generateRandomSplitKeys(14);
-        table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
-        while (table.getRegionLocations().size() != 15 ||
-            !admin.isTableAvailable(table.getName())) {
-          Thread.sleep(200);
-          LOG.info("Waiting for new region assignment to happen");
+        assertEquals("Column family not found in FS.", FAMILIES.length, dir);
+
+        // handle the split case
+        if (shouldChangeRegions) {
+          LOG.info("Changing regions in table");
+          admin.disableTable(table.getName());
+          while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
+              getRegionStates().isRegionsInTransition()) {
+            Threads.sleep(200);
+            LOG.info("Waiting on table to finish disabling");
+          }
+          util.deleteTable(table.getName());
+          byte[][] newSplitKeys = generateRandomSplitKeys(14);
+          table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
+          while (r.getAllRegionLocations().size() != 15 ||
+              !admin.isTableAvailable(table.getName())) {
+            Thread.sleep(200);
+            LOG.info("Waiting for new region assignment to happen");
+          }
         }
-      }
 
-      // Perform the actual load
-      new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
+        // Perform the actual load
+        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, r);
+      }
 
       // Ensure data shows up
       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@@ -473,7 +475,7 @@ public class TestHFileOutputFormat  {
   }
 
   private void runIncrementalPELoad(
-      Configuration conf, HTable table, Path outDir)
+      Configuration conf, Table table, RegionLocator regionLocator, Path outDir)
   throws Exception {
     Job job = new Job(conf, "testLocalMRIncrementalLoad");
     job.setWorkingDirectory(util.getDataTestDirOnTestFS("runIncrementalPELoad"));
@@ -482,12 +484,12 @@ public class TestHFileOutputFormat  {
         KeyValueSerialization.class.getName());
     setupRandomGeneratorMapper(job);
     HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(),
-        table.getRegionLocator());
+        regionLocator);
     FileOutputFormat.setOutputPath(job, outDir);
 
     Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
 
-    assertEquals(table.getRegionLocator().getAllRegionLocations().size(), job.getNumReduceTasks());
+    assertEquals(regionLocator.getAllRegionLocations().size(), job.getNumReduceTasks());
 
     assertTrue(job.waitForCompletion(true));
   }
@@ -910,7 +912,7 @@ public class TestHFileOutputFormat  {
       util.startMiniCluster();
       final FileSystem fs = util.getDFSCluster().getFileSystem();
       HBaseAdmin admin = util.getHBaseAdmin();
-      HTable table = util.createTable(TABLE_NAME, FAMILIES);
+      Table table = util.createTable(TABLE_NAME, FAMILIES);
       assertEquals("Should start with empty table", 0, util.countRows(table));
 
       // deep inspection: get the StoreFile dir
@@ -925,11 +927,13 @@ public class TestHFileOutputFormat  {
           true);
       util.startMiniMapReduceCluster();
 
-      for (int i = 0; i < 2; i++) {
-        Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);
-        runIncrementalPELoad(conf, table, testDir);
-        // Perform the actual load
-        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
+      try (RegionLocator regionLocator = util.getConnection().getRegionLocator(TABLE_NAME)) {
+        for (int i = 0; i < 2; i++) {
+          Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);
+          runIncrementalPELoad(conf, table, regionLocator, testDir);
+          // Perform the actual load
+          new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, regionLocator);
+        }
       }
 
       // Ensure data shows up
@@ -978,7 +982,7 @@ public class TestHFileOutputFormat  {
       Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction");
       final FileSystem fs = util.getDFSCluster().getFileSystem();
       Admin admin = util.getHBaseAdmin();
-      HTable table = util.createTable(TABLE_NAME, FAMILIES);
+      Table table = util.createTable(TABLE_NAME, FAMILIES);
       assertEquals("Should start with empty table", 0, util.countRows(table));
 
       // deep inspection: get the StoreFile dir
@@ -1004,10 +1008,13 @@ public class TestHFileOutputFormat  {
       conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
           true);
       util.startMiniMapReduceCluster();
-      runIncrementalPELoad(conf, table, testDir);
 
-      // Perform the actual load
-      new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
+      try (RegionLocator locator = util.getConnection().getRegionLocator(TABLE_NAME)) {
+        runIncrementalPELoad(conf, table, locator, testDir);
+
+        // Perform the actual load
+        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
+      }
 
       // Ensure data shows up
       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@@ -1066,17 +1073,17 @@ public class TestHFileOutputFormat  {
     if ("newtable".equals(args[0])) {
       TableName tname = TableName.valueOf(args[1]);
       byte[][] splitKeys = generateRandomSplitKeys(4);
-      HTable table = util.createTable(tname, FAMILIES, splitKeys);
+      Table table = util.createTable(tname, FAMILIES, splitKeys);
     } else if ("incremental".equals(args[0])) {
       TableName tname = TableName.valueOf(args[1]);
-      HTable table = (HTable) util.getConnection().getTable(tname);
+      Table table = util.getConnection().getTable(tname);
       Path outDir = new Path("incremental-out");
-      runIncrementalPELoad(conf, table, outDir);
+      try (RegionLocator locator = util.getConnection().getRegionLocator(tname)) {
+        runIncrementalPELoad(conf, table, locator, outDir);
+      }
     } else {
       throw new RuntimeException(
           "usage: TestHFileOutputFormat newtable | incremental");
     }
   }
-
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index 3b066f2..56e475b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.HRegionLocator;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -388,25 +389,26 @@ public class TestHFileOutputFormat2  {
     Configuration conf = util.getConfiguration();
     byte[][] splitKeys = generateRandomSplitKeys(4);
     util.startMiniCluster();
-    try {
-      HTable table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
-      Admin admin = table.getConnection().getAdmin();
+
+    Table table = util.createTable(TABLE_NAME, FAMILIES, splitKeys);
+    try (RegionLocator r = util.getConnection().getRegionLocator(TABLE_NAME)) {
+
+      Admin admin = util.getConnection().getAdmin();
       Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");
       assertEquals("Should start with empty table",
           0, util.countRows(table));
-      int numRegions = -1;
-      try (RegionLocator r = table.getRegionLocator()) {
+      int numRegions;
+
         numRegions = r.getStartKeys().length;
-      }
-      assertEquals("Should make 5 regions", numRegions, 5);
 
-      // Generate the bulk load files
-      util.startMiniMapReduceCluster();
-      runIncrementalPELoad(conf, table.getTableDescriptor(), table.getRegionLocator(), testDir);
-      // This doesn't write into the table, just makes files
-      assertEquals("HFOF should not touch actual table",
-          0, util.countRows(table));
+        assertEquals("Should make 5 regions", numRegions, 5);
 
+        // Generate the bulk load files
+        util.startMiniMapReduceCluster();
+        runIncrementalPELoad(conf, table.getTableDescriptor(), r, testDir);
+        // This doesn't write into the table, just makes files
+        assertEquals("HFOF should not touch actual table",
+            0, util.countRows(table));
 
       // Make sure that a directory was created for every CF
       int dir = 0;
@@ -432,7 +434,8 @@ public class TestHFileOutputFormat2  {
         byte[][] newSplitKeys = generateRandomSplitKeys(14);
         table = util.createTable(TABLE_NAME, FAMILIES, newSplitKeys);
 
-        while (table.getRegionLocator().getAllRegionLocations().size() != 15 ||
+        while (util.getConnection().getRegionLocator(TABLE_NAME)
+            .getAllRegionLocations().size() != 15 ||
             !admin.isTableAvailable(table.getName())) {
           Thread.sleep(200);
           LOG.info("Waiting for new region assignment to happen");
@@ -440,7 +443,7 @@ public class TestHFileOutputFormat2  {
       }
 
       // Perform the actual load
-      new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
+      new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, r);
 
       // Ensure data shows up
       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
@@ -911,9 +914,10 @@ public class TestHFileOutputFormat2  {
 
     util.startMiniCluster();
     try (Connection conn = ConnectionFactory.createConnection();
-        Admin admin = conn.getAdmin()) {
+        Admin admin = conn.getAdmin();
+        Table table = util.createTable(TABLE_NAME, FAMILIES);
+        RegionLocator locator = conn.getRegionLocator(TABLE_NAME)) {
       final FileSystem fs = util.getDFSCluster().getFileSystem();
-      HTable table = util.createTable(TABLE_NAME, FAMILIES);
       assertEquals("Should start with empty table", 0, util.countRows(table));
 
       // deep inspection: get the StoreFile dir
@@ -933,7 +937,7 @@ public class TestHFileOutputFormat2  {
         runIncrementalPELoad(conf, table.getTableDescriptor(), conn.getRegionLocator(TABLE_NAME),
             testDir);
         // Perform the actual load
-        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
+        new LoadIncrementalHFiles(conf).doBulkLoad(testDir, admin, table, locator);
       }
 
       // Ensure data shows up
@@ -1077,7 +1081,7 @@ public class TestHFileOutputFormat2  {
     if ("newtable".equals(args[0])) {
       TableName tname = TableName.valueOf(args[1]);
       byte[][] splitKeys = generateRandomSplitKeys(4);
-      try (HTable table = util.createTable(tname, FAMILIES, splitKeys)) {
+      try (Table table = util.createTable(tname, FAMILIES, splitKeys)) {
       }
     } else if ("incremental".equals(args[0])) {
       TableName tname = TableName.valueOf(args[1]);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java
index 762f530..1138d05 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHashTable.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -68,7 +69,7 @@ public class TestHashTable {
   
   @Test
   public void testHashTable() throws Exception {
-    final String tableName = "testHashTable";
+    final TableName tableName = TableName.valueOf("testHashTable");
     final byte[] family = Bytes.toBytes("family");
     final byte[] column1 = Bytes.toBytes("c1");
     final byte[] column2 = Bytes.toBytes("c2");
@@ -85,7 +86,7 @@ public class TestHashTable {
     
     long timestamp = 1430764183454L;
     // put rows into the first table
-    HTable t1 = TEST_UTIL.createTable(TableName.valueOf(tableName), family, splitRows);
+    Table t1 = TEST_UTIL.createTable(tableName, family, splitRows);
     for (int i = 0; i < numRows; i++) {
       Put p = new Put(Bytes.toBytes(i), timestamp);
       p.addColumn(family, column1, column1);
@@ -97,21 +98,21 @@ public class TestHashTable {
     
     HashTable hashTable = new HashTable(TEST_UTIL.getConfiguration());
     
-    Path testDir = TEST_UTIL.getDataTestDirOnTestFS(tableName);
+    Path testDir = TEST_UTIL.getDataTestDirOnTestFS(tableName.getNameAsString());
     
     long batchSize = 300;
     int code = hashTable.run(new String[] { 
         "--batchsize=" + batchSize,
         "--numhashfiles=" + numHashFiles,
         "--scanbatch=2",
-        tableName,
+        tableName.getNameAsString(),
         testDir.toString()});
     assertEquals("test job failed", 0, code);
     
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     
     HashTable.TableHash tableHash = HashTable.TableHash.read(fs.getConf(), testDir);
-    assertEquals(tableName, tableHash.tableName);
+    assertEquals(tableName.getNameAsString(), tableHash.tableName);
     assertEquals(batchSize, tableHash.batchSize);
     assertEquals(numHashFiles, tableHash.numHashFiles);
     assertEquals(numHashFiles - 1, tableHash.partitions.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java
index 7ec026f..7360a63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithOperationAttributes.java
@@ -110,32 +110,32 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
 
   @Test
   public void testMROnTable() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
 
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.MAPPER_CONF_KEY
             + "=org.apache.hadoop.hbase.mapreduce.TsvImporterCustomTestMapperForOprAttr",
         "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest=>myvalue\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1, true);
     util.deleteTable(tableName);
   }
 
   @Test
   public void testMROnTableWithInvalidOperationAttr() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
 
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.MAPPER_CONF_KEY
             + "=org.apache.hadoop.hbase.mapreduce.TsvImporterCustomTestMapperForOprAttr",
         "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_ATTRIBUTES_KEY",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE1\u001bVALUE2\u001btest1=>myvalue\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1, false);
     util.deleteTable(tableName);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java
index 53bdf70..445620f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithTTLs.java
@@ -101,16 +101,16 @@ public class TestImportTSVWithTTLs implements Configurable {
 
   @Test
   public void testMROnTable() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
 
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.MAPPER_CONF_KEY
             + "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
         "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_TTL",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE1\u001bVALUE2\u001b1000000\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1);
     util.deleteTable(tableName);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
index 8d974ab..6f0696e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTSVWithVisibilityLabels.java
@@ -154,16 +154,16 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
 
   @Test
   public void testMROnTable() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
 
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.MAPPER_CONF_KEY
             + "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
         "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1);
     util.deleteTable(tableName);
   }
@@ -222,25 +222,25 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
 
   @Test
   public void testMROnTableWithBulkload() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
-    Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName), "hfiles");
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
+    Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName.getNameAsString()), "hfiles");
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + hfiles.toString(),
         "-D" + ImportTsv.COLUMNS_CONF_KEY
             + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE1\u001bVALUE2\u001bsecret&private\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1);
     util.deleteTable(tableName);
   }
 
   @Test
   public void testBulkOutputWithTsvImporterTextMapper() throws Exception {
-    String table = "test-" + UUID.randomUUID();
+    TableName table = TableName.valueOf("test-" + UUID.randomUUID());
     String FAMILY = "FAM";
-    Path bulkOutputPath = new Path(util.getDataTestDirOnTestFS(table),"hfiles");
+    Path bulkOutputPath = new Path(util.getDataTestDirOnTestFS(table.getNameAsString()),"hfiles");
     // Prepare the arguments required for the test.
     String[] args =
         new String[] {
@@ -249,7 +249,8 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
             "-D" + ImportTsv.COLUMNS_CONF_KEY
                 + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
             "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b",
-            "-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + bulkOutputPath.toString(), table
+            "-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + bulkOutputPath.toString(),
+            table.getNameAsString()
             };
     String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n";
     doMROnTableTest(util, FAMILY, data, args, 4);
@@ -258,17 +259,17 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
 
   @Test
   public void testMRWithOutputFormat() throws Exception {
-    String tableName = "test-" + UUID.randomUUID();
-    Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName), "hfiles");
+    TableName tableName = TableName.valueOf("test-" + UUID.randomUUID());
+    Path hfiles = new Path(util.getDataTestDirOnTestFS(tableName.getNameAsString()), "hfiles");
     // Prepare the arguments required for the test.
     String[] args = new String[] {
         "-D" + ImportTsv.MAPPER_CONF_KEY
             + "=org.apache.hadoop.hbase.mapreduce.TsvImporterMapper",
         "-D" + ImportTsv.BULK_OUTPUT_CONF_KEY + "=" + hfiles.toString(),
         "-D" + ImportTsv.COLUMNS_CONF_KEY + "=HBASE_ROW_KEY,FAM:A,FAM:B,HBASE_CELL_VISIBILITY",
-        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName };
+        "-D" + ImportTsv.SEPARATOR_CONF_KEY + "=\u001b", tableName.getNameAsString() };
     String data = "KEY\u001bVALUE4\u001bVALUE8\u001bsecret&private\n";
-    util.createTable(TableName.valueOf(tableName), FAMILY);
+    util.createTable(tableName, FAMILY);
     doMROnTableTest(util, FAMILY, data, args, 1);
     util.deleteTable(tableName);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
index 9f36587..ca19af4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportTsv.java
@@ -125,7 +125,7 @@ public class TestImportTsv implements Configurable {
     doMROnTableTest(null, 1);
     util.deleteTable(table);
   }
-  
+
   @Test
   public void testMROnTableWithTimestamp() throws Exception {
     util.createTable(TableName.valueOf(table), FAMILY);
@@ -147,7 +147,7 @@ public class TestImportTsv implements Configurable {
     doMROnTableTest(null, 3);
     util.deleteTable(table);
   }
-  
+
   @Test
   public void testBulkOutputWithoutAnExistingTable() throws Exception {
     // Prepare the arguments required for the test.
@@ -169,7 +169,7 @@ public class TestImportTsv implements Configurable {
     doMROnTableTest(null, 3);
     util.deleteTable(table);
   }
-  
+
   @Test
   public void testBulkOutputWithAnExistingTableNoStrictTrue() throws Exception {
     util.createTable(TableName.valueOf(table), FAMILY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
index 2ca0ca5..80cbb70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -177,8 +178,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
     // create HFiles for different column families
     LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
     Path bulk1 = buildBulkFiles(table, value);
-    try (Table t = connection.getTable(table)) {
-      lih.doBulkLoad(bulk1, (HTable)t);
+    try (Table t = connection.getTable(table);
+        RegionLocator locator = connection.getRegionLocator(table);
+        Admin admin = connection.getAdmin()) {
+        lih.doBulkLoad(bulk1, admin, t, locator);
     }
   }
 
@@ -223,7 +226,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
   @BeforeClass
   public static void setupCluster() throws Exception {
     util = new HBaseTestingUtility();
-    util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,"");
+    util.getConfiguration().set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
     util.startMiniCluster(1);
   }
 
@@ -298,8 +301,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
       try {
         // create HFiles for different column families
         Path dir = buildBulkFiles(table, 1);
-        try (Table t = connection.getTable(table)) {
-          lih.doBulkLoad(dir, (HTable)t);
+        try (Table t = connection.getTable(table);
+            RegionLocator locator = connection.getRegionLocator(table);
+            Admin admin = connection.getAdmin()) {
+          lih.doBulkLoad(dir, admin, t, locator);
         }
       } finally {
         util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@@ -364,9 +369,11 @@ public class TestLoadIncrementalHFilesSplitRecovery {
       };
 
       // create HFiles for different column families
-      try (Table t = connection.getTable(table)) {
+      try (Table t = connection.getTable(table);
+          RegionLocator locator = connection.getRegionLocator(table);
+          Admin admin = connection.getAdmin()) {
         Path bulk = buildBulkFiles(table, 2);
-        lih2.doBulkLoad(bulk, (HTable)t);
+        lih2.doBulkLoad(bulk, admin, t, locator);
       }
 
       // check that data was loaded
@@ -408,8 +415,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
 
       // create HFiles for different column families
       Path bulk = buildBulkFiles(table, 2);
-      try (Table t = connection.getTable(table)) {
-        lih.doBulkLoad(bulk, (HTable)t);
+      try (Table t = connection.getTable(table);
+          RegionLocator locator = connection.getRegionLocator(table);
+          Admin admin = connection.getAdmin()) {
+        lih.doBulkLoad(bulk, admin, t, locator);
       }
       assertExpectedTable(connection, table, ROWCOUNT, 2);
       assertEquals(20, countedLqis.get());
@@ -446,8 +455,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
 
       // create HFiles for different column families
       Path dir = buildBulkFiles(table,1);
-      try (Table t = connection.getTable(table)) {
-        lih.doBulkLoad(dir, (HTable)t);
+      try (Table t = connection.getTable(table);
+          RegionLocator locator = connection.getRegionLocator(table);
+          Admin admin = connection.getAdmin()) {
+        lih.doBulkLoad(dir, admin, t, locator);
       }
     }
 
@@ -472,7 +483,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
 
       protected List<LoadQueueItem> groupOrSplit(
           Multimap<ByteBuffer, LoadQueueItem> regionGroups,
-          final LoadQueueItem item, final HTable htable,
+          final LoadQueueItem item, final Table htable,
           final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
         List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
         if (lqis != null) {
@@ -483,8 +494,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
     };
 
     // do bulkload when there is no region hole in hbase:meta.
-    try {
-      loader.doBulkLoad(dir, (HTable)table);
+    try (Table t = connection.getTable(tableName);
+        RegionLocator locator = connection.getRegionLocator(tableName);
+        Admin admin = connection.getAdmin()) {
+      loader.doBulkLoad(dir, admin, t, locator);
     } catch (Exception e) {
       LOG.error("exeception=", e);
     }
@@ -502,10 +515,12 @@ public class TestLoadIncrementalHFilesSplitRecovery {
       }
     }
 
-    try {
-      loader.doBulkLoad(dir, (HTable)table);
+    try (Table t = connection.getTable(tableName);
+        RegionLocator locator = connection.getRegionLocator(tableName);
+        Admin admin = connection.getAdmin()) {
+      loader.doBulkLoad(dir, admin, t, locator);
     } catch (Exception e) {
-      LOG.error("exeception=", e);
+      LOG.error("exception=", e);
       assertTrue("IOException expected", e instanceof IOException);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java
index 1bfd14a..98b7e42 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormat.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.mapreduce;
 
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
@@ -40,9 +41,9 @@ public class TestMultiTableInputFormat extends MultiTableInputFormatTestBase {
   @BeforeClass
   public static void setupLogging() {
     TEST_UTIL.enableDebug(MultiTableInputFormat.class);
-  }
+      }
 
-  @Override
+    @Override
   protected void initJob(List<Scan> scans, Job job) throws IOException {
     TableMapReduceUtil.initTableMapperJob(scans, ScanMapper.class,
         ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
index 6180632..8dce0ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultithreadedTableMapper.java
@@ -70,7 +70,7 @@ public class TestMultithreadedTableMapper {
   @BeforeClass
   public static void beforeClass() throws Exception {
     UTIL.startMiniCluster();
-    HTable table =
+    Table table =
         UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { INPUT_FAMILY,
             OUTPUT_FAMILY });
     UTIL.loadTable(table, INPUT_FAMILY, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
index 80af874..2024429 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRowCounter.java
@@ -155,7 +155,7 @@ public class TestRowCounter {
     long ts;
 
     // clean up content of TABLE_NAME
-    HTable table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
+    Table table = TEST_UTIL.deleteTableData(TableName.valueOf(TABLE_NAME));
     ts = System.currentTimeMillis();
     put1.add(family, col1, ts, Bytes.toBytes("val1"));
     table.put(put1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java
index a86270f..94348b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSyncTable.java
@@ -80,8 +80,8 @@ public class TestSyncTable {
   
   @Test
   public void testSyncTable() throws Exception {
-    String sourceTableName = "testSourceTable";
-    String targetTableName = "testTargetTable";
+    TableName sourceTableName = TableName.valueOf("testSourceTable");
+    TableName targetTableName = TableName.valueOf("testTargetTable");
     Path testDir = TEST_UTIL.getDataTestDirOnTestFS("testSyncTable");
     
     writeTestData(sourceTableName, targetTableName);
@@ -101,10 +101,10 @@ public class TestSyncTable {
     TEST_UTIL.cleanupDataTestDirOnTestFS();
   }
 
-  private void assertEqualTables(int expectedRows, String sourceTableName, String targetTableName) 
-      throws Exception {
-    Table sourceTable = TEST_UTIL.getConnection().getTable(TableName.valueOf(sourceTableName));
-    Table targetTable = TEST_UTIL.getConnection().getTable(TableName.valueOf(targetTableName));
+  private void assertEqualTables(int expectedRows, TableName sourceTableName,
+      TableName targetTableName) throws Exception {
+    Table sourceTable = TEST_UTIL.getConnection().getTable(sourceTableName);
+    Table targetTable = TEST_UTIL.getConnection().getTable(targetTableName);
     
     ResultScanner sourceScanner = sourceTable.getScanner(new Scan());
     ResultScanner targetScanner = targetTable.getScanner(new Scan());
@@ -177,13 +177,13 @@ public class TestSyncTable {
     targetTable.close();
   }
 
-  private Counters syncTables(String sourceTableName, String targetTableName,
+  private Counters syncTables(TableName sourceTableName, TableName targetTableName,
       Path testDir) throws Exception {
     SyncTable syncTable = new SyncTable(TEST_UTIL.getConfiguration());
     int code = syncTable.run(new String[] { 
         testDir.toString(),
-        sourceTableName,
-        targetTableName
+        sourceTableName.getNameAsString(),
+        targetTableName.getNameAsString()
         });
     assertEquals("sync table job failed", 0, code);
     
@@ -191,7 +191,7 @@ public class TestSyncTable {
     return syncTable.counters;
   }
 
-  private void hashSourceTable(String sourceTableName, Path testDir)
+  private void hashSourceTable(TableName sourceTableName, Path testDir)
       throws Exception, IOException {
     int numHashFiles = 3;
     long batchSize = 100;  // should be 2 batches per region
@@ -201,14 +201,14 @@ public class TestSyncTable {
         "--batchsize=" + batchSize,
         "--numhashfiles=" + numHashFiles,
         "--scanbatch=" + scanBatch,
-        sourceTableName,
+        sourceTableName.getNameAsString(),
         testDir.toString()});
     assertEquals("hash table job failed", 0, code);
     
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     
     HashTable.TableHash tableHash = HashTable.TableHash.read(fs.getConf(), testDir);
-    assertEquals(sourceTableName, tableHash.tableName);
+    assertEquals(sourceTableName.getNameAsString(), tableHash.tableName);
     assertEquals(batchSize, tableHash.batchSize);
     assertEquals(numHashFiles, tableHash.numHashFiles);
     assertEquals(numHashFiles - 1, tableHash.partitions.size());
@@ -216,7 +216,7 @@ public class TestSyncTable {
     LOG.info("Hash table completed");
   }
 
-  private void writeTestData(String sourceTableName, String targetTableName)
+  private void writeTestData(TableName sourceTableName, TableName targetTableName)
       throws Exception {
     final byte[] family = Bytes.toBytes("family");
     final byte[] column1 = Bytes.toBytes("c1");
@@ -229,10 +229,10 @@ public class TestSyncTable {
     int sourceRegions = 10;
     int targetRegions = 6;
     
-    HTable sourceTable = TEST_UTIL.createTable(TableName.valueOf(sourceTableName),
+    Table sourceTable = TEST_UTIL.createTable(sourceTableName,
         family, generateSplits(numRows, sourceRegions));
 
-    HTable targetTable = TEST_UTIL.createTable(TableName.valueOf(targetTableName),
+    Table targetTable = TEST_UTIL.createTable(targetTableName,
         family, generateSplits(numRows, targetRegions));
 
     long timestamp = 1430764183454L;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
index 5cca54b..33d1a66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatScanBase.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.NullWritable;
@@ -63,12 +64,12 @@ public abstract class TestTableInputFormatScanBase {
   private static final Log LOG = LogFactory.getLog(TestTableInputFormatScanBase.class);
   static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  static final byte[] TABLE_NAME = Bytes.toBytes("scantest");
+  static final TableName TABLE_NAME = TableName.valueOf("scantest");
   static final byte[] INPUT_FAMILY = Bytes.toBytes("contents");
   static final String KEY_STARTROW = "startRow";
   static final String KEY_LASTROW = "stpRow";
 
-  private static HTable table = null;
+  private static Table table = null;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -82,7 +83,7 @@ public abstract class TestTableInputFormatScanBase {
     // start mini hbase cluster
     TEST_UTIL.startMiniCluster(3);
     // create and fill table
-    table = TEST_UTIL.createMultiRegionTable(TableName.valueOf(TABLE_NAME), INPUT_FAMILY);
+    table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, INPUT_FAMILY);
     TEST_UTIL.loadTable(table, INPUT_FAMILY, false);
     // start MR cluster
     TEST_UTIL.startMiniMapReduceCluster();
@@ -182,7 +183,7 @@ public abstract class TestTableInputFormatScanBase {
     String jobName = "ScanFromConfig" + (start != null ? start.toUpperCase() : "Empty") +
       "To" + (stop != null ? stop.toUpperCase() : "Empty");
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
-    c.set(TableInputFormat.INPUT_TABLE, Bytes.toString(TABLE_NAME));
+    c.set(TableInputFormat.INPUT_TABLE, TABLE_NAME.getNameAsString());
     c.set(TableInputFormat.SCAN_COLUMN_FAMILY, Bytes.toString(INPUT_FAMILY));
     c.set(KEY_STARTROW, start != null ? start : "");
     c.set(KEY_LASTROW, last != null ? last : "");
@@ -233,7 +234,7 @@ public abstract class TestTableInputFormatScanBase {
     LOG.info("scan before: " + scan);
     Job job = new Job(c, jobName);
     TableMapReduceUtil.initTableMapperJob(
-      Bytes.toString(TABLE_NAME), scan, ScanMapper.class,
+      TABLE_NAME, scan, ScanMapper.class,
       ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
     job.setReducerClass(ScanReducer.class);
     job.setNumReduceTasks(1); // one to get final "first" and "last" key
@@ -264,11 +265,11 @@ public abstract class TestTableInputFormatScanBase {
     c.set(KEY_STARTROW, "");
     c.set(KEY_LASTROW, "");
     Job job = new Job(c, jobName);
-    TableMapReduceUtil.initTableMapperJob(Bytes.toString(TABLE_NAME), scan, ScanMapper.class,
+    TableMapReduceUtil.initTableMapperJob(TABLE_NAME.getNameAsString(), scan, ScanMapper.class,
             ImmutableBytesWritable.class, ImmutableBytesWritable.class, job);
     TableInputFormat tif = new TableInputFormat();
     tif.setConf(job.getConfiguration());
-    Assert.assertEquals(new String(TABLE_NAME), new String(table.getTableName()));
+    Assert.assertEquals(TABLE_NAME, table.getName());
     List<InputSplit> splits = tif.getSplits(job);
     Assert.assertEquals(expectedNumOfSplits, splits.size());
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java
index 2972222..572d9ae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableMapReduceBase.java
@@ -76,7 +76,7 @@ public abstract class TestTableMapReduceBase {
   @BeforeClass
   public static void beforeClass() throws Exception {
     UTIL.startMiniCluster();
-    HTable table =
+    Table table =
         UTIL.createMultiRegionTable(MULTI_REGION_TABLE_NAME, new byte[][] { INPUT_FAMILY,
             OUTPUT_FAMILY });
     UTIL.loadTable(table, INPUT_FAMILY, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
index 323957e..6b68bfe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentManagerOnCluster.java
@@ -176,9 +176,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testAssignRegion() throws Exception {
-    String table = "testAssignRegion";
+    TableName table = TableName.valueOf("testAssignRegion");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -203,7 +203,7 @@ public class TestAssignmentManagerOnCluster {
       RegionState newState = regionStates.getRegionState(hri);
       assertTrue(newState.isOpened());
     } finally {
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -212,7 +212,7 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=120000)
   public void testAssignRegionOnRestartedServer() throws Exception {
-    String table = "testAssignRegionOnRestartedServer";
+    TableName table = TableName.valueOf("testAssignRegionOnRestartedServer");
     TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 20);
     TEST_UTIL.getMiniHBaseCluster().stopMaster(0);
     //restart the master so that conf take into affect
@@ -221,7 +221,7 @@ public class TestAssignmentManagerOnCluster {
     ServerName deadServer = null;
     HMaster master = null;
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -260,7 +260,7 @@ public class TestAssignmentManagerOnCluster {
         master.serverManager.expireServer(deadServer);
       }
 
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
 
       // reset the value for other tests
       TEST_UTIL.getMiniHBaseCluster().getConf().setInt("hbase.assignment.maximum.attempts", 3);
@@ -431,9 +431,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testAssignWhileClosing() throws Exception {
-    String table = "testAssignWhileClosing";
+    TableName table = TableName.valueOf("testAssignWhileClosing");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -471,7 +471,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
     } finally {
       MyRegionObserver.preCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -480,9 +480,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testCloseFailed() throws Exception {
-    String table = "testCloseFailed";
+    TableName table = TableName.valueOf("testCloseFailed");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -517,7 +517,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
     } finally {
       MyRegionObserver.preCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -526,9 +526,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testOpenFailed() throws Exception {
-    String table = "testOpenFailed";
+    TableName table = TableName.valueOf("testOpenFailed");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -558,7 +558,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
     } finally {
       MyLoadBalancer.controledRegion = null;
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -654,9 +654,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testCloseHang() throws Exception {
-    String table = "testCloseHang";
+    TableName table = TableName.valueOf("testCloseHang");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -690,7 +690,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
     } finally {
       MyRegionObserver.postCloseEnabled.set(false);
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -699,9 +699,9 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testOpenCloseRacing() throws Exception {
-    String table = "testOpenCloseRacing";
+    TableName table = TableName.valueOf("testOpenCloseRacing");
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -761,7 +761,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 6000);
     } finally {
       MyRegionObserver.postOpenEnabled.set(false);
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 
@@ -770,11 +770,11 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testAssignRacingWithSSH() throws Exception {
-    String table = "testAssignRacingWithSSH";
+    TableName table = TableName.valueOf("testAssignRacingWithSSH");
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     MyMaster master = null;
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -836,7 +836,7 @@ public class TestAssignmentManagerOnCluster {
       if (master != null) {
         master.enableSSH(true);
       }
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
       cluster.startRegionServer();
     }
   }
@@ -939,11 +939,11 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testAssignOfflinedRegionBySSH() throws Exception {
-    String table = "testAssignOfflinedRegionBySSH";
+    TableName table = TableName.valueOf("testAssignOfflinedRegionBySSH");
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     MyMaster master = null;
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -1008,7 +1008,7 @@ public class TestAssignmentManagerOnCluster {
       TEST_UTIL.assertRegionOnlyOnServer(hri, serverName, 200);
     } finally {
       MyRegionServer.abortedServer = null;
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
       cluster.startRegionServer();
     }
   }
@@ -1018,11 +1018,11 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test (timeout=60000)
   public void testAssignDisabledRegionBySSH() throws Exception {
-    String table = "testAssignDisabledRegionBySSH";
+    TableName table = TableName.valueOf("testAssignDisabledRegionBySSH");
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
-    MyMaster master = null;
+    MyMaster master;
     try {
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
@@ -1085,7 +1085,7 @@ public class TestAssignmentManagerOnCluster {
       assertTrue(regionStates.isRegionOffline(hri));
     } finally {
       MyRegionServer.abortedServer = null;
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
       cluster.startRegionServer();
     }
   }
@@ -1095,10 +1095,10 @@ public class TestAssignmentManagerOnCluster {
    */
   @Test(timeout = 60000)
   public void testReportRegionStateTransition() throws Exception {
-    String table = "testReportRegionStateTransition";
+    TableName table = TableName.valueOf("testReportRegionStateTransition");
     try {
       MyRegionServer.simulateRetry = true;
-      HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
+      HTableDescriptor desc = new HTableDescriptor(table);
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
       Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
@@ -1114,13 +1114,13 @@ public class TestAssignmentManagerOnCluster {
       // Assert the the region is actually open on the server
       TEST_UTIL.assertRegionOnServer(hri, serverName, 200);
       // Closing region should just work fine
-      admin.disableTable(TableName.valueOf(table));
+      admin.disableTable(table);
       assertTrue(regionStates.isRegionOffline(hri));
       List<HRegionInfo> regions = TEST_UTIL.getHBaseAdmin().getOnlineRegions(serverName);
       assertTrue(!regions.contains(hri));
     } finally {
       MyRegionServer.simulateRetry = false;
-      TEST_UTIL.deleteTable(Bytes.toBytes(table));
+      TEST_UTIL.deleteTable(table);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
index bc437fc..813eb49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -343,10 +343,11 @@ public class TestDistributedLogSplitting {
     master.balanceSwitch(false);
 
     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
-    HTable ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
+    Table ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
     NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
     NonceGenerator oldNg =
-        ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)ht.getConnection(), ng);
+        ConnectionUtils.injectNonceGeneratorForTesting(
+            (ClusterConnection)TEST_UTIL.getConnection(), ng);
 
     try {
       List<Increment> reqs = new ArrayList<Increment>();
@@ -380,7 +381,8 @@ public class TestDistributedLogSplitting {
         }
       }
     } finally {
-      ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection) ht.getConnection(), oldNg);
+      ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)
+              TEST_UTIL.getConnection(), oldNg);
       ht.close();
       zkw.close();
     }
@@ -711,7 +713,7 @@ public class TestDistributedLogSplitting {
 
     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
-    HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
 
     List<HRegionInfo> regions = null;
     HRegionServer hrs = null;
@@ -901,7 +903,7 @@ public class TestDistributedLogSplitting {
 
     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
-    HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+    Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
 
     Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
@@ -1399,7 +1401,7 @@ public class TestDistributedLogSplitting {
     LOG.info("testReadWriteSeqIdFiles");
     startCluster(2);
     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
-    HTable ht = installTable(zkw, "table", "family", 10);
+    Table ht = installTable(zkw, "table", "family", 10);
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
     Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf("table"));
     List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir);
@@ -1425,19 +1427,19 @@ public class TestDistributedLogSplitting {
     ht.close();
   } 
   
-  HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
+  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
     return installTable(zkw, tname, fname, nrs, 0);
   }
 
-  HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
+  Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
       int existingRegions) throws Exception {
     // Create a table with regions
     TableName table = TableName.valueOf(tname);
     byte [] family = Bytes.toBytes(fname);
     LOG.info("Creating table with " + nrs + " regions");
-    HTable ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
+    Table ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
     int numRegions = -1;
-    try (RegionLocator r = ht.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
       numRegions = r.getStartKeys().length;
     }
     assertEquals(nrs, numRegions);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java
index abb6520..d2bb11a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetLastFlushedSequenceId.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -72,9 +73,8 @@ public class TestGetLastFlushedSequenceId {
   public void test() throws IOException, InterruptedException {
     testUtil.getHBaseAdmin().createNamespace(
       NamespaceDescriptor.create(tableName.getNamespaceAsString()).build());
-    HTable table = testUtil.createTable(tableName, families);
+    Table table = testUtil.createTable(tableName, families);
     table.put(new Put(Bytes.toBytes("k")).add(family, Bytes.toBytes("q"), Bytes.toBytes("v")));
-    table.flushCommits();
     MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
     List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
     Region region = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index 8028756..27ee31b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -83,7 +84,7 @@ public class TestMaster {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     HMaster m = cluster.getMaster();
 
-    try (HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
+    try (Table ht = TEST_UTIL.createTable(TABLENAME, FAMILYNAME)) {
       assertTrue(m.assignmentManager.getTableStateManager().isTableState(TABLENAME,
         TableState.State.ENABLED));
       TEST_UTIL.loadTable(ht, FAMILYNAME, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
index 20e0e54..3a4075b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRestartAfterDisablingTable.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -67,9 +68,9 @@ public class TestMasterRestartAfterDisablingTable {
     TableName table = TableName.valueOf("tableRestart");
     byte[] family = Bytes.toBytes("family");
     log("Creating table with " + NUM_REGIONS_TO_CREATE + " regions");
-    HTable ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
+    Table ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
     int numRegions = -1;
-    try (RegionLocator r = ht.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
       numRegions = r.getStartKeys().length;
     }
     numRegions += 1; // catalogs

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
index a09c5f2..7cea0df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java
@@ -66,9 +66,9 @@ public class TestMasterTransitions {
     TEST_UTIL.startMiniCluster(2);
     // Create a table of three families.  This will assign a region.
     TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILIES);
-    HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME);
+    Table t = TEST_UTIL.getConnection().getTable(TABLENAME);
     int countOfRegions = -1;
-    try (RegionLocator r = t.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLENAME)) {
       countOfRegions = r.getStartKeys().length;
     }
     TEST_UTIL.waitUntilAllRegionsAssigned(TABLENAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
index 9cbf680..69c5f89 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -50,7 +51,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.FavoredNodesPlan;
@@ -546,11 +549,10 @@ public class TestRegionPlacement {
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, splitKeys);
 
-    HTable ht = (HTable) CONNECTION.getTable(tableName);
-    @SuppressWarnings("deprecation")
-    Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
-    assertEquals("Tried to create " + expectedRegions + " regions "
-        + "but only found " + regions.size(), expectedRegions, regions.size());
-    ht.close();
+    try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) {
+      List<HRegionLocation> regions = r.getAllRegionLocations();
+      assertEquals("Tried to create " + expectedRegions + " regions "
+          + "but only found " + regions.size(), expectedRegions, regions.size());
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
index 72a0e0c..3d12c12 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRollingRestart.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -78,9 +79,9 @@ public class  TestRollingRestart {
     TableName table = TableName.valueOf("tableRestart");
     byte [] family = Bytes.toBytes("family");
     log("Creating table with " + NUM_REGIONS_TO_CREATE + " regions");
-    HTable ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
+    Table ht = TEST_UTIL.createMultiRegionTable(table, family, NUM_REGIONS_TO_CREATE);
     int numRegions = -1;
-    try (RegionLocator r = ht.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
       numRegions = r.getStartKeys().length;
     }
     numRegions += 1; // catalogs

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
index a2f08ab..3c070e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestSnapshotFromMaster.java
@@ -288,12 +288,12 @@ public class TestSnapshotFromMaster {
     UTIL.deleteTable(TABLE_NAME);
     HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
     htd.setCompactionEnabled(false);
-    UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
+    UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
     // load the table (creates 4 hfiles)
     UTIL.loadTable(UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
     UTIL.flush(TABLE_NAME);
     // Put some more data into the table so for sure we get more storefiles.
-    UTIL.loadTable((HTable) UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
+    UTIL.loadTable(UTIL.getConnection().getTable(TABLE_NAME), TEST_FAM);
 
     // disable the table so we can take a snapshot
     admin.disableTable(TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 50ab46d..52bfba8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -446,7 +446,7 @@ public class TestNamespaceAuditor {
     // This call will pass.
     ADMIN.createTable(tableDescOne);
     Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
-    HTable htable = (HTable)connection.getTable(tableOne);
+    Table htable = connection.getTable(tableOne);
     UTIL.loadNumericRows(htable, Bytes.toBytes("info"), 1, 1000);
     ADMIN.flush(tableOne);
     stateInfo = getNamespaceState(nsp1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
index d19906b..6d2dd3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -66,7 +67,7 @@ public class TestQuotaThrottle {
   };
 
   private static ManualEnvironmentEdge envEdge;
-  private static HTable[] tables;
+  private static Table[] tables;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -81,7 +82,7 @@ public class TestQuotaThrottle {
     TEST_UTIL.waitTableAvailable(QuotaTableUtil.QUOTA_TABLE_NAME);
     QuotaCache.TEST_FORCE_REFRESH = true;
 
-    tables = new HTable[TABLE_NAMES.length];
+    tables = new Table[TABLE_NAMES.length];
     for (int i = 0; i < TABLE_NAMES.length; ++i) {
       tables[i] = TEST_UTIL.createTable(TABLE_NAMES[i], FAMILY);
     }
@@ -505,13 +506,13 @@ public class TestQuotaThrottle {
     assertEquals(30, doGets(30, tables[1]));
   }
 
-  private int doPuts(int maxOps, final HTable... tables) throws Exception {
+  private int doPuts(int maxOps, final Table... tables) throws Exception {
     int count = 0;
     try {
       while (count < maxOps) {
         Put put = new Put(Bytes.toBytes("row-" + count));
         put.add(FAMILY, QUALIFIER, Bytes.toBytes("data-" + count));
-        for (final HTable table: tables) {
+        for (final Table table: tables) {
           table.put(put);
         }
         count += tables.length;
@@ -527,12 +528,12 @@ public class TestQuotaThrottle {
     return count;
   }
 
-  private long doGets(int maxOps, final HTable... tables) throws Exception {
+  private long doGets(int maxOps, final Table... tables) throws Exception {
     int count = 0;
     try {
       while (count < maxOps) {
         Get get = new Get(Bytes.toBytes("row-" + count));
-        for (final HTable table: tables) {
+        for (final Table table: tables) {
           table.get(get);
         }
         count += tables.length;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
index be43950..cd87344 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -94,7 +95,7 @@ public class TestEndToEndSplitTransaction {
   public void testMasterOpsWhileSplitting() throws Exception {
     TableName tableName = TableName.valueOf("TestSplit");
     byte[] familyName = Bytes.toBytes("fam");
-    try (HTable ht = TEST_UTIL.createTable(tableName, familyName)) {
+    try (Table ht = TEST_UTIL.createTable(tableName, familyName)) {
       TEST_UTIL.loadTable(ht, familyName, false);
     }
     HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(0);
@@ -318,19 +319,23 @@ public class TestEndToEndSplitTransaction {
 
     /** verify region boundaries obtained from HTable.getStartEndKeys() */
     void verifyRegionsUsingHTable() throws IOException {
-      HTable table = null;
+      Table table = null;
       try {
         //HTable.getStartEndKeys()
-        table = (HTable) connection.getTable(tableName);
-        Pair<byte[][], byte[][]> keys = table.getRegionLocator().getStartEndKeys();
-        verifyStartEndKeys(keys);
-
-        //HTable.getRegionsInfo()
-        Set<HRegionInfo> regions = new TreeSet<HRegionInfo>();
-        for (HRegionLocation loc : table.getRegionLocator().getAllRegionLocations()) {
-          regions.add(loc.getRegionInfo());
+        table = connection.getTable(tableName);
+
+        try(RegionLocator rl = connection.getRegionLocator(tableName)) {
+          Pair<byte[][], byte[][]> keys = rl.getStartEndKeys();
+          verifyStartEndKeys(keys);
+
+          //HTable.getRegionsInfo()
+          Set<HRegionInfo> regions = new TreeSet<HRegionInfo>();
+          for (HRegionLocation loc : rl.getAllRegionLocations()) {
+            regions.add(loc.getRegionInfo());
+          }
+          verifyTableRegions(regions);
         }
-        verifyTableRegions(regions);
+
       } finally {
         IOUtils.closeQuietly(table);
       }


[5/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
index 953f641..294d7a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java
@@ -56,7 +56,7 @@ public class TestClientPushback {
   private static final Log LOG = LogFactory.getLog(TestClientPushback.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
-  private static final byte[] tableName = Bytes.toBytes("client-pushback");
+  private static final TableName tableName = TableName.valueOf("client-pushback");
   private static final byte[] family = Bytes.toBytes("f");
   private static final byte[] qualifier = Bytes.toBytes("q");
   private static long flushSizeBytes = 1024;
@@ -87,31 +87,28 @@ public class TestClientPushback {
   @Test(timeout=60000)
   public void testClientTracksServerPushback() throws Exception{
     Configuration conf = UTIL.getConfiguration();
-    TableName tablename = TableName.valueOf(tableName);
-    Connection conn = ConnectionFactory.createConnection(conf);
-    HTable table = (HTable) conn.getTable(tablename);
+    ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    HTable table = (HTable) conn.getTable(tableName);
 
     HRegionServer rs = UTIL.getHBaseCluster().getRegionServer(0);
-    Region region = rs.getOnlineRegions(tablename).get(0);
+    Region region = rs.getOnlineRegions(tableName).get(0);
 
-    LOG.debug("Writing some data to "+tablename);
+    LOG.debug("Writing some data to "+tableName);
     // write some data
     Put p = new Put(Bytes.toBytes("row"));
-    p.add(family, qualifier, Bytes.toBytes("value1"));
+    p.addColumn(family, qualifier, Bytes.toBytes("value1"));
     table.put(p);
-    table.flushCommits();
 
     // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
     int load = (int)((((HRegion)region).addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes);
-    LOG.debug("Done writing some data to "+tablename);
+    LOG.debug("Done writing some data to "+tableName);
 
     // get the stats for the region hosting our table
-    ClusterConnection connection = table.connection;
-    ClientBackoffPolicy backoffPolicy = connection.getBackoffPolicy();
+    ClientBackoffPolicy backoffPolicy = conn.getBackoffPolicy();
     assertTrue("Backoff policy is not correctly configured",
       backoffPolicy instanceof ExponentialClientBackoffPolicy);
     
-    ServerStatisticTracker stats = connection.getStatisticsTracker();
+    ServerStatisticTracker stats = conn.getStatisticsTracker();
     assertNotNull( "No stats configured for the client!", stats);
     // get the names so we can query the stats
     ServerName server = rs.getServerName();
@@ -135,8 +132,9 @@ public class TestClientPushback {
     ops.add(p);
     final CountDownLatch latch = new CountDownLatch(1);
     final AtomicLong endTime = new AtomicLong();
-    long startTime = EnvironmentEdgeManager.currentTime();    
-    table.mutator.ap.submit(tablename, ops, true, new Batch.Callback<Result>() {
+    long startTime = EnvironmentEdgeManager.currentTime();
+
+    table.mutator.ap.submit(tableName, ops, true, new Batch.Callback<Result>() {
       @Override
       public void update(byte[] region, byte[] row, Result result) {
         endTime.set(EnvironmentEdgeManager.currentTime());

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 027a348..d945cce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -115,7 +115,7 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 /**
- * Run tests that use the HBase clients; {@link HTable}.
+ * Run tests that use the HBase clients; {@link Table}.
  * Sets up the HBase mini cluster once at start and runs through all client tests.
  * Each creates a table named for the method and does its stuff against that.
  */
@@ -304,7 +304,7 @@ public class TestFromClientSide {
     TableName TABLE = TableName.valueOf("testGetConfiguration");
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     Configuration conf = TEST_UTIL.getConfiguration();
-    Table table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
+    Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
     assertSame(conf, table.getConfiguration());
   }
 
@@ -320,7 +320,7 @@ public class TestFromClientSide {
     byte [][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
-    HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
+    Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
     String value = "this is the value";
     String value2 = "this is some other value";
     String keyPrefix1 = UUID.randomUUID().toString();
@@ -329,7 +329,6 @@ public class TestFromClientSide {
     putRows(ht, 3, value, keyPrefix1);
     putRows(ht, 3, value, keyPrefix2);
     putRows(ht, 3, value, keyPrefix3);
-    ht.flushCommits();
     putRows(ht, 3, value2, keyPrefix1);
     putRows(ht, 3, value2, keyPrefix2);
     putRows(ht, 3, value2, keyPrefix3);
@@ -445,14 +444,14 @@ public class TestFromClientSide {
   public void testFilterAcrossMultipleRegions()
   throws IOException, InterruptedException {
     TableName name = TableName.valueOf("testFilterAcrossMutlipleRegions");
-    HTable t = TEST_UTIL.createTable(name, FAMILY);
+    Table t = TEST_UTIL.createTable(name, FAMILY);
     int rowCount = TEST_UTIL.loadTable(t, FAMILY, false);
     assertRowCount(t, rowCount);
     // Split the table.  Should split on a reasonable key; 'lqj'
-    Map<HRegionInfo, ServerName> regions  = splitTable(t);
+    List<HRegionLocation> regions  = splitTable(t);
     assertRowCount(t, rowCount);
     // Get end key of first region.
-    byte [] endKey = regions.keySet().iterator().next().getEndKey();
+    byte [] endKey = regions.get(0).getRegionInfo().getEndKey();
     // Count rows with a filter that stops us before passed 'endKey'.
     // Should be count of rows in first region.
     int endKeyCount = countRows(t, createScanWithRowFilter(endKey));
@@ -551,13 +550,13 @@ public class TestFromClientSide {
    * @return Map of regions to servers.
    * @throws IOException
    */
-  private Map<HRegionInfo, ServerName> splitTable(final HTable t)
+  private List<HRegionLocation> splitTable(final Table t)
   throws IOException, InterruptedException {
     // Split this table in two.
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     admin.split(t.getName());
     admin.close();
-    Map<HRegionInfo, ServerName> regions = waitOnSplit(t);
+    List<HRegionLocation> regions = waitOnSplit(t);
     assertTrue(regions.size() > 1);
     return regions;
   }
@@ -568,32 +567,35 @@ public class TestFromClientSide {
    * @param t
    * @return Map of table regions; caller needs to check table actually split.
    */
-  private Map<HRegionInfo, ServerName> waitOnSplit(final HTable t)
+  private List<HRegionLocation> waitOnSplit(final Table t)
   throws IOException {
-    Map<HRegionInfo, ServerName> regions = t.getRegionLocations();
-    int originalCount = regions.size();
-    for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) {
-      Thread.currentThread();
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
+      List<HRegionLocation> regions = locator.getAllRegionLocations();
+      int originalCount = regions.size();
+      for (int i = 0; i < TEST_UTIL.getConfiguration().getInt("hbase.test.retries", 30); i++) {
+        Thread.currentThread();
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          e.printStackTrace();
+        }
+        regions = locator.getAllRegionLocations();
+        if (regions.size() > originalCount)
+          break;
       }
-      regions = t.getRegionLocations();
-      if (regions.size() > originalCount) break;
+      return regions;
     }
-    return regions;
   }
 
   @Test
   public void testSuperSimple() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testSuperSimple");
+    TableName TABLE = TableName.valueOf("testSuperSimple");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
     ht.put(put);
     Scan scan = new Scan();
-    scan.addColumn(FAMILY, TABLE);
+    scan.addColumn(FAMILY, TABLE.toBytes());
     ResultScanner scanner = ht.getScanner(scan);
     Result result = scanner.next();
     assertTrue("Expected null result", result == null);
@@ -602,7 +604,7 @@ public class TestFromClientSide {
 
   @Test
   public void testMaxKeyValueSize() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
+    TableName TABLE = TableName.valueOf("testMaxKeyValueSize");
     Configuration conf = TEST_UTIL.getConfiguration();
     String oldMaxSize = conf.get(TableConfiguration.MAX_KEYVALUE_SIZE_KEY);
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
@@ -629,7 +631,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilters() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testFilters");
+    TableName TABLE = TableName.valueOf("testFilters");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] QUALIFIERS = {
@@ -665,7 +667,7 @@ public class TestFromClientSide {
 
   @Test
   public void testFilterWithLongCompartor() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testFilterWithLongCompartor");
+    TableName TABLE = TableName.valueOf("testFilterWithLongCompartor");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] values = new byte[10][];
@@ -696,7 +698,7 @@ public class TestFromClientSide {
 
   @Test
   public void testKeyOnlyFilter() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
+    TableName TABLE = TableName.valueOf("testKeyOnlyFilter");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     byte [][] ROWS = makeN(ROW, 10);
     byte [][] QUALIFIERS = {
@@ -733,7 +735,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSimpleMissing() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testSimpleMissing");
+    TableName TABLE = TableName.valueOf("testSimpleMissing");
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
     byte [][] ROWS = makeN(ROW, 4);
 
@@ -844,7 +846,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testSingleRowMultipleFamily() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testSingleRowMultipleFamily");
+    TableName TABLE = TableName.valueOf("testSingleRowMultipleFamily");
     byte [][] ROWS = makeN(ROW, 3);
     byte [][] FAMILIES = makeNAscii(FAMILY, 10);
     byte [][] QUALIFIERS = makeN(QUALIFIER, 10);
@@ -1145,7 +1147,7 @@ public class TestFromClientSide {
 
   @Test
   public void testNull() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testNull");
+    TableName TABLE = TableName.valueOf("testNull");
 
     // Null table name (should NOT work)
     try {
@@ -1253,7 +1255,7 @@ public class TestFromClientSide {
 
   @Test
   public void testVersions() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testVersions");
+    TableName TABLE = TableName.valueOf("testVersions");
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -1467,7 +1469,7 @@ public class TestFromClientSide {
 
   @Test
   public void testVersionLimits() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testVersionLimits");
+    TableName TABLE = TableName.valueOf("testVersionLimits");
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
     int [] LIMITS = {1,3,5};
     long [] STAMPS = makeStamps(10);
@@ -1662,7 +1664,7 @@ public class TestFromClientSide {
   @Test
   public void testDeleteFamilyVersion() throws Exception {
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersion");
+    TableName TABLE = TableName.valueOf("testDeleteFamilyVersion");
 
     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 1);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1700,7 +1702,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeleteFamilyVersionWithOtherDeletes() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testDeleteFamilyVersionWithOtherDeletes");
+    TableName TABLE = TableName.valueOf("testDeleteFamilyVersionWithOtherDeletes");
 
     byte [][] QUALIFIERS = makeNAscii(QUALIFIER, 5);
     byte [][] VALUES = makeN(VALUE, 5);
@@ -1815,7 +1817,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDeletes() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testDeletes");
+    TableName TABLE = TableName.valueOf("testDeletes");
 
     byte [][] ROWS = makeNAscii(ROW, 6);
     byte [][] FAMILIES = makeNAscii(FAMILY, 3);
@@ -2184,7 +2186,7 @@ public class TestFromClientSide {
     int numRows = 10;
     int numColsPerRow = 2000;
 
-    byte [] TABLE = Bytes.toBytes("testJiraTest867");
+    TableName TABLE = TableName.valueOf("testJiraTest867");
 
     byte [][] ROWS = makeN(ROW, numRows);
     byte [][] QUALIFIERS = makeN(QUALIFIER, numColsPerRow);
@@ -2268,7 +2270,7 @@ public class TestFromClientSide {
   @Test
   public void testJiraTest861() throws Exception {
 
-    byte [] TABLE = Bytes.toBytes("testJiraTest861");
+    TableName TABLE = TableName.valueOf("testJiraTest861");
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2332,7 +2334,7 @@ public class TestFromClientSide {
   @Test
   public void testJiraTest33() throws Exception {
 
-    byte [] TABLE = Bytes.toBytes("testJiraTest33");
+    TableName TABLE = TableName.valueOf("testJiraTest33");
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2381,7 +2383,7 @@ public class TestFromClientSide {
   @Test
   public void testJiraTest1014() throws Exception {
 
-    byte [] TABLE = Bytes.toBytes("testJiraTest1014");
+    TableName TABLE = TableName.valueOf("testJiraTest1014");
 
     Table ht = TEST_UTIL.createTable(TABLE, FAMILY, 10);
 
@@ -2406,7 +2408,7 @@ public class TestFromClientSide {
   @Test
   public void testJiraTest1182() throws Exception {
 
-    byte [] TABLE = Bytes.toBytes("testJiraTest1182");
+    TableName TABLE = TableName.valueOf("testJiraTest1182");
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -2449,7 +2451,7 @@ public class TestFromClientSide {
    */
   @Test
   public void testJiraTest52() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testJiraTest52");
+    TableName TABLE = TableName.valueOf("testJiraTest52");
     byte [][] VALUES = makeNAscii(VALUE, 7);
     long [] STAMPS = makeStamps(7);
 
@@ -3279,7 +3281,7 @@ public class TestFromClientSide {
 
   @Test
   public void testDuplicateVersions() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testDuplicateVersions");
+    TableName TABLE = TableName.valueOf("testDuplicateVersions");
 
     long [] STAMPS = makeStamps(20);
     byte [][] VALUES = makeNAscii(VALUE, 20);
@@ -3494,7 +3496,7 @@ public class TestFromClientSide {
   @Test
   public void testUpdates() throws Exception {
 
-    byte [] TABLE = Bytes.toBytes("testUpdates");
+    TableName TABLE = TableName.valueOf("testUpdates");
     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
 
     // Write a column with values at timestamp 1, 2 and 3
@@ -3606,7 +3608,7 @@ public class TestFromClientSide {
   public void testMajorCompactionBetweenTwoUpdates() throws Exception {
 
     String tableName = "testMajorCompactionBetweenTwoUpdates";
-    byte [] TABLE = Bytes.toBytes(tableName);
+    TableName TABLE = TableName.valueOf(tableName);
     Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
 
@@ -3731,8 +3733,8 @@ public class TestFromClientSide {
     final byte [] row1 = Bytes.toBytes("row1");
     final byte [] row2 = Bytes.toBytes("row2");
     final byte [] value = Bytes.toBytes("abcd");
-    Table table = TEST_UTIL.createTable(Bytes.toBytes("testPut"),
-      new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testPut"),
+        new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY });
     Put put = new Put(row1);
     put.add(CONTENTS_FAMILY, null, value);
     table.put(put);
@@ -3768,7 +3770,7 @@ public class TestFromClientSide {
   public void testPutNoCF() throws IOException {
     final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
     final byte[] VAL = Bytes.toBytes(100);
-    Table table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), FAMILY);
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testPutNoCF"), FAMILY);
 
     boolean caughtNSCFE = false;
 
@@ -3789,7 +3791,7 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final int NB_BATCH_ROWS = 10;
     final byte[] value = Bytes.toBytes("abcd");
-    Table table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPut"),
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testRowsPut"),
       new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS; i++) {
@@ -3816,40 +3818,43 @@ public class TestFromClientSide {
     final byte [] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte [] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
-    HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
-      new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
-    table.setAutoFlush(false);
-    ArrayList<Put> rowsUpdate = new ArrayList<Put>();
-    for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
-      byte[] row = Bytes.toBytes("row" + i);
-      Put put = new Put(row);
-      put.setDurability(Durability.SKIP_WAL);
-      put.add(CONTENTS_FAMILY, null, value);
-      rowsUpdate.add(put);
-    }
-    table.put(rowsUpdate);
-
-    Scan scan = new Scan();
-    scan.addFamily(CONTENTS_FAMILY);
-    ResultScanner scanner = table.getScanner(scan);
-    int nbRows = 0;
-    for (@SuppressWarnings("unused")
-    Result row : scanner)
-      nbRows++;
-    assertEquals(0, nbRows);
-    scanner.close();
-
-    table.flushCommits();
+    Table t = TEST_UTIL.createTable(TableName.valueOf("testRowsPutBufferedOneFlush"),
+        new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY });
+
+    // Only do this test if it is a HTable
+    if(t instanceof HTableInterface) {
+      HTable table = (HTable) t;
+      table.setAutoFlush(false);
+      ArrayList<Put> rowsUpdate = new ArrayList<Put>();
+      for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
+        byte[] row = Bytes.toBytes("row" + i);
+        Put put = new Put(row);
+        put.setDurability(Durability.SKIP_WAL);
+        put.add(CONTENTS_FAMILY, null, value);
+        rowsUpdate.add(put);
+      }
+      table.put(rowsUpdate);
 
-    scan = new Scan();
-    scan.addFamily(CONTENTS_FAMILY);
-    scanner = table.getScanner(scan);
-    nbRows = 0;
-    for (@SuppressWarnings("unused")
-    Result row : scanner)
-      nbRows++;
-    assertEquals(NB_BATCH_ROWS * 10, nbRows);
-    table.close();
+      Scan scan = new Scan();
+      scan.addFamily(CONTENTS_FAMILY);
+      ResultScanner scanner = table.getScanner(scan);
+      int nbRows = 0;
+      for (@SuppressWarnings("unused") Result row : scanner)
+        nbRows++;
+      assertEquals(0, nbRows);
+      scanner.close();
+
+      table.flushCommits();
+
+      scan = new Scan();
+      scan.addFamily(CONTENTS_FAMILY);
+      scanner = table.getScanner(scan);
+      nbRows = 0;
+      for (@SuppressWarnings("unused") Result row : scanner)
+        nbRows++;
+      assertEquals(NB_BATCH_ROWS * 10, nbRows);
+      table.close();
+    }
   }
 
   @Test
@@ -3858,8 +3863,8 @@ public class TestFromClientSide {
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
     final byte[] value = Bytes.toBytes("abcd");
     final int NB_BATCH_ROWS = 10;
-    HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
-      new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testRowsPutBufferedManyManyFlushes"),
+        new byte[][] { CONTENTS_FAMILY, SMALL_FAMILY });
     table.setWriteBufferSize(10);
     ArrayList<Put> rowsUpdate = new ArrayList<Put>();
     for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
@@ -3920,7 +3925,7 @@ public class TestFromClientSide {
     final byte [] FAM1 = Bytes.toBytes("fam1");
     final byte [] FAM2 = Bytes.toBytes("fam2");
     // Open table
-    Table table = TEST_UTIL.createTable(Bytes.toBytes("testHBase737"),
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testHBase737"),
       new byte [][] {FAM1, FAM2});
     // Insert some values
     Put put = new Put(ROW);
@@ -4034,19 +4039,6 @@ public class TestFromClientSide {
   }
 
   /**
-   * creates an HTable for tableName using an unmanaged HConnection.
-   *
-   * @param tableName - table to create
-   * @return the created HTable object
-   * @throws IOException
-   */
-  HTable createUnmangedHConnectionHTable(final TableName tableName) throws IOException {
-    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
-    Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
-    return (HTable)conn.getTable(tableName);
-  }
-
-  /**
    * simple test that just executes parts of the client
    * API that accept a pre-created HConnection instance
    *
@@ -4055,8 +4047,10 @@ public class TestFromClientSide {
   @Test
   public void testUnmanagedHConnection() throws IOException {
     final TableName tableName = TableName.valueOf("testUnmanagedHConnection");
-    HTable t = createUnmangedHConnectionHTable(tableName);
-    HBaseAdmin ha = new HBaseAdmin(t.getConnection());
+    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
+    Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+    Table t = conn.getTable(tableName);
+    HBaseAdmin ha = new HBaseAdmin(conn);
     assertTrue(ha.tableExists(tableName));
     assertTrue(t.get(new Get(ROW)).isEmpty());
     ha.close();
@@ -4071,8 +4065,9 @@ public class TestFromClientSide {
   @Test
   public void testUnmanagedHConnectionReconnect() throws Exception {
     final TableName tableName = TableName.valueOf("testUnmanagedHConnectionReconnect");
-    HTable t = createUnmangedHConnectionHTable(tableName);
-    Connection conn = t.getConnection();
+    TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY);
+    Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
+    Table t = conn.getTable(tableName);
     try (HBaseAdmin ha = new HBaseAdmin(conn)) {
       assertTrue(ha.tableExists(tableName));
       assertTrue(t.get(new Get(ROW)).isEmpty());
@@ -4174,7 +4169,7 @@ public class TestFromClientSide {
 
   @Test
   public void testGetClosestRowBefore() throws IOException, InterruptedException {
-    final TableName tableAname = TableName.valueOf("testGetClosestRowBefore");
+    final TableName tableName = TableName.valueOf("testGetClosestRowBefore");
     final byte[] firstRow = Bytes.toBytes("row111");
     final byte[] secondRow = Bytes.toBytes("row222");
     final byte[] thirdRow = Bytes.toBytes("row333");
@@ -4184,87 +4179,92 @@ public class TestFromClientSide {
     final byte[] beforeThirdRow = Bytes.toBytes("row33");
     final byte[] beforeForthRow = Bytes.toBytes("row44");
 
-    HTable table =
-        TEST_UTIL.createTable(tableAname,
+    try (Table t =
+        TEST_UTIL.createTable(tableName,
           new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024);
-    // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
-    // in Store.rowAtOrBeforeFromStoreFile
-    String regionName = table.getRegionLocations().firstKey().getEncodedName();
-    Region region =
-        TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName);
-    Put put1 = new Put(firstRow);
-    Put put2 = new Put(secondRow);
-    Put put3 = new Put(thirdRow);
-    Put put4 = new Put(forthRow);
-    byte[] one = new byte[] { 1 };
-    byte[] two = new byte[] { 2 };
-    byte[] three = new byte[] { 3 };
-    byte[] four = new byte[] { 4 };
-
-    put1.add(HConstants.CATALOG_FAMILY, null, one);
-    put2.add(HConstants.CATALOG_FAMILY, null, two);
-    put3.add(HConstants.CATALOG_FAMILY, null, three);
-    put4.add(HConstants.CATALOG_FAMILY, null, four);
-    table.put(put1);
-    table.put(put2);
-    table.put(put3);
-    table.put(put4);
-    region.flush(true);
-    Result result = null;
-
-    // Test before first that null is returned
-    result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result == null);
-
-    // Test at first that first is returned
-    result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), firstRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
-
-    // Test in between first and second that first is returned
-    result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), firstRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
-
-    // Test at second make sure second is returned
-    result = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), secondRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
-
-    // Test in second and third, make sure second is returned
-    result = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), secondRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
-
-    // Test at third make sure third is returned
-    result = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), thirdRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
-
-    // Test in third and forth, make sure third is returned
-    result = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), thirdRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
-
-    // Test at forth make sure forth is returned
-    result = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), forthRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
-
-    // Test after forth make sure forth is returned
-    result = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
-    assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
-    assertTrue(Bytes.equals(result.getRow(), forthRow));
-    assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
-
-    table.close();
+        RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      if (t instanceof HTableInterface) {
+        HTableInterface table = (HTableInterface) t;
+
+        // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow
+        // in Store.rowAtOrBeforeFromStoreFile
+        String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+        Region region =
+            TEST_UTIL.getRSForFirstRegionInTable(tableName).getFromOnlineRegions(regionName);
+        Put put1 = new Put(firstRow);
+        Put put2 = new Put(secondRow);
+        Put put3 = new Put(thirdRow);
+        Put put4 = new Put(forthRow);
+        byte[] one = new byte[] { 1 };
+        byte[] two = new byte[] { 2 };
+        byte[] three = new byte[] { 3 };
+        byte[] four = new byte[] { 4 };
+
+        put1.add(HConstants.CATALOG_FAMILY, null, one);
+        put2.add(HConstants.CATALOG_FAMILY, null, two);
+        put3.add(HConstants.CATALOG_FAMILY, null, three);
+        put4.add(HConstants.CATALOG_FAMILY, null, four);
+        table.put(put1);
+        table.put(put2);
+        table.put(put3);
+        table.put(put4);
+        region.flush(true);
+
+        Result result;
+
+        // Test before first that null is returned
+        result = table.getRowOrBefore(beforeFirstRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result == null);
+
+        // Test at first that first is returned
+        result = table.getRowOrBefore(firstRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), firstRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
+
+        // Test in between first and second that first is returned
+        result = table.getRowOrBefore(beforeSecondRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), firstRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), one));
+
+        // Test at second make sure second is returned
+        result = table.getRowOrBefore(secondRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), secondRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
+
+        // Test in second and third, make sure second is returned
+        result = table.getRowOrBefore(beforeThirdRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), secondRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), two));
+
+        // Test at third make sure third is returned
+        result = table.getRowOrBefore(thirdRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), thirdRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
+
+        // Test in third and forth, make sure third is returned
+        result = table.getRowOrBefore(beforeForthRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), thirdRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), three));
+
+        // Test at forth make sure forth is returned
+        result = table.getRowOrBefore(forthRow, HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), forthRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
+
+        // Test after forth make sure forth is returned
+        result = table.getRowOrBefore(Bytes.add(forthRow, one), HConstants.CATALOG_FAMILY);
+        assertTrue(result.containsColumn(HConstants.CATALOG_FAMILY, null));
+        assertTrue(Bytes.equals(result.getRow(), forthRow));
+        assertTrue(Bytes.equals(result.getValue(HConstants.CATALOG_FAMILY, null), four));
+      }
+    }
   }
 
   /**
@@ -4381,8 +4381,8 @@ public class TestFromClientSide {
     a.add(FAMILY, QUALIFIERS[1], v1);
     a.add(FAMILY, QUALIFIERS[2], v2);
     Result r = t.append(a);
-    assertEquals(0, Bytes.compareTo(Bytes.add(v1,v2), r.getValue(FAMILY, QUALIFIERS[0])));
-    assertEquals(0, Bytes.compareTo(Bytes.add(v2,v1), r.getValue(FAMILY, QUALIFIERS[1])));
+    assertEquals(0, Bytes.compareTo(Bytes.add(v1, v2), r.getValue(FAMILY, QUALIFIERS[0])));
+    assertEquals(0, Bytes.compareTo(Bytes.add(v2, v1), r.getValue(FAMILY, QUALIFIERS[1])));
     // QUALIFIERS[2] previously not exist, verify both value and timestamp are correct
     assertEquals(0, Bytes.compareTo(v2, r.getValue(FAMILY, QUALIFIERS[2])));
     assertEquals(r.getColumnLatestCell(FAMILY, QUALIFIERS[0]).getTimestamp(),
@@ -4535,8 +4535,8 @@ public class TestFromClientSide {
   @Test
   public void testIncrementOnSameColumn() throws Exception {
     LOG.info("Starting testIncrementOnSameColumn");
-    final byte[] TABLENAME = Bytes.toBytes("testIncrementOnSameColumn");
-    HTable ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
+    final TableName TABLENAME = TableName.valueOf("testIncrementOnSameColumn");
+    Table ht = TEST_UTIL.createTable(TABLENAME, FAMILY);
 
     byte[][] QUALIFIERS =
         new byte[][] { Bytes.toBytes("A"), Bytes.toBytes("B"), Bytes.toBytes("C") };
@@ -4657,7 +4657,7 @@ public class TestFromClientSide {
     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "round-robin");
     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
 
-    Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, conf, Integer.MAX_VALUE);
+    Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY }, Integer.MAX_VALUE);
 
     final long ts = EnvironmentEdgeManager.currentTime();
     Get get = new Get(ROW);
@@ -4693,8 +4693,7 @@ public class TestFromClientSide {
     conf.set(HConstants.HBASE_CLIENT_IPC_POOL_TYPE, "thread-local");
     conf.setInt(HConstants.HBASE_CLIENT_IPC_POOL_SIZE, poolSize);
 
-    final Table table = TEST_UTIL.createTable(tableName,
-        new byte[][] { FAMILY }, conf, 3);
+    final Table table = TEST_UTIL.createTable(tableName, new byte[][] { FAMILY },  3);
 
     final long ts = EnvironmentEdgeManager.currentTime();
     final Get get = new Get(ROW);
@@ -4955,9 +4954,9 @@ public class TestFromClientSide {
 
     // Set up test table:
     // Create table:
-    HTable ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILY);
+    Table ht = TEST_UTIL.createMultiRegionTable(TABLENAME, FAMILY);
     int numOfRegions = -1;
-    try (RegionLocator r = ht.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLENAME)) {
       numOfRegions = r.getStartKeys().length;
     }
     // Create 3 rows in the table, with rowkeys starting with "zzz*" so that
@@ -5091,98 +5090,102 @@ public class TestFromClientSide {
   public void testCacheOnWriteEvictOnClose() throws Exception {
     TableName tableName = TableName.valueOf("testCOWEOCfromClient");
     byte [] data = Bytes.toBytes("data");
-    HTable table = TEST_UTIL.createTable(tableName, FAMILY);
-    // get the block cache and region
-    String regionName = table.getRegionLocations().firstKey().getEncodedName();
-    Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
-      .getFromOnlineRegions(regionName);
-    Store store = region.getStores().iterator().next();
-    CacheConfig cacheConf = store.getCacheConfig();
-    cacheConf.setCacheDataOnWrite(true);
-    cacheConf.setEvictOnClose(true);
-    BlockCache cache = cacheConf.getBlockCache();
-
-    // establish baseline stats
-    long startBlockCount = cache.getBlockCount();
-    long startBlockHits = cache.getStats().getHitCount();
-    long startBlockMiss = cache.getStats().getMissCount();
-
-    // wait till baseline is stable, (minimal 500 ms)
-    for (int i = 0; i < 5; i++) {
-      Thread.sleep(100);
-      if (startBlockCount != cache.getBlockCount()
-          || startBlockHits != cache.getStats().getHitCount()
-          || startBlockMiss != cache.getStats().getMissCount()) {
-        startBlockCount = cache.getBlockCount();
-        startBlockHits = cache.getStats().getHitCount();
-        startBlockMiss = cache.getStats().getMissCount();
-        i = -1;
+    Table table = TEST_UTIL.createTable(tableName, FAMILY);
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      // get the block cache and region
+      String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+
+      Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName)
+          .getFromOnlineRegions(regionName);
+      Store store = region.getStores().iterator().next();
+      CacheConfig cacheConf = store.getCacheConfig();
+      cacheConf.setCacheDataOnWrite(true);
+      cacheConf.setEvictOnClose(true);
+      BlockCache cache = cacheConf.getBlockCache();
+
+      // establish baseline stats
+      long startBlockCount = cache.getBlockCount();
+      long startBlockHits = cache.getStats().getHitCount();
+      long startBlockMiss = cache.getStats().getMissCount();
+
+
+      // wait till baseline is stable, (minimal 500 ms)
+      for (int i = 0; i < 5; i++) {
+        Thread.sleep(100);
+        if (startBlockCount != cache.getBlockCount()
+            || startBlockHits != cache.getStats().getHitCount()
+            || startBlockMiss != cache.getStats().getMissCount()) {
+          startBlockCount = cache.getBlockCount();
+          startBlockHits = cache.getStats().getHitCount();
+          startBlockMiss = cache.getStats().getMissCount();
+          i = -1;
+        }
       }
-    }
 
-    // insert data
-    Put put = new Put(ROW);
-    put.add(FAMILY, QUALIFIER, data);
-    table.put(put);
-    assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
-    // data was in memstore so don't expect any changes
-    assertEquals(startBlockCount, cache.getBlockCount());
-    assertEquals(startBlockHits, cache.getStats().getHitCount());
-    assertEquals(startBlockMiss, cache.getStats().getMissCount());
-    // flush the data
-    System.out.println("Flushing cache");
-    region.flush(true);
-    // expect one more block in cache, no change in hits/misses
-    long expectedBlockCount = startBlockCount + 1;
-    long expectedBlockHits = startBlockHits;
-    long expectedBlockMiss = startBlockMiss;
-    assertEquals(expectedBlockCount, cache.getBlockCount());
-    assertEquals(expectedBlockHits, cache.getStats().getHitCount());
-    assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
-    // read the data and expect same blocks, one new hit, no misses
-    assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
-    assertEquals(expectedBlockCount, cache.getBlockCount());
-    assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
-    assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
-    // insert a second column, read the row, no new blocks, one new hit
-    byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
-    byte [] data2 = Bytes.add(data, data);
-    put = new Put(ROW);
-    put.add(FAMILY, QUALIFIER2, data2);
-    table.put(put);
-    Result r = table.get(new Get(ROW));
-    assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
-    assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
-    assertEquals(expectedBlockCount, cache.getBlockCount());
-    assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
-    assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
-    // flush, one new block
-    System.out.println("Flushing cache");
-    region.flush(true);
-    assertEquals(++expectedBlockCount, cache.getBlockCount());
-    assertEquals(expectedBlockHits, cache.getStats().getHitCount());
-    assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
-    // compact, net minus two blocks, two hits, no misses
-    System.out.println("Compacting");
-    assertEquals(2, store.getStorefilesCount());
-    store.triggerMajorCompaction();
-    region.compact(true);
-    waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
-    assertEquals(1, store.getStorefilesCount());
-    expectedBlockCount -= 2; // evicted two blocks, cached none
-    assertEquals(expectedBlockCount, cache.getBlockCount());
-    expectedBlockHits += 2;
-    assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
-    assertEquals(expectedBlockHits, cache.getStats().getHitCount());
-    // read the row, this should be a cache miss because we don't cache data
-    // blocks on compaction
-    r = table.get(new Get(ROW));
-    assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
-    assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
-    expectedBlockCount += 1; // cached one data block
-    assertEquals(expectedBlockCount, cache.getBlockCount());
-    assertEquals(expectedBlockHits, cache.getStats().getHitCount());
-    assertEquals(++expectedBlockMiss, cache.getStats().getMissCount());
+      // insert data
+      Put put = new Put(ROW);
+      put.add(FAMILY, QUALIFIER, data);
+      table.put(put);
+      assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
+      // data was in memstore so don't expect any changes
+      assertEquals(startBlockCount, cache.getBlockCount());
+      assertEquals(startBlockHits, cache.getStats().getHitCount());
+      assertEquals(startBlockMiss, cache.getStats().getMissCount());
+      // flush the data
+      System.out.println("Flushing cache");
+      region.flush(true);
+      // expect one more block in cache, no change in hits/misses
+      long expectedBlockCount = startBlockCount + 1;
+      long expectedBlockHits = startBlockHits;
+      long expectedBlockMiss = startBlockMiss;
+      assertEquals(expectedBlockCount, cache.getBlockCount());
+      assertEquals(expectedBlockHits, cache.getStats().getHitCount());
+      assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
+      // read the data and expect same blocks, one new hit, no misses
+      assertTrue(Bytes.equals(table.get(new Get(ROW)).value(), data));
+      assertEquals(expectedBlockCount, cache.getBlockCount());
+      assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
+      assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
+      // insert a second column, read the row, no new blocks, one new hit
+      byte [] QUALIFIER2 = Bytes.add(QUALIFIER, QUALIFIER);
+      byte [] data2 = Bytes.add(data, data);
+      put = new Put(ROW);
+      put.add(FAMILY, QUALIFIER2, data2);
+      table.put(put);
+      Result r = table.get(new Get(ROW));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
+      assertEquals(expectedBlockCount, cache.getBlockCount());
+      assertEquals(++expectedBlockHits, cache.getStats().getHitCount());
+      assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
+      // flush, one new block
+      System.out.println("Flushing cache");
+      region.flush(true);
+      assertEquals(++expectedBlockCount, cache.getBlockCount());
+      assertEquals(expectedBlockHits, cache.getStats().getHitCount());
+      assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
+      // compact, net minus two blocks, two hits, no misses
+      System.out.println("Compacting");
+      assertEquals(2, store.getStorefilesCount());
+      store.triggerMajorCompaction();
+      region.compact(true);
+      waitForStoreFileCount(store, 1, 10000); // wait 10 seconds max
+      assertEquals(1, store.getStorefilesCount());
+      expectedBlockCount -= 2; // evicted two blocks, cached none
+      assertEquals(expectedBlockCount, cache.getBlockCount());
+      expectedBlockHits += 2;
+      assertEquals(expectedBlockMiss, cache.getStats().getMissCount());
+      assertEquals(expectedBlockHits, cache.getStats().getHitCount());
+      // read the row, this should be a cache miss because we don't cache data
+      // blocks on compaction
+      r = table.get(new Get(ROW));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER), data));
+      assertTrue(Bytes.equals(r.getValue(FAMILY, QUALIFIER2), data2));
+      expectedBlockCount += 1; // cached one data block
+      assertEquals(expectedBlockCount, cache.getBlockCount());
+      assertEquals(expectedBlockHits, cache.getStats().getHitCount());
+      assertEquals(++expectedBlockMiss, cache.getStats().getMissCount());
+    }
   }
 
   private void waitForStoreFileCount(Store store, int count, int timeout)
@@ -5206,15 +5209,16 @@ public class TestFromClientSide {
     TableName TABLE = TableName.valueOf("testNonCachedGetRegionLocation");
     byte [] family1 = Bytes.toBytes("f1");
     byte [] family2 = Bytes.toBytes("f2");
-    try (HTable table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
-        Admin admin = TEST_UTIL.getHBaseAdmin()) {
-      Map <HRegionInfo, ServerName> regionsMap = table.getRegionLocations();
-      assertEquals(1, regionsMap.size());
-      HRegionInfo regionInfo = regionsMap.keySet().iterator().next();
-      ServerName addrBefore = regionsMap.get(regionInfo);
+    try (Table table = TEST_UTIL.createTable(TABLE, new byte[][] {family1, family2}, 10);
+        Admin admin = TEST_UTIL.getHBaseAdmin();
+        RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      List<HRegionLocation> allRegionLocations = locator.getAllRegionLocations();
+      assertEquals(1, allRegionLocations.size());
+      HRegionInfo regionInfo = allRegionLocations.get(0).getRegionInfo();
+      ServerName addrBefore = allRegionLocations.get(0).getServerName();
       // Verify region location before move.
-      HRegionLocation addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
-      HRegionLocation addrNoCache = table.getRegionLocation(regionInfo.getStartKey(),  true);
+      HRegionLocation addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false);
+      HRegionLocation addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(),  true);
 
       assertEquals(addrBefore.getPort(), addrCache.getPort());
       assertEquals(addrBefore.getPort(), addrNoCache.getPort());
@@ -5235,8 +5239,8 @@ public class TestFromClientSide {
       }
 
       // Verify the region was moved.
-      addrCache = table.getRegionLocation(regionInfo.getStartKey(), false);
-      addrNoCache = table.getRegionLocation(regionInfo.getStartKey(), true);
+      addrCache = locator.getRegionLocation(regionInfo.getStartKey(), false);
+      addrNoCache = locator.getRegionLocation(regionInfo.getStartKey(), true);
       assertNotNull(addrAfter);
       assertTrue(addrAfter.getPort() != addrCache.getPort());
       assertEquals(addrAfter.getPort(), addrNoCache.getPort());
@@ -5253,55 +5257,60 @@ public class TestFromClientSide {
     byte [] startKey = Bytes.toBytes("ddc");
     byte [] endKey = Bytes.toBytes("mmm");
     TableName TABLE = TableName.valueOf("testGetRegionsInRange");
-    HTable table = TEST_UTIL.createMultiRegionTable(TABLE, new byte[][] { FAMILY }, 10);
-    int numOfRegions = -1;
-    try (RegionLocator r = table.getRegionLocator()) {
-      numOfRegions = r.getStartKeys().length;
-    }
-    assertEquals(26, numOfRegions);
-
-    // Get the regions in this range
-    List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
-      endKey);
-    assertEquals(10, regionsList.size());
-
-    // Change the start key
-    startKey = Bytes.toBytes("fff");
-    regionsList = table.getRegionsInRange(startKey, endKey);
-    assertEquals(7, regionsList.size());
-
-    // Change the end key
-    endKey = Bytes.toBytes("nnn");
-    regionsList = table.getRegionsInRange(startKey, endKey);
-    assertEquals(8, regionsList.size());
+    Table t = TEST_UTIL.createMultiRegionTable(TABLE, new byte[][] { FAMILY }, 10);
 
-    // Empty start key
-    regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
-    assertEquals(13, regionsList.size());
+    if (t instanceof HTable){
+      HTable table = (HTable) t;
 
-    // Empty end key
-    regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
-    assertEquals(21, regionsList.size());
-
-    // Both start and end keys empty
-    regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
-      HConstants.EMPTY_END_ROW);
-    assertEquals(26, regionsList.size());
-
-    // Change the end key to somewhere in the last block
-    endKey = Bytes.toBytes("zzz1");
-    regionsList = table.getRegionsInRange(startKey, endKey);
-    assertEquals(21, regionsList.size());
-
-    // Change the start key to somewhere in the first block
-    startKey = Bytes.toBytes("aac");
-    regionsList = table.getRegionsInRange(startKey, endKey);
-    assertEquals(26, regionsList.size());
-
-    // Make start and end key the same
-    startKey = endKey = Bytes.toBytes("ccc");
-    regionsList = table.getRegionsInRange(startKey, endKey);
-    assertEquals(1, regionsList.size());
+      int numOfRegions = -1;
+      try (RegionLocator r = table.getRegionLocator()) {
+        numOfRegions = r.getStartKeys().length;
+      }
+      assertEquals(26, numOfRegions);
+
+      // Get the regions in this range
+      List<HRegionLocation> regionsList = table.getRegionsInRange(startKey,
+          endKey);
+      assertEquals(10, regionsList.size());
+
+      // Change the start key
+      startKey = Bytes.toBytes("fff");
+      regionsList = table.getRegionsInRange(startKey, endKey);
+      assertEquals(7, regionsList.size());
+
+      // Change the end key
+      endKey = Bytes.toBytes("nnn");
+      regionsList = table.getRegionsInRange(startKey, endKey);
+      assertEquals(8, regionsList.size());
+
+      // Empty start key
+      regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW, endKey);
+      assertEquals(13, regionsList.size());
+
+      // Empty end key
+      regionsList = table.getRegionsInRange(startKey, HConstants.EMPTY_END_ROW);
+      assertEquals(21, regionsList.size());
+
+      // Both start and end keys empty
+      regionsList = table.getRegionsInRange(HConstants.EMPTY_START_ROW,
+          HConstants.EMPTY_END_ROW);
+      assertEquals(26, regionsList.size());
+
+      // Change the end key to somewhere in the last block
+      endKey = Bytes.toBytes("zzz1");
+      regionsList = table.getRegionsInRange(startKey, endKey);
+      assertEquals(21, regionsList.size());
+
+      // Change the start key to somewhere in the first block
+      startKey = Bytes.toBytes("aac");
+      regionsList = table.getRegionsInRange(startKey, endKey);
+      assertEquals(26, regionsList.size());
+
+      // Make start and end key the same
+      startKey = endKey = Bytes.toBytes("ccc");
+      regionsList = table.getRegionsInRange(startKey, endKey);
+      assertEquals(1, regionsList.size());
+    }
   }
 
   @Test
@@ -5323,7 +5332,8 @@ public class TestFromClientSide {
     scan.setStartRow(Bytes.toBytes(1));
     scan.setStopRow(Bytes.toBytes(3));
     scan.addColumn(FAMILY, FAMILY);
-    scan.setFilter(new RowFilter(CompareFilter.CompareOp.NOT_EQUAL, new BinaryComparator(Bytes.toBytes(1))));
+    scan.setFilter(new RowFilter(CompareFilter.CompareOp.NOT_EQUAL,
+        new BinaryComparator(Bytes.toBytes(1))));
 
     ResultScanner scanner = foo.getScanner(scan);
     Result[] bar = scanner.next(100);
@@ -5730,8 +5740,7 @@ public class TestFromClientSide {
     for (Result result : scanner) {
       assertEquals(result.size(), 1);
       assertTrue(Bytes.equals(result.rawCells()[0].getRow(), ROWS[expectedIndex]));
-      assertTrue(Bytes.equals(result.rawCells()[0].getQualifier(),
-          QUALIFIERS[expectedIndex]));
+      assertTrue(Bytes.equals(result.rawCells()[0].getQualifier(), QUALIFIERS[expectedIndex]));
       expectedIndex--;
     }
     assertEquals(expectedIndex, 0);
@@ -5889,7 +5898,7 @@ public class TestFromClientSide {
     byte[][] FAMILIES = makeNAscii(FAMILY, 3);
     byte[][] VALUES = makeN(VALUE, 5);
     long[] ts = { 1000, 2000, 3000, 4000, 5000 };
-    Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration(), 3);
+    Table ht = TEST_UTIL.createTable(TABLE, FAMILIES, 3);
 
     Put put = new Put(ROW);
     put.add(FAMILIES[0], QUALIFIER, ts[0], VALUES[0]);
@@ -6079,10 +6088,12 @@ public class TestFromClientSide {
         Bytes.toBytes("007"),
         Bytes.add(Bytes.toBytes("007"), Bytes.multiple(maxByteArray, 4)),
         Bytes.toBytes("008"), Bytes.multiple(maxByteArray, 2) };
-    HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
+    Table table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
 
-    assertEquals(splitRows.length + 1, table.getRegionLocations().size());
+    try(RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      assertEquals(splitRows.length + 1, l.getAllRegionLocations().size());
+    }
     // Insert one row each region
     int insertNum = splitRows.length;
     for (int i = 0; i < insertNum; i++) {
@@ -6131,10 +6142,12 @@ public class TestFromClientSide {
     byte[][] splitRows = new byte[][]{
         Bytes.toBytes("000"), Bytes.toBytes("002"), Bytes.toBytes("004"),
         Bytes.toBytes("006"), Bytes.toBytes("008"), Bytes.toBytes("010")};
-    HTable table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
+    Table table = TEST_UTIL.createTable(TABLE, FAMILY, splitRows);
     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
 
-    assertEquals(splitRows.length + 1, table.getRegionLocations().size());
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      assertEquals(splitRows.length + 1, l.getAllRegionLocations().size());
+    }
     for (byte[] splitRow : splitRows) {
       Put put = new Put(splitRow);
       put.add(FAMILY, QUALIFIER, VALUE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index 1e7fbc7..22309ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -100,7 +100,7 @@ public class TestFromClientSide3 {
     for (HTableDescriptor htd: TEST_UTIL.getHBaseAdmin().listTables()) {
       LOG.info("Tear down, remove table=" + htd.getTableName());
       TEST_UTIL.deleteTable(htd.getTableName());
-    }
+  }
   }
 
   private void randomCFPuts(Table table, byte[] row, byte[] family, int nPuts)
@@ -114,28 +114,23 @@ public class TestFromClientSide3 {
     table.put(put);
   }
 
-  private void performMultiplePutAndFlush(HBaseAdmin admin, HTable table,
+  private void performMultiplePutAndFlush(HBaseAdmin admin, Table table,
       byte[] row, byte[] family, int nFlushes, int nPuts)
   throws Exception {
 
-    // connection needed for poll-wait
-    HRegionLocation loc = table.getRegionLocation(row, true);
-    AdminProtos.AdminService.BlockingInterface server =
-      admin.getConnection().getAdmin(loc.getServerName());
-    byte[] regName = loc.getRegionInfo().getRegionName();
-
-    for (int i = 0; i < nFlushes; i++) {
-      randomCFPuts(table, row, family, nPuts);
-      List<String> sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY);
-      int sfCount = sf.size();
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(table.getName())) {
+      // connection needed for poll-wait
+      HRegionLocation loc = locator.getRegionLocation(row, true);
+      AdminProtos.AdminService.BlockingInterface server =
+          admin.getConnection().getAdmin(loc.getServerName());
+      byte[] regName = loc.getRegionInfo().getRegionName();
 
-      // TODO: replace this api with a synchronous flush after HBASE-2949
-      admin.flush(table.getName());
+      for (int i = 0; i < nFlushes; i++) {
+        randomCFPuts(table, row, family, nPuts);
+        List<String> sf = ProtobufUtil.getStoreFiles(server, regName, FAMILY);
+        int sfCount = sf.size();
 
-      // synchronously poll wait for a new storefile to appear (flush happened)
-      while (ProtobufUtil.getStoreFiles(
-          server, regName, FAMILY).size() == sfCount) {
-        Thread.sleep(40);
+        admin.flush(table.getName());
       }
     }
   }
@@ -156,151 +151,147 @@ public class TestFromClientSide3 {
 
     String tableName = "testAdvancedConfigOverride";
     TableName TABLE = TableName.valueOf(tableName);
-    HTable hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
+    Table hTable = TEST_UTIL.createTable(TABLE, FAMILY, 10);
     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-    ClusterConnection connection = (ClusterConnection)TEST_UTIL.getConnection();
+    ClusterConnection connection = (ClusterConnection) TEST_UTIL.getConnection();
 
     // Create 3 store files.
     byte[] row = Bytes.toBytes(random.nextInt());
     performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 100);
 
-    // Verify we have multiple store files.
-    HRegionLocation loc = hTable.getRegionLocation(row, true);
-    byte[] regionName = loc.getRegionInfo().getRegionName();
-    AdminProtos.AdminService.BlockingInterface server =
-      connection.getAdmin(loc.getServerName());
-    assertTrue(ProtobufUtil.getStoreFiles(
-      server, regionName, FAMILY).size() > 1);
-
-    // Issue a compaction request
-    admin.compact(TABLE.getName());
-
-    // poll wait for the compactions to happen
-    for (int i = 0; i < 10 * 1000 / 40; ++i) {
-      // The number of store files after compaction should be lesser.
-      loc = hTable.getRegionLocation(row, true);
-      if (!loc.getRegionInfo().isOffline()) {
-        regionName = loc.getRegionInfo().getRegionName();
-        server = connection.getAdmin(loc.getServerName());
-        if (ProtobufUtil.getStoreFiles(
-            server, regionName, FAMILY).size() <= 1) {
-          break;
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      // Verify we have multiple store files.
+      HRegionLocation loc = locator.getRegionLocation(row, true);
+      byte[] regionName = loc.getRegionInfo().getRegionName();
+      AdminProtos.AdminService.BlockingInterface server = connection.getAdmin(loc.getServerName());
+      assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() > 1);
+
+      // Issue a compaction request
+      admin.compact(TABLE.getName());
+
+      // poll wait for the compactions to happen
+      for (int i = 0; i < 10 * 1000 / 40; ++i) {
+        // The number of store files after compaction should be lesser.
+        loc = locator.getRegionLocation(row, true);
+        if (!loc.getRegionInfo().isOffline()) {
+          regionName = loc.getRegionInfo().getRegionName();
+          server = connection.getAdmin(loc.getServerName());
+          if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1) {
+            break;
+          }
         }
+        Thread.sleep(40);
       }
-      Thread.sleep(40);
-    }
-    // verify the compactions took place and that we didn't just time out
-    assertTrue(ProtobufUtil.getStoreFiles(
-      server, regionName, FAMILY).size() <= 1);
-
-    // change the compaction.min config option for this table to 5
-    LOG.info("hbase.hstore.compaction.min should now be 5");
-    HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
-    htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
-    admin.modifyTable(TABLE, htd);
-    Pair<Integer, Integer> st;
-    while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
-      LOG.debug(st.getFirst() + " regions left to update");
-      Thread.sleep(40);
-    }
-    LOG.info("alter status finished");
-
-    // Create 3 more store files.
-    performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
-
-    // Issue a compaction request
-    admin.compact(TABLE.getName());
-
-    // This time, the compaction request should not happen
-    Thread.sleep(10 * 1000);
-    loc = hTable.getRegionLocation(row, true);
-    regionName = loc.getRegionInfo().getRegionName();
-    server = connection.getAdmin(loc.getServerName());
-    int sfCount = ProtobufUtil.getStoreFiles(
-      server, regionName, FAMILY).size();
-    assertTrue(sfCount > 1);
-
-    // change an individual CF's config option to 2 & online schema update
-    LOG.info("hbase.hstore.compaction.min should now be 2");
-    HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
-    hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
-    htd.modifyFamily(hcd);
-    admin.modifyTable(TABLE, htd);
-    while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
-      LOG.debug(st.getFirst() + " regions left to update");
-      Thread.sleep(40);
-    }
-    LOG.info("alter status finished");
+      // verify the compactions took place and that we didn't just time out
+      assertTrue(ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() <= 1);
+
+      // change the compaction.min config option for this table to 5
+      LOG.info("hbase.hstore.compaction.min should now be 5");
+      HTableDescriptor htd = new HTableDescriptor(hTable.getTableDescriptor());
+      htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
+      admin.modifyTable(TABLE, htd);
+      Pair<Integer, Integer> st;
+      while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
+        LOG.debug(st.getFirst() + " regions left to update");
+        Thread.sleep(40);
+      }
+      LOG.info("alter status finished");
+
+      // Create 3 more store files.
+      performMultiplePutAndFlush(admin, hTable, row, FAMILY, 3, 10);
 
-    // Issue a compaction request
-    admin.compact(TABLE.getName());
+      // Issue a compaction request
+      admin.compact(TABLE.getName());
 
-    // poll wait for the compactions to happen
-    for (int i = 0; i < 10 * 1000 / 40; ++i) {
-      loc = hTable.getRegionLocation(row, true);
+      // This time, the compaction request should not happen
+      Thread.sleep(10 * 1000);
+      loc = locator.getRegionLocation(row, true);
       regionName = loc.getRegionInfo().getRegionName();
-      try {
-        server = connection.getAdmin(loc.getServerName());
-        if (ProtobufUtil.getStoreFiles(
-            server, regionName, FAMILY).size() < sfCount) {
-          break;
+      server = connection.getAdmin(loc.getServerName());
+      int sfCount = ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size();
+      assertTrue(sfCount > 1);
+
+      // change an individual CF's config option to 2 & online schema update
+      LOG.info("hbase.hstore.compaction.min should now be 2");
+      HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
+      hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
+      htd.modifyFamily(hcd);
+      admin.modifyTable(TABLE, htd);
+      while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
+        LOG.debug(st.getFirst() + " regions left to update");
+        Thread.sleep(40);
+      }
+      LOG.info("alter status finished");
+
+      // Issue a compaction request
+      admin.compact(TABLE.getName());
+
+      // poll wait for the compactions to happen
+      for (int i = 0; i < 10 * 1000 / 40; ++i) {
+        loc = locator.getRegionLocation(row, true);
+        regionName = loc.getRegionInfo().getRegionName();
+        try {
+          server = connection.getAdmin(loc.getServerName());
+          if (ProtobufUtil.getStoreFiles(server, regionName, FAMILY).size() < sfCount) {
+            break;
+          }
+        } catch (Exception e) {
+          LOG.debug("Waiting for region to come online: " + regionName);
         }
-      } catch (Exception e) {
-        LOG.debug("Waiting for region to come online: " + regionName);
+        Thread.sleep(40);
       }
-      Thread.sleep(40);
-    }
-    // verify the compaction took place and that we didn't just time out
-    assertTrue(ProtobufUtil.getStoreFiles(
-      server, regionName, FAMILY).size() < sfCount);
-
-    // Finally, ensure that we can remove a custom config value after we made it
-    LOG.info("Removing CF config value");
-    LOG.info("hbase.hstore.compaction.min should now be 5");
-    hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
-    hcd.setValue("hbase.hstore.compaction.min", null);
-    htd.modifyFamily(hcd);
-    admin.modifyTable(TABLE, htd);
-    while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
-      LOG.debug(st.getFirst() + " regions left to update");
-      Thread.sleep(40);
+
+      // verify the compaction took place and that we didn't just time out
+      assertTrue(ProtobufUtil.getStoreFiles(
+        server, regionName, FAMILY).size() < sfCount);
+
+      // Finally, ensure that we can remove a custom config value after we made it
+      LOG.info("Removing CF config value");
+      LOG.info("hbase.hstore.compaction.min should now be 5");
+      hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
+      hcd.setValue("hbase.hstore.compaction.min", null);
+      htd.modifyFamily(hcd);
+      admin.modifyTable(TABLE, htd);
+      while (null != (st = admin.getAlterStatus(TABLE)) && st.getFirst() > 0) {
+        LOG.debug(st.getFirst() + " regions left to update");
+        Thread.sleep(40);
+      }
+      LOG.info("alter status finished");
+      assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
+          "hbase.hstore.compaction.min"));
     }
-    LOG.info("alter status finished");
-    assertNull(hTable.getTableDescriptor().getFamily(FAMILY).getValue(
-        "hbase.hstore.compaction.min"));
   }
 
   @Test
-  public void testHTableBatchWithEmptyPut() throws Exception {
-    Table table = TEST_UTIL.createTable(
-      Bytes.toBytes("testHTableBatchWithEmptyPut"), new byte[][] { FAMILY });
+  public void testHTableBatchWithEmptyPut ()throws Exception {
+      Table table = TEST_UTIL.createTable(TableName.valueOf("testHTableBatchWithEmptyPut"),
+          new byte[][] { FAMILY });
     try {
       List actions = (List) new ArrayList();
       Object[] results = new Object[2];
       // create an empty Put
       Put put1 = new Put(ROW);
       actions.add(put1);
-
+      
       Put put2 = new Put(ANOTHERROW);
       put2.add(FAMILY, QUALIFIER, VALUE);
       actions.add(put2);
-
+      
       table.batch(actions, results);
       fail("Empty Put should have failed the batch call");
     } catch (IllegalArgumentException iae) {
-
+      
     } finally {
       table.close();
     }
   }
-
+  
   @Test
   public void testHTableExistsMethodSingleRegionSingleGet() throws Exception {
-
-    // Test with a single region table.
-
-    Table table = TEST_UTIL.createTable(
-      Bytes.toBytes("testHTableExistsMethodSingleRegionSingleGet"), new byte[][] { FAMILY });
+      // Test with a single region table.
+      Table table = TEST_UTIL.createTable(
+          TableName.valueOf("testHTableExistsMethodSingleRegionSingleGet"),
+          new byte[][] { FAMILY });
 
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
@@ -317,9 +308,8 @@ public class TestFromClientSide3 {
   }
 
   public void testHTableExistsMethodSingleRegionMultipleGets() throws Exception {
-
-    HTable table = TEST_UTIL.createTable(
-      Bytes.toBytes("testHTableExistsMethodSingleRegionMultipleGets"), new byte[][] { FAMILY });
+    Table table = TEST_UTIL.createTable(TableName.valueOf(
+        "testHTableExistsMethodSingleRegionMultipleGets"), new byte[][] { FAMILY });
 
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
@@ -330,7 +320,7 @@ public class TestFromClientSide3 {
     gets.add(null);
     gets.add(new Get(ANOTHERROW));
 
-    Boolean[] results = table.exists(gets);
+    boolean[] results = table.existsAll(gets);
     assertEquals(results[0], true);
     assertEquals(results[1], false);
     assertEquals(results[2], false);
@@ -338,11 +328,11 @@ public class TestFromClientSide3 {
 
   @Test
   public void testHTableExistsBeforeGet() throws Exception {
-    Table table = TEST_UTIL.createTable(
-      Bytes.toBytes("testHTableExistsBeforeGet"), new byte[][] { FAMILY });
+    Table table = TEST_UTIL.createTable(TableName.valueOf("testHTableExistsBeforeGet"),
+        new byte[][] { FAMILY });
     try {
       Put put = new Put(ROW);
-      put.add(FAMILY, QUALIFIER, VALUE);
+      put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
 
       Get get = new Get(ROW);
@@ -362,13 +352,13 @@ public class TestFromClientSide3 {
   public void testHTableExistsAllBeforeGet() throws Exception {
     final byte[] ROW2 = Bytes.add(ROW, Bytes.toBytes("2"));
     Table table = TEST_UTIL.createTable(
-      Bytes.toBytes("testHTableExistsAllBeforeGet"), new byte[][] { FAMILY });
+        TableName.valueOf("testHTableExistsAllBeforeGet"), new byte[][] { FAMILY });
     try {
       Put put = new Put(ROW);
-      put.add(FAMILY, QUALIFIER, VALUE);
+      put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
       put = new Put(ROW2);
-      put.add(FAMILY, QUALIFIER, VALUE);
+      put.addColumn(FAMILY, QUALIFIER, VALUE);
       table.put(put);
 
       Get get = new Get(ROW);
@@ -393,12 +383,11 @@ public class TestFromClientSide3 {
 
   @Test
   public void testHTableExistsMethodMultipleRegionsSingleGet() throws Exception {
-
     Table table = TEST_UTIL.createTable(
       TableName.valueOf("testHTableExistsMethodMultipleRegionsSingleGet"), new byte[][] { FAMILY },
       1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
     Put put = new Put(ROW);
-    put.add(FAMILY, QUALIFIER, VALUE);
+    put.addColumn(FAMILY, QUALIFIER, VALUE);
 
     Get get = new Get(ROW);
 
@@ -413,8 +402,8 @@ public class TestFromClientSide3 {
 
   @Test
   public void testHTableExistsMethodMultipleRegionsMultipleGets() throws Exception {
-    HTable table = TEST_UTIL.createTable(
-      TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"),
+    Table table = TEST_UTIL.createTable(
+      TableName.valueOf("testHTableExistsMethodMultipleRegionsMultipleGets"), 
       new byte[][] { FAMILY }, 1, new byte[] { 0x00 }, new byte[] { (byte) 0xff }, 255);
     Put put = new Put(ROW);
     put.add(FAMILY, QUALIFIER, VALUE);
@@ -427,7 +416,7 @@ public class TestFromClientSide3 {
     gets.add(new Get(Bytes.add(ANOTHERROW, new byte[] { 0x00 })));
 
     LOG.info("Calling exists");
-    Boolean[] results = table.exists(gets);
+    boolean[] results = table.existsAll(gets);
     assertEquals(results[0], false);
     assertEquals(results[1], false);
     assertEquals(results[2], true);
@@ -441,7 +430,7 @@ public class TestFromClientSide3 {
     gets = new ArrayList<Get>();
     gets.add(new Get(new byte[] { 0x00 }));
     gets.add(new Get(new byte[] { 0x00, 0x00 }));
-    results = table.exists(gets);
+    results = table.existsAll(gets);
     assertEquals(results[0], true);
     assertEquals(results[1], false);
 
@@ -454,7 +443,7 @@ public class TestFromClientSide3 {
     gets.add(new Get(new byte[] { (byte) 0xff }));
     gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff }));
     gets.add(new Get(new byte[] { (byte) 0xff, (byte) 0xff, (byte) 0xff }));
-    results = table.exists(gets);
+    results = table.existsAll(gets);
     assertEquals(results[0], false);
     assertEquals(results[1], true);
     assertEquals(results[2], false);
@@ -491,8 +480,8 @@ public class TestFromClientSide3 {
 
   @Test
   public void testLeaseRenewal() throws Exception {
-    HTable table = TEST_UTIL.createTable(
-      Bytes.toBytes("testLeaseRenewal"), FAMILY);
+    Table table = TEST_UTIL.createTable(
+      TableName.valueOf("testLeaseRenewal"), FAMILY);
     Put p = new Put(ROW_BYTES);
     p.addColumn(FAMILY, COL_QUAL, VAL_BYTES);
     table.put(p);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
index f5807c2..323d2f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideNoCodec.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.AbstractRpcClient;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -61,10 +62,10 @@ public class TestFromClientSideNoCodec {
 
   @Test
   public void testBasics() throws IOException {
-    final byte [] t = Bytes.toBytes("testBasics");
+    final TableName t = TableName.valueOf("testBasics");
     final byte [][] fs = new byte[][] {Bytes.toBytes("cf1"), Bytes.toBytes("cf2"),
       Bytes.toBytes("cf3") };
-    HTable ht = TEST_UTIL.createTable(t, fs);
+    Table ht = TEST_UTIL.createTable(t, fs);
     // Check put and get.
     final byte [] row = Bytes.toBytes("row");
     Put p = new Put(row);
@@ -79,10 +80,13 @@ public class TestFromClientSideNoCodec {
         Bytes.equals(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
           f, 0, f.length));
     }
-    // Check getRowOrBefore
-    byte [] f = fs[0];
-    r = ht.getRowOrBefore(row, f);
-    assertTrue(r.toString(), r.containsColumn(f, f));
+    if(ht instanceof HTableInterface) {
+      HTableInterface hti = (HTableInterface) ht;
+      // Check getRowOrBefore
+      byte[] f = fs[0];
+      r = hti.getRowOrBefore(row, f);
+      assertTrue(r.toString(), r.containsColumn(f, f));
+    }
     // Check scan.
     ResultScanner scanner = ht.getScanner(new Scan());
     int count = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
index 9ed5be6..e946b21 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
@@ -139,33 +139,34 @@ public class TestHCM {
     Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
     Connection con2 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), otherPool);
     // make sure the internally created ExecutorService is the one passed
-    assertTrue(otherPool == ((ConnectionImplementation)con2).getCurrentBatchPool());
+    assertTrue(otherPool == ((ConnectionImplementation) con2).getCurrentBatchPool());
 
-    String tableName = "testClusterConnection";
-    TEST_UTIL.createTable(tableName.getBytes(), FAM_NAM).close();
-    HTable t = (HTable)con1.getTable(TableName.valueOf(tableName), otherPool);
+    TableName tableName = TableName.valueOf("testClusterConnection");
+    TEST_UTIL.createTable(tableName, FAM_NAM).close();
+    HTable t = (HTable)con1.getTable(tableName, otherPool);
     // make sure passing a pool to the getTable does not trigger creation of an internal pool
-    assertNull("Internal Thread pool should be null", ((ConnectionImplementation)con1).getCurrentBatchPool());
+    assertNull("Internal Thread pool should be null",
+        ((ConnectionImplementation) con1).getCurrentBatchPool());
     // table should use the pool passed
     assertTrue(otherPool == t.getPool());
     t.close();
 
-    t = (HTable)con2.getTable(TableName.valueOf(tableName));
+    t = (HTable)con2.getTable(tableName);
     // table should use the connectin's internal pool
     assertTrue(otherPool == t.getPool());
     t.close();
 
-    t = (HTable)con2.getTable(TableName.valueOf(tableName));
+    t = (HTable)con2.getTable(tableName);
     // try other API too
     assertTrue(otherPool == t.getPool());
     t.close();
 
-    t = (HTable)con2.getTable(TableName.valueOf(tableName));
+    t = (HTable)con2.getTable(tableName);
     // try other API too
     assertTrue(otherPool == t.getPool());
     t.close();
 
-    t = (HTable)con1.getTable(TableName.valueOf(tableName));
+    t = (HTable)con1.getTable(tableName);
     ExecutorService pool = ((ConnectionImplementation)con1).getCurrentBatchPool();
     // make sure an internal pool was created
     assertNotNull("An internal Thread pool should have been created", pool);
@@ -173,7 +174,7 @@ public class TestHCM {
     assertTrue(t.getPool() == pool);
     t.close();
 
-    t = (HTable)con1.getTable(TableName.valueOf(tableName));
+    t = (HTable)con1.getTable(tableName);
     // still using the *same* internal pool
     assertTrue(t.getPool() == pool);
     t.close();
@@ -214,25 +215,27 @@ public class TestHCM {
     rs.waitForServerOnline();
     final ServerName sn = rs.getRegionServer().getServerName();
 
-    HTable t = TEST_UTIL.createTable(tn, cf);
+    Table t = TEST_UTIL.createTable(tn, cf);
     TEST_UTIL.waitTableAvailable(tn);
 
     while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
         getRegionStates().isRegionsInTransition()){
       Thread.sleep(1);
     }
-    final ConnectionImplementation hci =  (ConnectionImplementation)t.getConnection();
-    while (t.getRegionLocation(rk).getPort() != sn.getPort()){
-      TEST_UTIL.getHBaseAdmin().move(t.getRegionLocation(rk).getRegionInfo().
-          getEncodedNameAsBytes(), Bytes.toBytes(sn.toString()));
-      while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
-          getRegionStates().isRegionsInTransition()){
-        Thread.sleep(1);
+    final ConnectionImplementation hci =  (ConnectionImplementation)TEST_UTIL.getConnection();
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tn)) {
+      while (l.getRegionLocation(rk).getPort() != sn.getPort()) {
+        TEST_UTIL.getHBaseAdmin().move(l.getRegionLocation(rk).getRegionInfo().
+            getEncodedNameAsBytes(), Bytes.toBytes(sn.toString()));
+        while (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
+            getRegionStates().isRegionsInTransition()) {
+          Thread.sleep(1);
+        }
+        hci.clearRegionCache(tn);
       }
-      hci.clearRegionCache(tn);
+      Assert.assertNotNull(hci.clusterStatusListener);
+      TEST_UTIL.assertRegionOnServer(l.getRegionLocation(rk).getRegionInfo(), sn, 20000);
     }
-    Assert.assertNotNull(hci.clusterStatusListener);
-    TEST_UTIL.assertRegionOnServer(t.getRegionLocation(rk).getRegionInfo(), sn, 20000);
 
     Put p1 = new Put(rk);
     p1.add(cf, "qual".getBytes(), "val".getBytes());
@@ -286,27 +289,31 @@ public class TestHCM {
   public void testOperationTimeout() throws Exception {
     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
     hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
-    HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
+    Table t = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, null);
 
-    // Check that it works if the timeout is big enough
-    table.setOperationTimeout(120 * 1000);
-    table.get(new Get(FAM_NAM));
+    if (t instanceof HTable) {
+      HTable table = (HTable) t;
 
-    // Resetting and retrying. Will fail this time, not enough time for the second try
-    SleepAndFailFirstTime.ct.set(0);
-    try {
-      table.setOperationTimeout(30 * 1000);
+      // Check that it works if the timeout is big enough
+      table.setOperationTimeout(120 * 1000);
       table.get(new Get(FAM_NAM));
-      Assert.fail("We expect an exception here");
-    } catch (SocketTimeoutException e) {
-      // The client has a CallTimeout class, but it's not shared.We're not very clean today,
-      //  in the general case you can expect the call to stop, but the exception may vary.
-      // In this test however, we're sure that it will be a socket timeout.
-      LOG.info("We received an exception, as expected ", e);
-    } catch (IOException e) {
-      Assert.fail("Wrong exception:" + e.getMessage());
-    } finally {
-      table.close();
+
+      // Resetting and retrying. Will fail this time, not enough time for the second try
+      SleepAndFailFirstTime.ct.set(0);
+      try {
+        table.setOperationTimeout(30 * 1000);
+        table.get(new Get(FAM_NAM));
+        Assert.fail("We expect an exception here");
+      } catch (SocketTimeoutException e) {
+        // The client has a CallTimeout class, but it's not shared.We're not very clean today,
+        //  in the general case you can expect the call to stop, but the exception may vary.
+        // In this test however, we're sure that it will be a socket timeout.
+        LOG.info("We received an exception, as expected ", e);
+      } catch (IOException e) {
+        Assert.fail("Wrong exception:" + e.getMessage());
+      } finally {
+        table.close();
+      }
     }
   }
 
@@ -326,10 +333,10 @@ public class TestHCM {
     c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt);
 
     Connection connection = ConnectionFactory.createConnection(c2);
-    final HTable table = (HTable) connection.getTable(tableName);
+    final Table table = connection.getTable(tableName);
 
     Put put = new Put(ROW);
-    put.add(FAM_NAM, ROW, ROW);
+    put.addColumn(FAM_NAM, ROW, ROW);
     table.put(put);
 
     // 4 steps: ready=0; doGets=1; mustStop=2; stopped=3
@@ -364,9 +371,12 @@ public class TestHCM {
       }
     });
 
-    ServerName sn = table.getRegionLocation(ROW).getServerName();
+    ServerName sn;
+    try(RegionLocator rl = connection.getRegionLocator(tableName)) {
+      sn = rl.getRegionLocation(ROW).getServerName();
+    }
     ConnectionImplementation conn =
-        (ConnectionImplementation) table.getConnection();
+        (ConnectionImplementation) connection;
     RpcClient rpcClient = conn.getRpcClient();
 
     LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn);
@@ -469,14 +479,18 @@ public class TestHCM {
     c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000);
 
     final Connection connection = ConnectionFactory.createConnection(c2);
-    final HTable table = (HTable) connection.getTable(tableName);
+    final Table table = connection.getTable(tableName);
 
     Put p = new Put(FAM_NAM);
-    p.add(FAM_NAM, FAM_NAM, FAM_NAM);
+    p.addColumn(FAM_NAM, FAM_NAM, FAM_NAM);
     table.put(p);
 
-    final ConnectionImplementation hci =  (ConnectionImplementation)table.getConnection();
-    final HRegionLocation loc = table.getRegionLocation(FAM_NAM);
+    final ConnectionImplementation hci =  (ConnectionImplementation) connection;
+
+    final HRegionLocation loc;
+    try(RegionLocator rl = connection.getRegionLocator(tableName)) {
+      loc = rl.getRegionLocation(FAM_NAM);
+    }
 
     Get get = new Get(FAM_NAM);
     Assert.assertNotNull(table.get(get));
@@ -553,14 +567,13 @@ public class TestHCM {
     Configuration conf =  new Configuration(TEST_UTIL.getConfiguration());
     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
     Connection connection = ConnectionFactory.createConnection(conf);
-    final HTable table = (HTable) connection.getTable(TABLE_NAME);
+    final Table table = connection.getTable(TABLE_NAME);
 
     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
     Put put = new Put(ROW);
     put.add(FAM_NAM, ROW, ROW);
     table.put(put);
-    ConnectionImplementation conn =
-      (ConnectionImplementation)table.getConnection();
+    ConnectionImplementation conn = (ConnectionImplementation) connection;
 
     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
 
@@ -756,12 +769,11 @@ public class TestHCM {
    */
   @Test(timeout = 60000)
   public void testCacheSeqNums() throws Exception{
-    HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM);
+    Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM);
     Put put = new Put(ROW);
     put.add(FAM_NAM, ROW, ROW);
     table.put(put);
-    ConnectionImplementation conn =
-      (ConnectionImplementation)table.getConnection();
+    ConnectionImplementation conn = (ConnectionImplementation) TEST_UTIL.getConnection();
 
     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
     assertNotNull(location);
@@ -870,10 +882,10 @@ public class TestHCM {
 
   @Test (timeout=30000)
   public void testMulti() throws Exception {
-    HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
+    Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
      try {
        ConnectionImplementation conn =
-           (ConnectionImplementation)table.getConnection();
+           (ConnectionImplementation)TEST_UTIL.getConnection();
 
        // We're now going to move the region and check that it works for the client
        // First a new put to add the location in the cache
@@ -1053,7 +1065,7 @@ public class TestHCM {
     Configuration config = new Configuration(TEST_UTIL.getConfiguration());
 
     TableName tableName = TableName.valueOf("testConnectionRideOverClusterRestart");
-    TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close();
+    TEST_UTIL.createTable(tableName, new byte[][] {FAM_NAM}).close();
 
     Connection connection = ConnectionFactory.createConnection(config);
     Table table = connection.getTable(tableName);


[4/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java
index 43ba242..908fcdf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexer.java
@@ -64,7 +64,7 @@ public class TestHTableMultiplexer {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private static void checkExistence(HTable htable, byte[] row, byte[] family, byte[] quality)
+  private static void checkExistence(Table htable, byte[] row, byte[] family, byte[] quality)
       throws Exception {
     // verify that the Get returns the correct result
     Result r;
@@ -93,63 +93,65 @@ public class TestHTableMultiplexer {
     HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), 
         PER_REGIONSERVER_QUEUE_SIZE);
 
-    HTable htable1 =
+    Table htable1 =
         TEST_UTIL.createTable(TABLE_1, new byte[][] { FAMILY }, VERSION,
         Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
-    HTable htable2 =
+    Table htable2 =
         TEST_UTIL.createTable(TABLE_2, new byte[][] { FAMILY }, VERSION, Bytes.toBytes("aaaaa"),
           Bytes.toBytes("zzzzz"), NUM_REGIONS);
     TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_1);
     TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_2);
 
-    byte[][] startRows = htable1.getStartKeys();
-    byte[][] endRows = htable1.getEndKeys();
-
-    // SinglePut case
-    for (int i = 0; i < NUM_REGIONS; i++) {
-      byte [] row = startRows[i];
-      if (row == null || row.length <= 0) continue;
-      Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
-      success = multiplexer.put(TABLE_1, put);
-      assertTrue("multiplexer.put returns", success);
-
-      put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
-      success = multiplexer.put(TABLE_2, put);
-      assertTrue("multiplexer.put failed", success);
-
-      LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1));
+    try (RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(TABLE_1)) {
+      byte[][] startRows = rl.getStartKeys();
+      byte[][] endRows = rl.getEndKeys();
+
+      // SinglePut case
+      for (int i = 0; i < NUM_REGIONS; i++) {
+        byte [] row = startRows[i];
+        if (row == null || row.length <= 0) continue;
+        Put put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
+        success = multiplexer.put(TABLE_1, put);
+        assertTrue("multiplexer.put returns", success);
+
+        put = new Put(row).add(FAMILY, QUALIFIER, VALUE1);
+        success = multiplexer.put(TABLE_2, put);
+        assertTrue("multiplexer.put failed", success);
+
+        LOG.info("Put for " + Bytes.toStringBinary(startRows[i]) + " @ iteration " + (i + 1));
+
+        // verify that the Get returns the correct result
+        checkExistence(htable1, startRows[i], FAMILY, QUALIFIER);
+        checkExistence(htable2, startRows[i], FAMILY, QUALIFIER);
+      }
+
+      // MultiPut case
+      List<Put> multiput = new ArrayList<Put>();
+      for (int i = 0; i < NUM_REGIONS; i++) {
+        byte [] row = endRows[i];
+        if (row == null || row.length <= 0) continue;
+        Put put = new Put(row);
+        put.add(FAMILY, QUALIFIER, VALUE2);
+        multiput.add(put);
+      }
+      failedPuts = multiplexer.put(TABLE_1, multiput);
+      assertTrue(failedPuts == null);
 
       // verify that the Get returns the correct result
-      checkExistence(htable1, startRows[i], FAMILY, QUALIFIER);
-      checkExistence(htable2, startRows[i], FAMILY, QUALIFIER);
-    }
-
-    // MultiPut case
-    List<Put> multiput = new ArrayList<Put>();
-    for (int i = 0; i < NUM_REGIONS; i++) {
-      byte [] row = endRows[i];
-      if (row == null || row.length <= 0) continue;
-      Put put = new Put(row);
-      put.add(FAMILY, QUALIFIER, VALUE2);
-      multiput.add(put);
-    }
-    failedPuts = multiplexer.put(TABLE_1, multiput);
-    assertTrue(failedPuts == null);
-
-    // verify that the Get returns the correct result
-    for (int i = 0; i < NUM_REGIONS; i++) {
-      byte [] row = endRows[i];
-      if (row == null || row.length <= 0) continue;
-      Get get = new Get(row);
-      get.addColumn(FAMILY, QUALIFIER);
-      Result r;
-      int nbTry = 0;
-      do {
-        assertTrue(nbTry++ < 50);
-        Thread.sleep(100);
-        r = htable1.get(get);
-      } while (r == null || r.getValue(FAMILY, QUALIFIER) == null ||
-          Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0);
+      for (int i = 0; i < NUM_REGIONS; i++) {
+        byte [] row = endRows[i];
+        if (row == null || row.length <= 0) continue;
+        Get get = new Get(row);
+        get.addColumn(FAMILY, QUALIFIER);
+        Result r;
+        int nbTry = 0;
+        do {
+          assertTrue(nbTry++ < 50);
+          Thread.sleep(100);
+          r = htable1.get(get);
+        } while (r == null || r.getValue(FAMILY, QUALIFIER) == null ||
+            Bytes.compareTo(VALUE2, r.getValue(FAMILY, QUALIFIER)) != 0);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
index 4091e58..b71e881 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHTableMultiplexerFlushCache.java
@@ -64,7 +64,7 @@ public class TestHTableMultiplexerFlushCache {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  private static void checkExistence(final HTable htable, final byte[] row, final byte[] family,
+  private static void checkExistence(final Table htable, final byte[] row, final byte[] family,
       final byte[] quality,
       final byte[] value) throws Exception {
     // verify that the Get returns the correct result
@@ -86,31 +86,33 @@ public class TestHTableMultiplexerFlushCache {
   public void testOnRegionChange() throws Exception {
     TableName TABLE = TableName.valueOf("testOnRegionChange");
     final int NUM_REGIONS = 10;
-    HTable htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
+    Table htable = TEST_UTIL.createTable(TABLE, new byte[][] { FAMILY }, 3,
       Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
 
     HTableMultiplexer multiplexer = new HTableMultiplexer(TEST_UTIL.getConfiguration(), 
       PER_REGIONSERVER_QUEUE_SIZE);
     
-    byte[][] startRows = htable.getStartKeys();
-    byte[] row = startRows[1];
-    assertTrue("2nd region should not start with empty row", row != null && row.length > 0);
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      byte[][] startRows = r.getStartKeys();
+      byte[] row = startRows[1];
+      assertTrue("2nd region should not start with empty row", row != null && row.length > 0);
 
-    Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1);
-    assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
-    
-    checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1);
+      Put put = new Put(row).add(FAMILY, QUALIFIER1, VALUE1);
+      assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
+
+      checkExistence(htable, row, FAMILY, QUALIFIER1, VALUE1);
 
-    // Now let's shutdown the regionserver and let regions moved to other servers.
-    HRegionLocation loc = htable.getRegionLocation(row);
-    MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster(); 
-    hbaseCluster.stopRegionServer(loc.getServerName());
-    TEST_UTIL.waitUntilAllRegionsAssigned(TABLE);
+      // Now let's shutdown the regionserver and let regions moved to other servers.
+      HRegionLocation loc = r.getRegionLocation(row);
+      MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
+      hbaseCluster.stopRegionServer(loc.getServerName());
+      TEST_UTIL.waitUntilAllRegionsAssigned(TABLE);
 
-    // put with multiplexer.
-    put = new Put(row).add(FAMILY, QUALIFIER2, VALUE2);
-    assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
+      // put with multiplexer.
+      put = new Put(row).addColumn(FAMILY, QUALIFIER2, VALUE2);
+      assertTrue("multiplexer.put returns", multiplexer.put(TABLE, put));
 
-    checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
+      checkExistence(htable, row, FAMILY, QUALIFIER2, VALUE2);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index b145109..0e5bd9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -157,19 +157,19 @@ public class TestMetaWithReplicas {
     byte[] data = ZKUtil.getData(zkw, primaryMetaZnode);
     ServerName primary = ServerName.parseFrom(data);
 
-    byte[] TABLE = Bytes.toBytes("testShutdownHandling");
+    TableName TABLE = TableName.valueOf("testShutdownHandling");
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     if (util.getHBaseAdmin().tableExists(TABLE)) {
       util.getHBaseAdmin().disableTable(TABLE);
       util.getHBaseAdmin().deleteTable(TABLE);
     }
-    Table htable = util.createTable(TABLE, FAMILIES, conf);
+    Table htable = util.createTable(TABLE, FAMILIES);
 
     util.getHBaseAdmin().flush(TableName.META_TABLE_NAME);
     Thread.sleep(conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
        30000) * 6);
     Connection c = ConnectionFactory.createConnection(util.getConfiguration());
-    List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(c, TableName.valueOf(TABLE));
+    List<HRegionInfo> regions = MetaTableAccessor.getTableRegions(c, TABLE);
     HRegionLocation hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0));
     // Ensure that the primary server for test table is not the same one as the primary
     // of the meta region since we will be killing the srv holding the meta's primary...
@@ -198,11 +198,11 @@ public class TestMetaWithReplicas {
     }
     ((ClusterConnection)c).clearRegionCache();
     htable.close();
-    htable = c.getTable(TableName.valueOf(TABLE));
+    htable = c.getTable(TABLE);
     byte[] row = "test".getBytes();
     Put put = new Put(row);
     put.add("foo".getBytes(), row, row);
-    BufferedMutator m = c.getBufferedMutator(TableName.valueOf(TABLE));
+    BufferedMutator m = c.getBufferedMutator(TABLE);
     m.mutate(put);
     m.flush();
     // Try to do a get of the row that was just put
@@ -217,22 +217,22 @@ public class TestMetaWithReplicas {
     ((ClusterConnection)c).clearRegionCache();
     htable.close();
     conf.setBoolean(HConstants.USE_META_REPLICAS, false);
-    htable = c.getTable(TableName.valueOf(TABLE));
+    htable = c.getTable(TABLE);
     r = htable.get(get);
     assertTrue(Arrays.equals(r.getRow(), row));
   }
 
   @Test
   public void testMetaLookupThreadPoolCreated() throws Exception {
-    byte[] TABLE = Bytes.toBytes("testMetaLookupThreadPoolCreated");
+    TableName TABLE = TableName.valueOf("testMetaLookupThreadPoolCreated");
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("foo") };
     if (TEST_UTIL.getHBaseAdmin().tableExists(TABLE)) {
       TEST_UTIL.getHBaseAdmin().disableTable(TABLE);
       TEST_UTIL.getHBaseAdmin().deleteTable(TABLE);
     }
-    Table htable = TEST_UTIL.createTable(TABLE, FAMILIES, TEST_UTIL.getConfiguration());
+    Table htable = TEST_UTIL.createTable(TABLE, FAMILIES);
     byte[] row = "test".getBytes();
-    ConnectionImplementation c = ((ConnectionImplementation)((HTable)htable).connection);
+    ConnectionImplementation c = ((ConnectionImplementation) TEST_UTIL.getConnection());
     // check that metalookup pool would get created
     c.relocateRegion(TABLE, row);
     ExecutorService ex = c.getCurrentMetaLookupPool();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
index 2958834..95faf1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
@@ -75,7 +75,7 @@ public class TestMultiParallel {
     //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
     UTIL.startMiniCluster(slaves);
-    HTable t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
+    Table t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
     UTIL.waitTableEnabled(TEST_TABLE);
     t.close();
     CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 91a8673..e6bde4e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -142,10 +142,10 @@ public class TestReplicaWithCluster {
     HTableDescriptor hdt = HTU.createTableDescriptor("testCreateDeleteTable");
     hdt.setRegionReplication(NB_SERVERS);
     hdt.addCoprocessor(SlowMeCopro.class.getName());
-    Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
+    Table table = HTU.createTable(hdt, new byte[][]{f}, null);
 
     Put p = new Put(row);
-    p.add(f, row, row);
+    p.addColumn(f, row, row);
     table.put(p);
 
     Get g = new Get(row);
@@ -174,11 +174,11 @@ public class TestReplicaWithCluster {
     HTableDescriptor hdt = HTU.createTableDescriptor("testChangeTable");
     hdt.setRegionReplication(NB_SERVERS);
     hdt.addCoprocessor(SlowMeCopro.class.getName());
-    Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
+    Table table = HTU.createTable(hdt, new byte[][]{f}, null);
 
     // basic test: it should work.
     Put p = new Put(row);
-    p.add(f, row, row);
+    p.addColumn(f, row, row);
     table.put(p);
 
     Get g = new Get(row);
@@ -314,7 +314,7 @@ public class TestReplicaWithCluster {
     HTableDescriptor hdt = HTU.createTableDescriptor("testBulkLoad");
     hdt.setRegionReplication(NB_SERVERS);
     hdt.addCoprocessor(SlowMeCopro.class.getName());
-    Table table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
+    Table table = HTU.createTable(hdt, new byte[][]{f}, null);
 
     // create hfiles to load.
     LOG.debug("Creating test data");

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index bfc1230..37e98e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -85,7 +85,7 @@ public class TestReplicasClient {
   }
 
   private static final int NB_SERVERS = 1;
-  private static HTable table = null;
+  private static Table table = null;
   private static final byte[] row = TestReplicasClient.class.getName().getBytes();
 
   private static HRegionInfo hriPrimary;
@@ -177,9 +177,11 @@ public class TestReplicasClient {
     // Create table then get the single region for our new table.
     HTableDescriptor hdt = HTU.createTableDescriptor(TestReplicasClient.class.getSimpleName());
     hdt.addCoprocessor(SlowMeCopro.class.getName());
-    table = HTU.createTable(hdt, new byte[][]{f}, HTU.getConfiguration());
+    table = HTU.createTable(hdt, new byte[][]{f}, null);
 
-    hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
+    try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
+      hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
+    }
 
     // mock a secondary region info to open
     hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),
@@ -547,8 +549,7 @@ public class TestReplicasClient {
 
       Thread.sleep(1000 + REFRESH_PERIOD * 2);
 
-      AsyncProcess ap = ((ClusterConnection) HTU.getHBaseAdmin().getConnection())
-          .getAsyncProcess();
+      AsyncProcess ap = ((ClusterConnection) HTU.getConnection()).getAsyncProcess();
 
       // Make primary slowdown
       SlowMeCopro.getCdl().set(new CountDownLatch(1));
@@ -563,8 +564,10 @@ public class TestReplicasClient {
       g.setConsistency(Consistency.TIMELINE);
       gets.add(g);
       Object[] results = new Object[2];
-      AsyncRequestFuture reqs = ap.submitAll(table.getPool(), table.getName(),
-          gets, null, results);
+
+      AsyncRequestFuture reqs = ap.submitAll(
+          HTable.getDefaultExecutor(HTU.getConfiguration()),
+          table.getName(), gets, null, results);
       reqs.waitUntilDone();
       // verify we got the right results back
       for (Object r : results) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
index 1f3a95b..24494dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
@@ -71,8 +71,7 @@ public class TestResultSizeEstimation {
 
     TableName TABLE = TableName.valueOf("testResultSizeEstimation");
     byte[][] FAMILIES = new byte[][] { FAMILY };
-    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
-    HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
+    Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
     Put p = new Put(ROW1);
     p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE));
     table.put(p);
@@ -102,8 +101,7 @@ public class TestResultSizeEstimation {
 
     TableName TABLE = TableName.valueOf("testResultSizeEstimationWithTags");
     byte[][] FAMILIES = new byte[][] { FAMILY };
-    Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
-    HTable table = TEST_UTIL.createTable(TABLE, FAMILIES, conf);
+    Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
     Put p = new Put(ROW1);
     p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
       new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 5afc226..54963ae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -297,9 +297,9 @@ public class TestScannersFromClientSide {
     }
 
     assertTrue("Expected row count: " + expectedRowCount + " Actual row count: " + rowCount,
-      expectedRowCount == rowCount);
+        expectedRowCount == rowCount);
     assertTrue("Expected cell count: " + expectedCellCount + " Actual cell count: " + cellCount,
-      expectedCellCount == cellCount);
+        expectedCellCount == cellCount);
     scanner.close();
   }
 
@@ -310,7 +310,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetMaxResults() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testGetMaxResults");
+    TableName TABLE = TableName.valueOf("testGetMaxResults");
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -430,7 +430,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testScanMaxResults() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testScanLimit");
+    TableName TABLE = TableName.valueOf("testScanLimit");
     byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
@@ -480,7 +480,7 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testGetRowOffset() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testGetRowOffset");
+    TableName TABLE = TableName.valueOf("testGetRowOffset");
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 20);
 
@@ -579,7 +579,7 @@ public class TestScannersFromClientSide {
     TableName TABLE = TableName.valueOf("testScanOnReopenedRegion");
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 2);
 
-    HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
+    Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
 
     Put put;
     Scan scan;
@@ -599,7 +599,11 @@ public class TestScannersFromClientSide {
     scan = new Scan(ROW);
     scanner = ht.getScanner(scan);
 
-    HRegionLocation loc = ht.getRegionLocation(ROW);
+    HRegionLocation loc;
+
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLE)) {
+      loc = locator.getRegionLocation(ROW);
+    }
     HRegionInfo hri = loc.getRegionInfo();
     MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
     byte[] regionName = hri.getRegionName();
@@ -649,12 +653,12 @@ public class TestScannersFromClientSide {
    */
   @Test
   public void testAsyncScanner() throws Exception {
-    byte [] TABLE = Bytes.toBytes("testAsyncScan");
+    TableName TABLE = TableName.valueOf("testAsyncScan");
     byte [][] ROWS = HTestConst.makeNAscii(ROW, 2);
     byte [][] FAMILIES = HTestConst.makeNAscii(FAMILY, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, 10);
 
-    HTable ht = TEST_UTIL.createTable(TABLE, FAMILIES);
+    Table ht = TEST_UTIL.createTable(TABLE, FAMILIES);
 
     Put put;
     Scan scan;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 095b3c6..e6656fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -182,7 +182,7 @@ public class TestSnapshotCloneIndependence {
     runTestSnapshotDeleteIndependent(true);
   }
 
-  private static void waitOnSplit(final HTable t, int originalCount) throws Exception {
+  private static void waitOnSplit(Connection c, final Table t, int originalCount) throws Exception {
     for (int i = 0; i < 200; i++) {
       try {
         Thread.sleep(50);
@@ -190,8 +190,10 @@ public class TestSnapshotCloneIndependence {
         // Restore the interrupted status
         Thread.currentThread().interrupt();
       }
-      if (t.getAllRegionLocations().size() > originalCount) {
-        return;
+      try (RegionLocator locator = c.getRegionLocator(t.getName())) {
+        if (locator.getAllRegionLocations().size() > originalCount) {
+          return;
+        }
       }
     }
     throw new Exception("Split did not increase the number of regions");
@@ -276,7 +278,7 @@ public class TestSnapshotCloneIndependence {
     final long startTime = System.currentTimeMillis();
     final TableName localTableName =
         TableName.valueOf(STRING_TABLE_NAME + startTime);
-    HTable original = UTIL.createTable(localTableName, TEST_FAM);
+    Table original = UTIL.createTable(localTableName, TEST_FAM);
     UTIL.loadTable(original, TEST_FAM);
     final int loadedTableCount = UTIL.countRows(original);
     System.out.println("Original table has: " + loadedTableCount + " rows");
@@ -298,7 +300,7 @@ public class TestSnapshotCloneIndependence {
     admin.cloneSnapshot(snapshotName, cloneTableName);
 
     // Verify that region information is the same pre-split
-    original.clearRegionCache();
+    ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
     List<HRegionInfo> originalTableHRegions = admin.getTableRegions(localTableName);
 
     final int originalRegionCount = originalTableHRegions.size();
@@ -309,7 +311,7 @@ public class TestSnapshotCloneIndependence {
 
     // Split a region on the parent table
     admin.splitRegion(originalTableHRegions.get(0).getRegionName());
-    waitOnSplit(original, originalRegionCount);
+    waitOnSplit(UTIL.getConnection(), original, originalRegionCount);
 
     // Verify that the cloned table region is not split
     final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
@@ -332,7 +334,7 @@ public class TestSnapshotCloneIndependence {
     final long startTime = System.currentTimeMillis();
     final TableName localTableName =
         TableName.valueOf(STRING_TABLE_NAME + startTime);
-    HTable original = UTIL.createTable(localTableName, TEST_FAM);
+    Table original = UTIL.createTable(localTableName, TEST_FAM);
     UTIL.loadTable(original, TEST_FAM);
 
     final String snapshotNameAsString = "snapshot_" + localTableName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index 079a588..287a9e1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -97,7 +97,7 @@ public class TestSnapshotFromClient {
   public void setup() throws Exception {
     HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
     htd.setRegionReplication(getNumReplicas());
-    UTIL.createTable(htd, new byte[][]{TEST_FAM}, UTIL.getConfiguration());
+    UTIL.createTable(htd, new byte[][]{TEST_FAM}, null);
   }
 
   protected int getNumReplicas() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
index 860f6e2..dfa0898 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestAggregateProtocol.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
 import org.apache.hadoop.hbase.client.coprocessor.LongColumnInterpreter;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -84,7 +85,7 @@ public class TestAggregateProtocol {
     util.startMiniCluster(2);
     final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1],
         ROWS[rowSeperator2] };
-    HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
+    Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
     /**
      * The testtable has one CQ which is always populated and one variable CQ
      * for each row rowkey1: CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
index ac75660..2cbd790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBigDecimalColumnInterpreter.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
 import org.apache.hadoop.hbase.client.coprocessor.BigDecimalColumnInterpreter;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -83,7 +84,7 @@ public class TestBigDecimalColumnInterpreter {
 
     util.startMiniCluster(2);
     final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] };
-    HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
+    Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
     /**
      * The testtable has one CQ which is always populated and one variable CQ for each row rowkey1:
      * CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2
@@ -92,11 +93,11 @@ public class TestBigDecimalColumnInterpreter {
       Put put = new Put(ROWS[i]);
       put.setDurability(Durability.SKIP_WAL);
       BigDecimal bd = new BigDecimal(i);
-      put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(bd));
+      put.addColumn(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(bd));
       table.put(put);
       Put p2 = new Put(ROWS[i]);
       put.setDurability(Durability.SKIP_WAL);
-      p2.add(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(bd)),
+      p2.addColumn(TEST_FAMILY, Bytes.add(TEST_MULTI_CQ, Bytes.toBytes(bd)),
         Bytes.toBytes(bd.multiply(new BigDecimal("0.10"))));
       table.put(p2);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
index 1ca7676..8301264 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
@@ -26,11 +26,14 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -173,9 +176,12 @@ public class TestCoprocessorEndpoint {
 
   @Test
   public void testCoprocessorService() throws Throwable {
-    HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
-    NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
+    Table table = util.getConnection().getTable(TEST_TABLE);
 
+    List<HRegionLocation> regions;
+    try(RegionLocator rl = util.getConnection().getRegionLocator(TEST_TABLE)) {
+      regions = rl.getAllRegionLocations();
+    }
     final TestProtos.EchoRequestProto request =
         TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build();
     final Map<byte[], String> results = Collections.synchronizedMap(
@@ -208,9 +214,9 @@ public class TestCoprocessorEndpoint {
         LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey()));
       }
       assertEquals(3, results.size());
-      for (HRegionInfo info : regions.navigableKeySet()) {
-        LOG.info("Region info is "+info.getRegionNameAsString());
-        assertTrue(results.containsKey(info.getRegionName()));
+      for (HRegionLocation info : regions) {
+        LOG.info("Region info is "+info.getRegionInfo().getRegionNameAsString());
+        assertTrue(results.containsKey(info.getRegionInfo().getRegionName()));
       }
       results.clear();
 
@@ -247,8 +253,11 @@ public class TestCoprocessorEndpoint {
 
   @Test
   public void testCoprocessorServiceNullResponse() throws Throwable {
-    HTable table = (HTable) util.getConnection().getTable(TEST_TABLE);
-    NavigableMap<HRegionInfo,ServerName> regions = table.getRegionLocations();
+    Table table = util.getConnection().getTable(TEST_TABLE);
+    List<HRegionLocation> regions;
+    try(RegionLocator rl = util.getConnection().getRegionLocator(TEST_TABLE)) {
+      regions = rl.getAllRegionLocations();
+    }
 
     final TestProtos.EchoRequestProto request =
         TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build();
@@ -273,7 +282,8 @@ public class TestCoprocessorEndpoint {
         LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey()));
       }
       assertEquals(3, results.size());
-      for (HRegionInfo info : regions.navigableKeySet()) {
+      for (HRegionLocation region : regions) {
+        HRegionInfo info = region.getRegionInfo();
         LOG.info("Region info is "+info.getRegionNameAsString());
         assertTrue(results.containsKey(info.getRegionName()));
         assertNull(results.get(info.getRegionName()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestDoubleColumnInterpreter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestDoubleColumnInterpreter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestDoubleColumnInterpreter.java
index baea95d..a1e67fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestDoubleColumnInterpreter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestDoubleColumnInterpreter.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.AggregationClient;
 import org.apache.hadoop.hbase.client.coprocessor.DoubleColumnInterpreter;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -81,7 +82,7 @@ public class TestDoubleColumnInterpreter {
 
     util.startMiniCluster(2);
     final byte[][] SPLIT_KEYS = new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] };
-    HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
+    Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
     /**
      * The testtable has one CQ which is always populated and one variable CQ for each row rowkey1:
      * CF:CQ CF:CQ1 rowKey2: CF:CQ CF:CQ2

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 1f95a53..dbba63b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -45,6 +46,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@@ -1612,15 +1615,15 @@ public class TestMasterObserver {
     cp.enableBypass(false);
     cp.resetStates();
 
-    HTable table = UTIL.createMultiRegionTable(tableName, TEST_FAMILY);
+    Table table = UTIL.createMultiRegionTable(tableName, TEST_FAMILY);
 
-    try {
+    try (RegionLocator r = UTIL.getConnection().getRegionLocator(tableName)) {
       UTIL.waitUntilAllRegionsAssigned(tableName);
 
-      NavigableMap<HRegionInfo, ServerName> regions = table.getRegionLocations();
-      Map.Entry<HRegionInfo, ServerName> firstGoodPair = null;
-      for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
-        if (e.getValue() != null) {
+      List<HRegionLocation> regions = r.getAllRegionLocations();
+      HRegionLocation firstGoodPair = null;
+      for (HRegionLocation e: regions) {
+        if (e.getServerName() != null) {
           firstGoodPair = e;
           break;
         }
@@ -1630,7 +1633,7 @@ public class TestMasterObserver {
       // Try to force a move
       Collection<ServerName> servers = master.getClusterStatus().getServers();
       String destName = null;
-      String serverNameForFirstRegion = firstGoodPair.getValue().toString();
+      String serverNameForFirstRegion = firstGoodPair.getServerName().toString();
       LOG.info("serverNameForFirstRegion=" + serverNameForFirstRegion);
       ServerName masterServerName = master.getServerName();
       boolean found = false;
@@ -1647,7 +1650,7 @@ public class TestMasterObserver {
       assertTrue("Found server", found);
       LOG.info("Found " + destName);
       master.getMasterRpcServices().moveRegion(null, RequestConverter.buildMoveRegionRequest(
-          firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
+          firstGoodPair.getRegionInfo().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
       assertTrue("Coprocessor should have been called on region move",
         cp.wasMoveCalled());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index d0e561f..c710d3c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RowMutations;
@@ -140,9 +141,8 @@ public class TestRegionObserverInterface {
           true, true, true, false, true, true, true });
 
       verifyMethodResult(SimpleRegionObserver.class,
-          new String[] {"getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose"},
-          tableName,
-          new Integer[] {1, 1, 0, 0});
+          new String[] { "getCtPreOpen", "getCtPostOpen", "getCtPreClose", "getCtPostClose" },
+          tableName, new Integer[] { 1, 1, 0, 0 });
 
       Get get = new Get(ROW);
       get.addColumn(A, A);
@@ -151,11 +151,9 @@ public class TestRegionObserverInterface {
       table.get(get);
 
       verifyMethodResult(SimpleRegionObserver.class,
-          new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
-      "hadDelete", "hadPrePreparedDeleteTS"},
-      tableName,
-      new Boolean[] {true, true, true, true, false, false}
-          );
+          new String[] { "hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut", "hadDelete",
+              "hadPrePreparedDeleteTS" }, tableName,
+          new Boolean[] { true, true, true, true, false, false });
 
       Delete delete = new Delete(ROW);
       delete.deleteColumn(A, A);
@@ -182,7 +180,7 @@ public class TestRegionObserverInterface {
   @Test (timeout=300000)
   public void testRowMutation() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
-    Table table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] { A, B, C });
     try {
       verifyMethodResult(SimpleRegionObserver.class,
         new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
@@ -219,7 +217,7 @@ public class TestRegionObserverInterface {
   @Test (timeout=300000)
   public void testIncrementHook() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
-    Table table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] { A, B, C });
     try {
       Increment inc = new Increment(Bytes.toBytes(0));
       inc.addColumn(A, A, 1);
@@ -254,11 +252,8 @@ public class TestRegionObserverInterface {
       p = new Put(Bytes.toBytes(0));
       p.add(A, A, A);
       verifyMethodResult(SimpleRegionObserver.class,
-          new String[] {"hadPreCheckAndPut",
-              "hadPreCheckAndPutAfterRowLock", "hadPostCheckAndPut"},
-          tableName,
-          new Boolean[] {false, false, false}
-          );
+          new String[] { "hadPreCheckAndPut", "hadPreCheckAndPutAfterRowLock",
+              "hadPostCheckAndPut" }, tableName, new Boolean[] { false, false, false });
       table.checkAndPut(Bytes.toBytes(0), A, A, A, p);
       verifyMethodResult(SimpleRegionObserver.class,
           new String[] {"hadPreCheckAndPut",
@@ -304,7 +299,7 @@ public class TestRegionObserverInterface {
   @Test (timeout=300000)
   public void testAppendHook() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
-    Table table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] { A, B, C });
     try {
       Append app = new Append(Bytes.toBytes(0));
       app.add(A, A, A);
@@ -337,11 +332,8 @@ public class TestRegionObserverInterface {
     util.waitUntilAllRegionsAssigned(tableName);
 
     verifyMethodResult(SimpleRegionObserver.class,
-        new String[] {"hadPreGet", "hadPostGet", "wasScannerNextCalled",
-            "wasScannerCloseCalled"},
-        tableName,
-        new Boolean[] {false, false, false, false}
-    );
+        new String[] { "hadPreGet", "hadPostGet", "wasScannerNextCalled", "wasScannerCloseCalled" },
+        tableName, new Boolean[] { false, false, false, false });
 
     Table table = util.getConnection().getTable(tableName);
     Put put = new Put(ROW);
@@ -562,8 +554,8 @@ public class TestRegionObserverInterface {
     String testName = TestRegionObserverInterface.class.getName()+".bulkLoadHFileTest";
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".bulkLoadHFileTest");
     Configuration conf = util.getConfiguration();
-    HTable table = util.createTable(tableName, new byte[][] {A, B, C});
-    try {
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
+    try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
       verifyMethodResult(SimpleRegionObserver.class,
           new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
           tableName,
@@ -574,10 +566,10 @@ public class TestRegionObserverInterface {
       final Path dir = util.getDataTestDirOnTestFS(testName).makeQualified(fs);
       Path familyDir = new Path(dir, Bytes.toString(A));
 
-      createHFile(util.getConfiguration(), fs, new Path(familyDir,Bytes.toString(A)), A, A);
+      createHFile(util.getConfiguration(), fs, new Path(familyDir, Bytes.toString(A)), A, A);
 
       // Bulk load
-      new LoadIncrementalHFiles(conf).doBulkLoad(dir, table);
+      new LoadIncrementalHFiles(conf).doBulkLoad(dir, util.getHBaseAdmin(), table, locator);
 
       verifyMethodResult(SimpleRegionObserver.class,
           new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},
@@ -595,21 +587,22 @@ public class TestRegionObserverInterface {
   public void testRecovery() throws Exception {
     LOG.info(TestRegionObserverInterface.class.getName() +".testRecovery");
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRecovery");
-    HTable table = util.createTable(tableName, new byte[][] {A, B, C});
-    try {
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
+    try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
+
       JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
       ServerName sn2 = rs1.getRegionServer().getServerName();
-      String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
+      String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
 
       util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
-      while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
+      while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())){
         Thread.sleep(100);
       }
 
       Put put = new Put(ROW);
-      put.add(A, A, A);
-      put.add(B, B, B);
-      put.add(C, C, C);
+      put.addColumn(A, A, A);
+      put.addColumn(B, B, B);
+      put.addColumn(C, C, C);
       table.put(put);
 
       verifyMethodResult(SimpleRegionObserver.class,
@@ -646,46 +639,48 @@ public class TestRegionObserverInterface {
   public void testLegacyRecovery() throws Exception {
     LOG.info(TestRegionObserverInterface.class.getName() +".testLegacyRecovery");
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testLegacyRecovery");
-    HTable table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
     try {
-      JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
-      ServerName sn2 = rs1.getRegionServer().getServerName();
-      String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
+      try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
+        JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
+        ServerName sn2 = rs1.getRegionServer().getServerName();
+        String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
+
+        util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
+        while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
+          Thread.sleep(100);
+        }
 
-      util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
-      while (!sn2.equals(table.getRegionLocations().firstEntry().getValue() )){
-        Thread.sleep(100);
+        Put put = new Put(ROW);
+        put.add(A, A, A);
+        put.add(B, B, B);
+        put.add(C, C, C);
+        table.put(put);
+
+        verifyMethodResult(SimpleRegionObserver.Legacy.class,
+            new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
+                "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
+            tableName,
+            new Boolean[] {false, false, true, true, true, true, false}
+        );
+
+        verifyMethodResult(SimpleRegionObserver.Legacy.class,
+            new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut",
+                "getCtPostPut", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
+            tableName,
+            new Integer[] {0, 0, 1, 1, 0, 0});
+
+        cluster.killRegionServer(rs1.getRegionServer().getServerName());
+        Threads.sleep(1000); // Let the kill soak in.
+        util.waitUntilAllRegionsAssigned(tableName);
+        LOG.info("All regions assigned");
+
+        verifyMethodResult(SimpleRegionObserver.Legacy.class,
+            new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut",
+                "getCtPostPut", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
+            tableName,
+            new Integer[]{1, 1, 0, 0, 1, 1});
       }
-
-      Put put = new Put(ROW);
-      put.add(A, A, A);
-      put.add(B, B, B);
-      put.add(C, C, C);
-      table.put(put);
-
-      verifyMethodResult(SimpleRegionObserver.Legacy.class,
-          new String[] {"hadPreGet", "hadPostGet", "hadPrePut", "hadPostPut",
-        "hadPreBatchMutate", "hadPostBatchMutate", "hadDelete"},
-        tableName,
-        new Boolean[] {false, false, true, true, true, true, false}
-          );
-
-      verifyMethodResult(SimpleRegionObserver.Legacy.class,
-          new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
-              "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
-          tableName,
-          new Integer[] {0, 0, 1, 1, 0, 0});
-
-      cluster.killRegionServer(rs1.getRegionServer().getServerName());
-      Threads.sleep(1000); // Let the kill soak in.
-      util.waitUntilAllRegionsAssigned(tableName);
-      LOG.info("All regions assigned");
-
-      verifyMethodResult(SimpleRegionObserver.Legacy.class,
-          new String[] {"getCtPreWALRestore", "getCtPostWALRestore", "getCtPrePut", "getCtPostPut",
-              "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},
-          tableName,
-          new Integer[]{1, 1, 0, 0, 1, 1});
     } finally {
       util.deleteTable(tableName);
       table.close();
@@ -696,27 +691,28 @@ public class TestRegionObserverInterface {
   public void testPreWALRestoreSkip() throws Exception {
     LOG.info(TestRegionObserverInterface.class.getName() + ".testPreWALRestoreSkip");
     TableName tableName = TableName.valueOf(SimpleRegionObserver.TABLE_SKIPPED);
-    HTable table = util.createTable(tableName, new byte[][] { A, B, C });
+    Table table = util.createTable(tableName, new byte[][] { A, B, C });
 
-    JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
-    ServerName sn2 = rs1.getRegionServer().getServerName();
-    String regEN = table.getRegionLocations().firstEntry().getKey().getEncodedName();
+    try (RegionLocator locator = util.getConnection().getRegionLocator(tableName)) {
+      JVMClusterUtil.RegionServerThread rs1 = cluster.startRegionServer();
+      ServerName sn2 = rs1.getRegionServer().getServerName();
+      String regEN = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
 
-    util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
-    while (!sn2.equals(table.getRegionLocations().firstEntry().getValue())) {
-      Thread.sleep(100);
-    }
+      util.getHBaseAdmin().move(regEN.getBytes(), sn2.getServerName().getBytes());
+      while (!sn2.equals(locator.getAllRegionLocations().get(0).getServerName())) {
+        Thread.sleep(100);
+      }
 
-    Put put = new Put(ROW);
-    put.add(A, A, A);
-    put.add(B, B, B);
-    put.add(C, C, C);
-    table.put(put);
-    table.flushCommits();
+      Put put = new Put(ROW);
+      put.add(A, A, A);
+      put.add(B, B, B);
+      put.add(C, C, C);
+      table.put(put);
 
-    cluster.killRegionServer(rs1.getRegionServer().getServerName());
-    Threads.sleep(20000); // just to be sure that the kill has fully started.
-    util.waitUntilAllRegionsAssigned(tableName);
+      cluster.killRegionServer(rs1.getRegionServer().getServerName());
+      Threads.sleep(20000); // just to be sure that the kill has fully started.
+      util.waitUntilAllRegionsAssigned(tableName);
+    }
 
     verifyMethodResult(SimpleRegionObserver.class, new String[] { "getCtPreWALRestore",
         "getCtPostWALRestore", "getCtPreWALRestoreDeprecated", "getCtPostWALRestoreDeprecated"},

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
index 3ac2cc6..ef62564 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -98,7 +99,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
       // hosts the region we attempted to write to) to abort.
       final byte[] TEST_FAMILY = Bytes.toBytes("aaa");
 
-      HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, TEST_FAMILY);
+      Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, TEST_FAMILY);
       TEST_UTIL.waitUntilAllRegionsAssigned(TABLE_NAME);
 
       // Note which regionServer will abort (after put is attempted).
@@ -109,7 +110,6 @@ public class TestRegionServerCoprocessorExceptionWithAbort {
         Put put = new Put(ROW);
         put.add(TEST_FAMILY, ROW, ROW);
         table.put(put);
-        table.flushCommits();
       } catch (IOException e) {
         // The region server is going to be aborted.
         // We may get an exception if we retry,

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
index ba18e41..ce76fea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithRemove.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -97,7 +98,7 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
     TableName TEST_TABLE = TableName.valueOf("observed_table");
     byte[] TEST_FAMILY = Bytes.toBytes("aaa");
 
-    HTable table = TEST_UTIL.createMultiRegionTable(TEST_TABLE, TEST_FAMILY);
+    Table table = TEST_UTIL.createMultiRegionTable(TEST_TABLE, TEST_FAMILY);
     TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
     // Note which regionServer that should survive the buggy coprocessor's
     // prePut().
@@ -108,12 +109,10 @@ public class TestRegionServerCoprocessorExceptionWithRemove {
     try {
       final byte[] ROW = Bytes.toBytes("aaa");
       Put put = new Put(ROW);
-      put.add(TEST_FAMILY, ROW, ROW);
+      put.addColumn(TEST_FAMILY, ROW, ROW);
       table.put(put);
-      table.flushCommits();
       // We may need two puts to reliably get an exception
       table.put(put);
-      table.flushCommits();
     } catch (IOException e) {
       threwIOE = true;
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
index 9f62335..cf131f8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestMultiRowRangeFilter.java
@@ -29,11 +29,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -51,7 +53,7 @@ public class TestMultiRowRangeFilter {
   private byte[] family = Bytes.toBytes("family");
   private byte[] qf = Bytes.toBytes("qf");
   private byte[] value = Bytes.toBytes("val");
-  private byte[] tableName;
+  private TableName tableName;
   private int numRows = 100;
 
   /**
@@ -218,8 +220,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithRangeOverlap() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithRangeOverlap");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithRangeOverlap");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -246,8 +248,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithoutRangeOverlap() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithoutRangeOverlap");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithoutRangeOverlap");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -273,8 +275,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithEmptyStartRow() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStartRow");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithEmptyStartRow");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
     Scan scan = new Scan();
     scan.setMaxVersions();
@@ -295,8 +297,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithEmptyStopRow() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithEmptyStopRow");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithEmptyStopRow");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
     Scan scan = new Scan();
     scan.setMaxVersions();
@@ -316,8 +318,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithInclusive() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithInclusive");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithInclusive");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -344,8 +346,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeFilterWithExclusive() throws IOException {
-    tableName = Bytes.toBytes("testMultiRowRangeFilterWithExclusive");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("testMultiRowRangeFilterWithExclusive");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -370,8 +372,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeWithFilterListAndOperator() throws IOException {
-    tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListAndOperator");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("TestMultiRowRangeFilterWithFilterListAndOperator");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -405,8 +407,8 @@ public class TestMultiRowRangeFilter {
 
   @Test
   public void testMultiRowRangeWithFilterListOrOperator() throws IOException {
-    tableName = Bytes.toBytes("TestMultiRowRangeFilterWithFilterListOrOperator");
-    HTable ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    tableName = TableName.valueOf("TestMultiRowRangeFilterWithFilterListOrOperator");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
     generateRows(numRows, ht, family, qf, value);
 
     Scan scan = new Scan();
@@ -440,18 +442,18 @@ public class TestMultiRowRangeFilter {
     ht.close();
   }
 
-  private void generateRows(int numberOfRows, HTable ht, byte[] family, byte[] qf, byte[] value)
+  private void generateRows(int numberOfRows, Table ht, byte[] family, byte[] qf, byte[] value)
       throws IOException {
     for (int i = 0; i < numberOfRows; i++) {
       byte[] row = Bytes.toBytes(i);
       Put p = new Put(row);
-      p.add(family, qf, value);
+      p.addColumn(family, qf, value);
       ht.put(p);
     }
     TEST_UTIL.flush();
   }
 
-  private List<Cell> getScanResult(byte[] startRow, byte[] stopRow, HTable ht) throws IOException {
+  private List<Cell> getScanResult(byte[] startRow, byte[] stopRow, Table ht) throws IOException {
     Scan scan = new Scan();
     scan.setMaxVersions();
     if(!Bytes.toString(startRow).isEmpty()) {
@@ -471,7 +473,7 @@ public class TestMultiRowRangeFilter {
     return kvList;
   }
 
-  private int getResultsSize(HTable ht, Scan scan) throws IOException {
+  private int getResultsSize(Table ht, Scan scan) throws IOException {
     ResultScanner scanner = ht.getScanner(scan);
     List<Cell> results = new ArrayList<Cell>();
     Result r;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
index c648a8b..a7cb4a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestLoadAndSwitchEncodeOnDisk.java
@@ -19,11 +19,15 @@ package org.apache.hadoop.hbase.io.encoding;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.ServerName;
@@ -75,7 +79,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
 
     HColumnDescriptor hcd = getColumnDesc(admin);
     System.err.println("\nDisabling encode-on-disk. Old column descriptor: " + hcd + "\n");
-    HTable t = (HTable) TEST_UTIL.getConnection().getTable(TABLE);
+    Table t = TEST_UTIL.getConnection().getTable(TABLE);
     assertAllOnLine(t);
 
     admin.disableTable(TABLE);
@@ -92,7 +96,7 @@ public class TestLoadAndSwitchEncodeOnDisk extends
     assertAllOnLine(t);
 
     System.err.println("\nCompacting the table\n");
-    admin.majorCompact(TABLE.getName());
+    admin.majorCompact(TABLE);
     // Wait until compaction completes
     Threads.sleepWithoutInterrupt(5000);
     HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
@@ -103,10 +107,13 @@ public class TestLoadAndSwitchEncodeOnDisk extends
     System.err.println("\nDone with the test, shutting down the cluster\n");
   }
 
-  private void assertAllOnLine(final HTable t) throws IOException {
-    NavigableMap<HRegionInfo, ServerName> regions = t.getRegionLocations();
-    for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
-      byte [] startkey = e.getKey().getStartKey();
+  private void assertAllOnLine(final Table t) throws IOException {
+    List<HRegionLocation> regions;
+    try(RegionLocator rl = TEST_UTIL.getConnection().getRegionLocator(t.getName())) {
+      regions = rl.getAllRegionLocations();
+    }
+    for (HRegionLocation e: regions) {
+      byte [] startkey = e.getRegionInfo().getStartKey();
       Scan s = new Scan(startkey);
       ResultScanner scanner = t.getScanner(s);
       Result r = scanner.next();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
index 6129b26..e77425a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatTestBase.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.NullWritable;
@@ -77,7 +78,7 @@ public abstract class MultiTableInputFormatTestBase {
     TEST_UTIL.startMiniCluster(3);
     // create and fill table
     for (String tableName : TABLES) {
-      try (HTable table =
+      try (Table table =
           TEST_UTIL.createMultiRegionTable(TableName.valueOf(tableName),
             INPUT_FAMILY, 4)) {
         TEST_UTIL.loadTable(table, INPUT_FAMILY, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java
index 6b23e37..2d3d6ef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCellCounter.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -80,38 +81,37 @@ public class TestCellCounter {
    */
   @Test (timeout=300000)
   public void testCellCounter() throws Exception {
-    String sourceTable = "sourceTable";
+    TableName sourceTable = TableName.valueOf("sourceTable");
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(sourceTable, families);
     try{
-    Put p = new Put(ROW1);
-    p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
-    p.add(FAMILY_B, QUALIFIER, now + 1, Bytes.toBytes("Data12"));
-    p.add(FAMILY_A, QUALIFIER, now + 2, Bytes.toBytes("Data13"));
-    t.put(p);
-    p = new Put(ROW2);
-    p.add(FAMILY_B, QUALIFIER, now, Bytes.toBytes("Dat21"));
-    p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
-    p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
-    t.put(p);
-    String[] args = { sourceTable, FQ_OUTPUT_DIR.toString(), ";", "^row1" };
-    runCount(args);
-    FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
-        "part-r-00000");
-    String data = IOUtils.toString(inputStream);
-    inputStream.close();
-    assertTrue(data.contains("Total Families Across all Rows" + "\t" + "2"));
-    assertTrue(data.contains("Total Qualifiers across all Rows" + "\t" + "2"));
-    assertTrue(data.contains("Total ROWS" + "\t" + "1"));
-    assertTrue(data.contains("b;q" + "\t" + "1"));
-    assertTrue(data.contains("a;q" + "\t" + "1"));
-    assertTrue(data.contains("row1;a;q_Versions" + "\t" + "1"));
-    assertTrue(data.contains("row1;b;q_Versions" + "\t" + "1"));
+      Put p = new Put(ROW1);
+      p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
+      p.add(FAMILY_B, QUALIFIER, now + 1, Bytes.toBytes("Data12"));
+      p.add(FAMILY_A, QUALIFIER, now + 2, Bytes.toBytes("Data13"));
+      t.put(p);
+      p = new Put(ROW2);
+      p.add(FAMILY_B, QUALIFIER, now, Bytes.toBytes("Dat21"));
+      p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
+      p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
+      t.put(p);
+      String[] args = { sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(), ";", "^row1" };
+      runCount(args);
+      FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
+          "part-r-00000");
+      String data = IOUtils.toString(inputStream);
+      inputStream.close();
+      assertTrue(data.contains("Total Families Across all Rows" + "\t" + "2"));
+      assertTrue(data.contains("Total Qualifiers across all Rows" + "\t" + "2"));
+      assertTrue(data.contains("Total ROWS" + "\t" + "1"));
+      assertTrue(data.contains("b;q" + "\t" + "1"));
+      assertTrue(data.contains("a;q" + "\t" + "1"));
+      assertTrue(data.contains("row1;a;q_Versions" + "\t" + "1"));
+      assertTrue(data.contains("row1;b;q_Versions" + "\t" + "1"));
     }finally{
       t.close();
       FileUtil.fullyDelete(new File(OUTPUT_DIR));
     }
-
   }
 
   /**
@@ -119,9 +119,9 @@ public class TestCellCounter {
    */
   @Test (timeout=300000)
   public void testCellCounterStartTimeRange() throws Exception {
-    String sourceTable = "testCellCounterStartTimeRange";
+    TableName sourceTable = TableName.valueOf("testCellCounterStartTimeRange");
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(sourceTable, families);
     try{
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@@ -134,7 +134,7 @@ public class TestCellCounter {
     p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
     t.put(p);
     String[] args = {
-      sourceTable, FQ_OUTPUT_DIR.toString(),  ";", "^row1", "--starttime=" + now,
+      sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(),  ";", "^row1", "--starttime=" + now,
       "--endtime=" + now + 2 };
     runCount(args);
     FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
@@ -159,9 +159,9 @@ public class TestCellCounter {
    */
   @Test (timeout=300000)
   public void testCellCounteEndTimeRange() throws Exception {
-    String sourceTable = "testCellCounterEndTimeRange";
+    TableName sourceTable = TableName.valueOf("testCellCounterEndTimeRange");
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(sourceTable, families);
     try{
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@@ -174,7 +174,8 @@ public class TestCellCounter {
     p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
     t.put(p);
     String[] args = {
-      sourceTable, FQ_OUTPUT_DIR.toString(),  ";", "^row1", "--endtime=" + now + 1 };
+      sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(),  ";", "^row1",
+        "--endtime=" + now + 1 };
     runCount(args);
     FileInputStream inputStream = new FileInputStream(OUTPUT_DIR + File.separator +
         "part-r-00000");
@@ -198,9 +199,9 @@ public class TestCellCounter {
    */
   @Test (timeout=300000)
   public void testCellCounteOutOfTimeRange() throws Exception {
-    String sourceTable = "testCellCounterOutTimeRange";
+    TableName sourceTable = TableName.valueOf("testCellCounterOutTimeRange");
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(sourceTable, families);
     try{
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@@ -213,7 +214,7 @@ public class TestCellCounter {
     p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
     t.put(p);
     String[] args = {
-      sourceTable, FQ_OUTPUT_DIR.toString(),  ";", "--starttime=" + now + 1,
+      sourceTable.getNameAsString(), FQ_OUTPUT_DIR.toString(),  ";", "--starttime=" + now + 1,
       "--endtime=" + now + 2 };
 
     runCount(args);
@@ -275,14 +276,14 @@ public class TestCellCounter {
    */
   @Test(timeout = 300000)
   public void testCellCounterForCompleteTable() throws Exception {
-    String sourceTable = "testCellCounterForCompleteTable";
+    TableName sourceTable = TableName.valueOf("testCellCounterForCompleteTable");
     String outputPath = OUTPUT_DIR + sourceTable;
     LocalFileSystem localFileSystem = new LocalFileSystem();
     Path outputDir =
         new Path(outputPath).makeQualified(localFileSystem.getUri(),
           localFileSystem.getWorkingDirectory());
     byte[][] families = { FAMILY_A, FAMILY_B };
-    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(sourceTable, families);
     try {
       Put p = new Put(ROW1);
       p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));
@@ -294,7 +295,7 @@ public class TestCellCounter {
       p.add(FAMILY_A, QUALIFIER, now + 1, Bytes.toBytes("Data22"));
       p.add(FAMILY_B, QUALIFIER, now + 2, Bytes.toBytes("Data23"));
       t.put(p);
-      String[] args = { sourceTable, outputDir.toString(), ";" };
+      String[] args = { sourceTable.getNameAsString(), outputDir.toString(), ";" };
       runCount(args);
       FileInputStream inputStream =
           new FileInputStream(outputPath + File.separator + "part-r-00000");

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
index c96d7c4..118395b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestCopyTable.java
@@ -189,13 +189,13 @@ public class TestCopyTable {
    */
   @Test
   public void testRenameFamily() throws Exception {
-    String sourceTable = "sourceTable";
-    String targetTable = "targetTable";
+    TableName sourceTable = TableName.valueOf("sourceTable");
+    TableName targetTable = TableName.valueOf("targetTable");
 
     byte[][] families = { FAMILY_A, FAMILY_B };
 
-    Table t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families);
-    Table t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families);
+    Table t = TEST_UTIL.createTable(sourceTable, families);
+    Table t2 = TEST_UTIL.createTable(targetTable, families);
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER,  Bytes.toBytes("Data11"));
     p.add(FAMILY_B, QUALIFIER,  Bytes.toBytes("Data12"));
@@ -210,7 +210,7 @@ public class TestCopyTable {
     long currentTime = System.currentTimeMillis();
     String[] args = new String[] { "--new.name=" + targetTable, "--families=a:b", "--all.cells",
         "--starttime=" + (currentTime - 100000), "--endtime=" + (currentTime + 100000),
-        "--versions=1", sourceTable };
+        "--versions=1", sourceTable.getNameAsString() };
     assertNull(t2.get(new Get(ROW1)).getRow());
 
     assertTrue(runCopy(args));


[6/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Posted by st...@apache.org.
HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b5b58530
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b5b58530
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b5b58530

Branch: refs/heads/master
Commit: b5b5853043e20a719f1b7afb485ede68ddd823f0
Parents: e6ed792
Author: stack <st...@apache.org>
Authored: Thu Jun 25 14:42:33 2015 -0700
Committer: stack <st...@apache.org>
Committed: Thu Jun 25 14:42:33 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Action.java  |   2 +-
 .../hbase/client/BufferedMutatorImpl.java       |   2 +-
 .../hadoop/hbase/client/ScannerCallable.java    |   2 +-
 .../hadoop/hbase/client/coprocessor/Batch.java  |  14 +-
 .../hbase/ipc/RegionCoprocessorRpcChannel.java  |   4 +-
 .../hadoop/hbase/client/TestOperation.java      |   8 +-
 .../mapreduce/IntegrationTestBulkLoad.java      |   2 +-
 ...tionTestWithCellVisibilityLoadAndVerify.java |   2 +-
 .../hadoop/hbase/rest/RowResourceBase.java      |  14 +-
 .../hadoop/hbase/rest/TestStatusResource.java   |   4 +-
 .../hbase/mapred/TableInputFormatBase.java      |   2 +-
 .../hbase/mapreduce/LoadIncrementalHFiles.java  |  10 +-
 .../hbase/mapreduce/TableInputFormatBase.java   |   4 +-
 .../hadoop/hbase/util/RegionSizeCalculator.java |   6 +-
 .../org/apache/hadoop/hbase/HBaseTestCase.java  |   2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       | 166 ++++-
 .../hadoop/hbase/TestFullLogReconstruction.java |   3 +-
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |   7 +-
 .../hbase/TestHBaseOnOtherDfsCluster.java       |   8 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |   6 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  | 517 +++++++-------
 .../apache/hadoop/hbase/client/TestAdmin2.java  |  22 +-
 .../hadoop/hbase/client/TestCheckAndMutate.java |   8 +-
 .../hadoop/hbase/client/TestClientPushback.java |  26 +-
 .../hadoop/hbase/client/TestFromClientSide.java | 705 ++++++++++---------
 .../hbase/client/TestFromClientSide3.java       | 283 ++++----
 .../hbase/client/TestFromClientSideNoCodec.java |  16 +-
 .../org/apache/hadoop/hbase/client/TestHCM.java | 124 ++--
 .../hbase/client/TestHTableMultiplexer.java     | 100 +--
 .../client/TestHTableMultiplexerFlushCache.java |  38 +-
 .../hbase/client/TestMetaWithReplicas.java      |  18 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |   2 +-
 .../hbase/client/TestReplicaWithCluster.java    |  10 +-
 .../hadoop/hbase/client/TestReplicasClient.java |  17 +-
 .../hbase/client/TestResultSizeEstimation.java  |   6 +-
 .../client/TestScannersFromClientSide.java      |  22 +-
 .../client/TestSnapshotCloneIndependence.java   |  16 +-
 .../hbase/client/TestSnapshotFromClient.java    |   2 +-
 .../coprocessor/TestAggregateProtocol.java      |   3 +-
 .../TestBigDecimalColumnInterpreter.java        |   7 +-
 .../coprocessor/TestCoprocessorEndpoint.java    |  26 +-
 .../TestDoubleColumnInterpreter.java            |   3 +-
 .../hbase/coprocessor/TestMasterObserver.java   |  19 +-
 .../TestRegionObserverInterface.java            | 168 +++--
 ...gionServerCoprocessorExceptionWithAbort.java |   4 +-
 ...ionServerCoprocessorExceptionWithRemove.java |   7 +-
 .../hbase/filter/TestMultiRowRangeFilter.java   |  44 +-
 .../encoding/TestLoadAndSwitchEncodeOnDisk.java |  19 +-
 .../MultiTableInputFormatTestBase.java          |   3 +-
 .../hadoop/hbase/mapreduce/TestCellCounter.java |  77 +-
 .../hadoop/hbase/mapreduce/TestCopyTable.java   |  10 +-
 .../hbase/mapreduce/TestHFileOutputFormat.java  | 119 ++--
 .../hbase/mapreduce/TestHFileOutputFormat2.java |  42 +-
 .../hadoop/hbase/mapreduce/TestHashTable.java   |  11 +-
 .../TestImportTSVWithOperationAttributes.java   |  12 +-
 .../hbase/mapreduce/TestImportTSVWithTTLs.java  |   6 +-
 .../TestImportTSVWithVisibilityLabels.java      |  29 +-
 .../hadoop/hbase/mapreduce/TestImportTsv.java   |   6 +-
 .../TestLoadIncrementalHFilesSplitRecovery.java |  49 +-
 .../mapreduce/TestMultiTableInputFormat.java    |   5 +-
 .../mapreduce/TestMultithreadedTableMapper.java |   2 +-
 .../hadoop/hbase/mapreduce/TestRowCounter.java  |   2 +-
 .../hadoop/hbase/mapreduce/TestSyncTable.java   |  30 +-
 .../mapreduce/TestTableInputFormatScanBase.java |  15 +-
 .../hbase/mapreduce/TestTableMapReduceBase.java |   2 +-
 .../master/TestAssignmentManagerOnCluster.java  |  70 +-
 .../master/TestDistributedLogSplitting.java     |  22 +-
 .../master/TestGetLastFlushedSequenceId.java    |   4 +-
 .../apache/hadoop/hbase/master/TestMaster.java  |   3 +-
 .../TestMasterRestartAfterDisablingTable.java   |   5 +-
 .../hbase/master/TestMasterTransitions.java     |   4 +-
 .../hbase/master/TestRegionPlacement.java       |  14 +-
 .../hadoop/hbase/master/TestRollingRestart.java |   5 +-
 .../master/cleaner/TestSnapshotFromMaster.java  |   4 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |   2 +-
 .../hadoop/hbase/quotas/TestQuotaThrottle.java  |  13 +-
 .../TestEndToEndSplitTransaction.java           |  27 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  74 +-
 .../regionserver/TestHRegionOnCluster.java      |  10 +-
 .../regionserver/TestPerColumnFamilyFlush.java  |  10 +-
 .../regionserver/TestRegionFavoredNodes.java    |   3 +-
 .../hbase/regionserver/TestRegionReplicas.java  |   7 +-
 .../regionserver/TestRegionServerMetrics.java   |   4 +-
 .../regionserver/TestRegionServerNoMaster.java  |   8 +-
 .../TestRegionServerOnlineConfigChange.java     |  20 +-
 .../regionserver/TestScannerWithBulkload.java   |  22 +-
 .../regionserver/TestServerCustomProtocol.java  |  45 +-
 .../TestSplitTransactionOnCluster.java          |  17 +-
 .../TestCompactionWithThroughputController.java |   2 +-
 .../replication/TestReplicationKillRS.java      |   2 +-
 ...egionReplicaReplicationEndpointNoMaster.java |  12 +-
 .../security/access/TestAccessController.java   |   8 +-
 .../apache/hadoop/hbase/util/TestHBaseFsck.java | 381 +++++-----
 .../hbase/util/TestProcessBasedCluster.java     |   2 +-
 .../hadoop/hbase/util/TestRegionSplitter.java   |  79 ++-
 .../util/hbck/OfflineMetaRebuildTestCore.java   |  14 +-
 96 files changed, 2030 insertions(+), 1783 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
index 5743fd5..1c38349 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
 /**
  * A Get, Put, Increment, Append, or Delete associated with it's region.  Used internally by  
- * {@link HTable#batch} to associate the action with it's region and maintain
+ * {@link Table#batch} to associate the action with it's region and maintain
  * the index from the original request. 
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
index 249edec..ba86986 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java
@@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit;
 
 /**
  * <p>
- * Used to communicate with a single HBase table similar to {@link HTable}
+ * Used to communicate with a single HBase table similar to {@link Table}
  * but meant for batched, potentially asynchronous puts. Obtain an instance from
  * a {@link Connection} and call {@link #close()} afterwards.
  * </p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
index afce287..65f74c8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ScannerCallable.java
@@ -57,7 +57,7 @@ import com.google.protobuf.TextFormat;
 
 /**
  * Scanner operations such as create, next, etc.
- * Used by {@link ResultScanner}s made by {@link HTable}. Passed to a retrying caller such as
+ * Used by {@link ResultScanner}s made by {@link Table}. Passed to a retrying caller such as
  * {@link RpcRetryingCaller} so fails are retried.
  */
 @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
index f8a0e1c..a9c23cc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java
@@ -36,13 +36,15 @@ public abstract class Batch {
    *
    * <p>
    * When used with
-   * {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
+   * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
+   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
    * the implementations {@link Batch.Call#call(Object)} method will be invoked
    * with a proxy to each region's coprocessor {@link com.google.protobuf.Service} implementation.
    * </p>
    * @see org.apache.hadoop.hbase.client.coprocessor
-   * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])
-   * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
+   * @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
+   * @see org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
+   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
    * @param <T> the instance type to be passed to
    * {@link Batch.Call#call(Object)}
    * @param <R> the return type from {@link Batch.Call#call(Object)}
@@ -59,13 +61,15 @@ public abstract class Batch {
    *
    * <p>
    * When used with
-   * {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
+   * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
+   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}
    * the implementation's {@link Batch.Callback#update(byte[], byte[], Object)}
    * method will be called with the {@link Batch.Call#call(Object)} return value
    * from each region in the selected range.
    * </p>
    * @param <R> the return type from the associated {@link Batch.Call#call(Object)}
-   * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
+   * @see org.apache.hadoop.hbase.client.Table#coprocessorService(Class, byte[], byte[],
+   * org.apache.hadoop.hbase.client.coprocessor.Batch.Call)
    */
   @InterfaceAudience.Public
   @InterfaceStability.Stable

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
index 092e24d..502fc21 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
@@ -40,10 +40,10 @@ import com.google.protobuf.Message;
 /**
  * Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
  * against a given table region.  An instance of this class may be obtained
- * by calling {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])},
+ * by calling {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])},
  * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to call the endpoint
  * methods.
- * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])
+ * @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
  */
 @InterfaceAudience.Private
 public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
index 37890f9..ae0d52c 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestOperation.java
@@ -382,7 +382,7 @@ public class TestOperation {
     Assert.assertEquals(0, c.size());
     Assert.assertEquals(HConstants.LATEST_TIMESTAMP, p.getTimeStamp());
 
-    p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
+    p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 1984L, ByteBuffer.wrap(VALUE));
     c = p.get(FAMILY, QUALIFIER);
     Assert.assertEquals(1, c.size());
     Assert.assertEquals(1984L, c.get(0).getTimestamp());
@@ -391,7 +391,7 @@ public class TestOperation {
     Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
 
     p = new Put(ROW);
-    p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
+    p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2013L, null);
     c = p.get(FAMILY, QUALIFIER);
     Assert.assertEquals(1, c.size());
     Assert.assertEquals(2013L, c.get(0).getTimestamp());
@@ -400,7 +400,7 @@ public class TestOperation {
     Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
 
     p = new Put(ByteBuffer.wrap(ROW));
-    p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
+    p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
     c = p.get(FAMILY, QUALIFIER);
     Assert.assertEquals(1, c.size());
     Assert.assertEquals(2001L, c.get(0).getTimestamp());
@@ -410,7 +410,7 @@ public class TestOperation {
     Assert.assertEquals(0, CellComparator.COMPARATOR.compare(c.get(0), new KeyValue(c.get(0))));
 
     p = new Put(ByteBuffer.wrap(ROW), 1970L);
-    p.add(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
+    p.addColumn(FAMILY, ByteBuffer.wrap(QUALIFIER), 2001L, null);
     c = p.get(FAMILY, QUALIFIER);
     Assert.assertEquals(1, c.size());
     Assert.assertEquals(2001L, c.get(0).getTimestamp());

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index b5f0360..b9f97a7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -237,7 +237,7 @@ public class IntegrationTestBulkLoad extends IntegrationTestBase {
     }
 
     util.createTable(
-        getTablename().getName(),
+        getTablename(),
         new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
         getSplits(16)
     );

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index 41a705f..b797740 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -377,7 +377,7 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
     }
     doLoad(getConf(), htd);
     doVerify(getConf(), htd);
-    getTestingUtil(getConf()).deleteTable(htd.getName());
+    getTestingUtil(getConf()).deleteTable(getTablename());
     return 0;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index 0e74b46..48cebb2 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -52,6 +52,9 @@ import org.junit.BeforeClass;
 public class RowResourceBase {
 
   protected static final String TABLE = "TestRowResource";
+
+  protected static final TableName TABLE_NAME = TableName.valueOf(TABLE);
+
   protected static final String CFA = "a";
   protected static final String CFB = "b";
   protected static final String COLUMN_1 = CFA + ":1";
@@ -76,6 +79,7 @@ public class RowResourceBase {
   protected static Configuration conf;
   protected static ObjectMapper jsonMapper;
 
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
@@ -102,10 +106,10 @@ public class RowResourceBase {
   @Before
   public void beforeMethod() throws Exception {
     Admin admin = TEST_UTIL.getHBaseAdmin();
-    if (admin.tableExists(TableName.valueOf(TABLE))) {
-      TEST_UTIL.deleteTable(Bytes.toBytes(TABLE));
+    if (admin.tableExists(TABLE_NAME)) {
+      TEST_UTIL.deleteTable(TABLE_NAME);
     }
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
+    HTableDescriptor htd = new HTableDescriptor(TABLE);
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);
@@ -114,8 +118,8 @@ public class RowResourceBase {
   @After
   public void afterMethod() throws Exception {
     Admin admin = TEST_UTIL.getHBaseAdmin();
-    if (admin.tableExists(TableName.valueOf(TABLE))) {
-      TEST_UTIL.deleteTable(Bytes.toBytes(TABLE));
+    if (admin.tableExists(TABLE_NAME)) {
+      TEST_UTIL.deleteTable(TABLE_NAME);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
index 75ab0fc..ca3b82f 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java
@@ -84,8 +84,8 @@ public class TestStatusResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     TEST_UTIL.startMiniCluster(1, 1);
-    TEST_UTIL.createTable(Bytes.toBytes("TestStatusResource"), Bytes.toBytes("D"));
-    TEST_UTIL.createTable(Bytes.toBytes("TestStatusResource2"), Bytes.toBytes("D"));
+    TEST_UTIL.createTable(TableName.valueOf("TestStatusResource"), Bytes.toBytes("D"));
+    TEST_UTIL.createTable(TableName.valueOf("TestStatusResource2"), Bytes.toBytes("D"));
     REST_TEST_UTIL.startServletContainer(conf);
     Cluster cluster = new Cluster();
     cluster.add("localhost", REST_TEST_UTIL.getServletPort());

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
index e735bb1..919b6ad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 
 /**
- * A Base for {@link TableInputFormat}s. Receives a {@link HTable}, a
+ * A Base for {@link TableInputFormat}s. Receives a {@link Table}, a
  * byte[] of input columns and optionally a {@link Filter}.
  * Subclasses may use other TableRecordReader implementations.
  *

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
index 8b74e4e..9b77619 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
@@ -433,7 +433,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
     // atomically bulk load the groups.
     Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
-    for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
+    for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()){
       final byte[] first = e.getKey().array();
       final Collection<LoadQueueItem> lqis =  e.getValue();
 
@@ -854,7 +854,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
    * Algo:
    * 1) Poll on the keys in order:
    *    a) Keep adding the mapped values to these keys (runningSum)
-   *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
+   *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to
+   *       a boundary list.
    * 2) Return the boundary list.
    */
   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
@@ -958,8 +959,9 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
 
       Path hfofDir = new Path(dirPath);
 
-      try (HTable table = (HTable) connection.getTable(tableName);) {
-        doBulkLoad(hfofDir, table);
+      try (Table table = connection.getTable(tableName);
+          RegionLocator locator = connection.getRegionLocator(tableName)) {
+          doBulkLoad(hfofDir, admin, table, locator);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index 987b377..b8f2a22 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -526,7 +526,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
       }
     }
     //transform the List of bytes to byte[]
-    byte result[] = new byte[resultBytesList.size()];
+    byte[] result = new byte[resultBytesList.size()];
     for (int k = 0; k < resultBytesList.size(); k++) {
       result[k] = (byte) resultBytesList.get(k);
     }
@@ -558,7 +558,7 @@ extends InputFormat<ImmutableBytesWritable, Result> {
   }
 
   /**
-   * Allows subclasses to get the {@link HTable}.
+   * Allows subclasses to get the {@link Table}.
    *
    * @deprecated use {@link #getTable()}
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java
index 643ab0e..957a490 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSizeCalculator.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 
 /**
  * Computes size of each region for given table and given column families.
@@ -66,10 +67,11 @@ public class RegionSizeCalculator {
    * @deprecated Use {@link #RegionSizeCalculator(RegionLocator, Admin)} instead.
    */
   @Deprecated
-  public RegionSizeCalculator(HTable table) throws IOException {
+  public RegionSizeCalculator(Table table) throws IOException {
     try (Connection conn = ConnectionFactory.createConnection(table.getConfiguration());
+        RegionLocator locator = conn.getRegionLocator(table.getName());
         Admin admin = conn.getAdmin()) {
-      init(table.getRegionLocator(), admin);
+      init(locator, admin);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
index 0d5b27e..597573a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestCase.java
@@ -478,7 +478,7 @@ public abstract class HBaseTestCase extends TestCase {
   }
 
   /**
-   * A class that makes a {@link Incommon} out of a {@link HTable}
+   * A class that makes a {@link Incommon} out of a {@link Table}
    */
   public static class HTableIncommon implements Incommon {
     final Table table;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index a6d4d77..e8b79a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Consistency;
@@ -64,6 +65,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.HRegionLocator;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
@@ -1262,10 +1264,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param family
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[])}
    */
-  public HTable createTable(byte[] tableName, byte[] family)
-  throws IOException{
-    return createTable(TableName.valueOf(tableName), new byte[][]{family});
+  @Deprecated
+  public HTable createTable(byte[] tableName, byte[] family) throws IOException {
+    return createTable(TableName.valueOf(tableName), new byte[][] { family });
   }
 
   /**
@@ -1321,7 +1324,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param families
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][])}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[][] families)
   throws IOException {
     return createTable(tableName, families,
@@ -1364,16 +1369,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     return createTable(tableName, families, splitKeys, new Configuration(getConfiguration()));
   }
 
-  public HTable createTable(byte[] tableName, byte[][] families,
-      int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
-    return createTable(TableName.valueOf(tableName), families, numVersions,
-        startKey, endKey, numRegions);
+  @Deprecated
+  public HTable createTable(byte[] tableName, byte[][] families, int numVersions, byte[] startKey,
+      byte[] endKey, int numRegions) throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey,
+        numRegions);
   }
 
-  public HTable createTable(String tableName, byte[][] families,
-      int numVersions, byte[] startKey, byte[] endKey, int numRegions) throws IOException {
-    return createTable(TableName.valueOf(tableName), families, numVersions,
-        startKey, endKey, numRegions);
+  @Deprecated
+  public HTable createTable(String tableName, byte[][] families, int numVersions, byte[] startKey,
+      byte[] endKey, int numRegions) throws IOException {
+    return createTable(TableName.valueOf(tableName), families, numVersions, startKey, endKey,
+        numRegions);
   }
 
   public HTable createTable(TableName tableName, byte[][] families,
@@ -1452,10 +1459,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param c Configuration to use
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][])}
    */
-  public HTable createTable(TableName tableName, byte[][] families,
-      final Configuration c)
-  throws IOException {
+  @Deprecated
+  public HTable createTable(TableName tableName, byte[][] families, final Configuration c)
+      throws IOException {
     return createTable(tableName, families, (byte[][]) null, c);
   }
 
@@ -1480,10 +1488,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param c Configuration to use
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][])}
    */
-  public HTable createTable(byte[] tableName, byte[][] families,
-      final Configuration c)
-  throws IOException {
+  @Deprecated
+  public HTable createTable(byte[] tableName, byte[][] families, final Configuration c)
+      throws IOException {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     for(byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family);
@@ -1505,7 +1514,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], int)}
    */
+  @Deprecated
   public HTable createTable(TableName tableName, byte[][] families,
       final Configuration c, int numVersions)
   throws IOException {
@@ -1529,14 +1540,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], int)}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[][] families,
       final Configuration c, int numVersions)
   throws IOException {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
-    for(byte[] family : families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family)
-          .setMaxVersions(numVersions);
+    for (byte[] family : families) {
+      HColumnDescriptor hcd = new HColumnDescriptor(family).setMaxVersions(numVersions);
       desc.addFamily(hcd);
     }
     getHBaseAdmin().createTable(desc);
@@ -1550,7 +1562,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[], int)}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[] family, int numVersions)
   throws IOException {
     return createTable(tableName, new byte[][]{family}, numVersions);
@@ -1576,10 +1590,11 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], int)}
    */
-  public HTable createTable(byte[] tableName, byte[][] families,
-      int numVersions)
-  throws IOException {
+  @Deprecated
+  public HTable createTable(byte[] tableName, byte[][] families, int numVersions)
+      throws IOException {
     return createTable(TableName.valueOf(tableName), families, numVersions);
   }
 
@@ -1591,9 +1606,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @return An HTable instance for the created table.
    * @throws IOException
    */
-  public HTable createTable(TableName tableName, byte[][] families,
-      int numVersions)
-  throws IOException {
+  public HTable createTable(TableName tableName, byte[][] families, int numVersions)
+      throws IOException {
     return createTable(tableName, families, numVersions, (byte[][]) null);
   }
 
@@ -1640,7 +1654,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param blockSize
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], int, int)}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[][] families,
     int numVersions, int blockSize) throws IOException {
     return createTable(TableName.valueOf(tableName),
@@ -1678,7 +1694,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param numVersions
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], int)}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[][] families,
       int[] numVersions)
   throws IOException {
@@ -1717,7 +1735,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param splitRows
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[], byte[][])}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows)
     throws IOException{
     return createTable(TableName.valueOf(tableName), family, splitRows);
@@ -1760,7 +1780,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param splitRows
    * @return An HTable instance for the created table.
    * @throws IOException
+   * @deprecated use {@link #createTable(TableName, byte[][], byte[][])}
    */
+  @Deprecated
   public HTable createTable(byte[] tableName, byte[][] families, byte[][] splitRows)
       throws IOException {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
@@ -1817,7 +1839,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Drop an existing table
    * @param tableName existing table
+   * @deprecated use {@link #deleteTable(TableName)}
    */
+  @Deprecated
   public void deleteTable(String tableName) throws IOException {
     deleteTable(TableName.valueOf(tableName));
   }
@@ -1825,7 +1849,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Drop an existing table
    * @param tableName existing table
+   * @deprecated use {@link #deleteTable(TableName)}
    */
+  @Deprecated
   public void deleteTable(byte[] tableName) throws IOException {
     deleteTable(TableName.valueOf(tableName));
   }
@@ -1870,9 +1896,27 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
   public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
 
+  @Deprecated
   public HTableDescriptor createTableDescriptor(final String name,
       final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
+    return this.createTableDescriptor(TableName.valueOf(name), minVersions, versions, ttl,
+        keepDeleted);
+  }
+
+  /**
+   * Create a table of name <code>name</code>.
+   * @param name Name to give table.
+   * @return Column descriptor.
+   */
+  @Deprecated
+  public HTableDescriptor createTableDescriptor(final String name) {
+    return createTableDescriptor(TableName.valueOf(name),  HColumnDescriptor.DEFAULT_MIN_VERSIONS,
+        MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
+  }
+
+  public HTableDescriptor createTableDescriptor(final TableName name,
+      final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+    HTableDescriptor htd = new HTableDescriptor(name);
     for (byte[] cfName : new byte[][]{ fam1, fam2, fam3 }) {
       htd.addFamily(new HColumnDescriptor(cfName)
           .setMinVersions(minVersions)
@@ -1890,7 +1934,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param name Name to give table.
    * @return Column descriptor.
    */
-  public HTableDescriptor createTableDescriptor(final String name) {
+  public HTableDescriptor createTableDescriptor(final TableName name) {
     return createTableDescriptor(name,  HColumnDescriptor.DEFAULT_MIN_VERSIONS,
         MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
   }
@@ -1942,11 +1986,33 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @throws IOException
    * @return A region on which you must call
              {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
+   * @deprecated use
+   * {@link #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...)}
    */
+  @Deprecated
   public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
       WAL wal, byte[]... families) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+    return this
+        .createLocalHRegion(TableName.valueOf(tableName), startKey, stopKey, isReadOnly, durability,
+            wal, families);
+  }
+
+  /**
+   * @param tableName
+   * @param startKey
+   * @param stopKey
+   * @param callingMethod
+   * @param conf
+   * @param isReadOnly
+   * @param families
+   * @return A region on which you must call
+   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
+   * @throws IOException
+   */
+  public HRegion createLocalHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
+      boolean isReadOnly, Durability durability, WAL wal, byte[]... families) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.setReadOnly(isReadOnly);
     for (byte[] family : families) {
       HColumnDescriptor hcd = new HColumnDescriptor(family);
@@ -1958,6 +2024,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false);
     return createLocalHRegion(info, htd, wal);
   }
+
   //
   // ==========================================================================
 
@@ -1967,7 +2034,9 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * @param tableName existing table
    * @return HTable to that new table
    * @throws IOException
+   * @deprecated use {@link #deleteTableData(TableName)}
    */
+  @Deprecated
   public HTable deleteTableData(byte[] tableName) throws IOException {
     return deleteTableData(TableName.valueOf(tableName));
   }
@@ -2021,11 +2090,15 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   /**
    * Truncate a table using the admin command.
    * Effectively disables, deletes, and recreates the table.
-   * @param tableName table which must exist.
+   *
+   * @param tableName       table which must exist.
    * @param preserveRegions keep the existing split points
    * @return HTable for the new table
+   * @deprecated use {@link #truncateTable(TableName, boolean)}
    */
-  public HTable truncateTable(final byte[] tableName, final boolean preserveRegions) throws IOException {
+  @Deprecated
+  public HTable truncateTable(final byte[] tableName, final boolean preserveRegions)
+      throws IOException {
     return truncateTable(TableName.valueOf(tableName), preserveRegions);
   }
 
@@ -2035,11 +2108,14 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    * For previous behavior of issuing row deletes, see
    * deleteTableData.
    * Expressly does not preserve regions of existing table.
+   *
    * @param tableName table which must exist.
    * @return HTable for the new table
+   * @deprecated use {@link #truncateTable(TableName)}
    */
+  @Deprecated
   public HTable truncateTable(final byte[] tableName) throws IOException {
-    return truncateTable(tableName, false);
+    return truncateTable(TableName.valueOf(tableName), false);
   }
 
   /**
@@ -2395,7 +2471,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public List<HRegionInfo> createMultiRegionsInMeta(final Configuration conf,
       final HTableDescriptor htd, byte [][] startKeys)
   throws IOException {
-    Table meta = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
+    Table meta = getConnection().getTable(TableName.META_TABLE_NAME);
     Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR);
     List<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(startKeys.length);
     MetaTableAccessor
@@ -2455,7 +2531,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
    */
   public List<byte[]> getMetaTableRows() throws IOException {
     // TODO: Redo using MetaTableAccessor class
-    Table t = (HTable) getConnection().getTable(TableName.META_TABLE_NAME);
+    Table t = getConnection().getTable(TableName.META_TABLE_NAME);
     List<byte[]> rows = new ArrayList<byte[]>();
     ResultScanner s = t.getScanner(new Scan());
     for (Result result : s) {
@@ -3510,8 +3586,17 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
             HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
   }
 
+  @Deprecated
+  public HTable createRandomTable(String tableName, final Collection<String> families,
+      final int maxVersions, final int numColsPerRow, final int numFlushes, final int numRegions,
+      final int numRowsPerFlush) throws IOException, InterruptedException {
+    return (HTable) this
+        .createRandomTable(TableName.valueOf(tableName), families, maxVersions, numColsPerRow,
+            numFlushes, numRegions, numRowsPerFlush);
+  }
+
   /** Creates a random table with the given parameters */
-  public HTable createRandomTable(String tableName,
+  public Table createRandomTable(TableName tableName,
       final Collection<String> families,
       final int maxVersions,
       final int numColsPerRow,
@@ -3541,7 +3626,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
     final int splitStartKey = actualStartKey + keysPerRegion;
     final int splitEndKey = actualEndKey - keysPerRegion;
     final String keyFormat = "%08x";
-    final HTable table = createTable(tableName, cfBytes,
+    final Table table = createTable(tableName, cfBytes,
         maxVersions,
         Bytes.toBytes(String.format(keyFormat, splitStartKey)),
         Bytes.toBytes(String.format(keyFormat, splitEndKey)),
@@ -3551,6 +3636,8 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
       getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME);
     }
 
+    BufferedMutator mutator = getConnection().getBufferedMutator(tableName);
+
     for (int iFlush = 0; iFlush < numFlushes; ++iFlush) {
       for (int iRow = 0; iRow < numRowsPerFlush; ++iRow) {
         final byte[] row = Bytes.toBytes(String.format(keyFormat,
@@ -3575,19 +3662,20 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
         }
 
         if (!put.isEmpty()) {
-          table.put(put);
+          mutator.mutate(put);
         }
 
         if (!del.isEmpty()) {
-          table.delete(del);
+          mutator.mutate(del);
         }
       }
       LOG.info("Initiating flush #" + iFlush + " for table " + tableName);
-      table.flushCommits();
+      mutator.flush();
       if (hbaseCluster != null) {
         getMiniHBaseCluster().flushcache(table.getName());
       }
     }
+    mutator.close();
 
     return table;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
index d5b6a9c..7fd8902 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -91,7 +92,7 @@ public class TestFullLogReconstruction {
    */
   @Test (timeout=300000)
   public void testReconstruction() throws Exception {
-    HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
+    Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
 
     // Load up the table with simple rows and count them
     int initialCount = TEST_UTIL.loadTable(table, FAMILY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
index b82443b..ab0e6b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -73,12 +74,12 @@ public class TestGlobalMemStoreSize {
     cluster.waitForActiveAndReadyMaster();
 
     // Create a table with regions
-    byte [] table = Bytes.toBytes("TestGlobalMemStoreSize");
+    TableName table = TableName.valueOf("TestGlobalMemStoreSize");
     byte [] family = Bytes.toBytes("family");
     LOG.info("Creating table with " + regionNum + " regions");
-    HTable ht = TEST_UTIL.createMultiRegionTable(TableName.valueOf(table), family, regionNum);
+    Table ht = TEST_UTIL.createMultiRegionTable(table, family, regionNum);
     int numRegions = -1;
-    try (RegionLocator r = ht.getRegionLocator()) {
+    try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
       numRegions = r.getStartKeys().length;
     }
     assertEquals(regionNum,numRegions);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java
index 7135a24..5134a7b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -65,12 +66,11 @@ public class TestHBaseOnOtherDfsCluster {
 
     // do a simple create/write to ensure the cluster works as expected
     byte[] family = Bytes.toBytes("testfamily");
-    byte[] tablename = Bytes.toBytes("testtable");
-    HTable table = util2.createTable(tablename, family);
+    TableName tablename = TableName.valueOf("testtable");
+    Table table = util2.createTable(tablename, family);
     Put p = new Put(new byte[] { 1, 2, 3 });
-    p.add(family, null, new byte[] { 1 });
+    p.addColumn(family, null, new byte[] { 1 });
     table.put(p);
-    table.flushCommits();
 
     // shutdown and make sure cleanly shutting down
     util2.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index f78fbe8..479a528 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -98,9 +98,9 @@ public class TestMetaTableAccessor {
     final TableName name =
         TableName.valueOf("testRetrying");
     LOG.info("Started " + name);
-    HTable t = UTIL.createMultiRegionTable(name, HConstants.CATALOG_FAMILY);
+    Table t = UTIL.createMultiRegionTable(name, HConstants.CATALOG_FAMILY);
     int regionCount = -1;
-    try (RegionLocator r = t.getRegionLocator()) {
+    try (RegionLocator r = UTIL.getConnection().getRegionLocator(name)) {
       regionCount = r.getStartKeys().length;
     }
     // Test it works getting a region from just made user table.
@@ -494,7 +494,7 @@ public class TestMetaTableAccessor {
         new byte[][] { Bytes.toBytes("region_a"), Bytes.toBytes("region_b") };
 
     UTIL.createTable(TABLENAME, FAMILY, SPLIT_KEYS);
-    HTable table = (HTable) connection.getTable(TABLENAME);
+    Table table = connection.getTable(TABLENAME);
     // Make sure all the regions are deployed
     UTIL.countRows(table);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index 02bc3c7..572b72a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.MasterNotRunningException;
@@ -86,8 +87,7 @@ public class TestAdmin1 {
     TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
     TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
     TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
-    TEST_UTIL.getConfiguration().setBoolean(
-        "hbase.master.enabletable.roundrobin", true);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
     TEST_UTIL.startMiniCluster(3);
   }
 
@@ -104,7 +104,7 @@ public class TestAdmin1 {
   @After
   public void tearDown() throws Exception {
     for (HTableDescriptor htd : this.admin.listTables()) {
-      TEST_UTIL.deleteTable(htd.getName());
+      TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
 
@@ -362,9 +362,9 @@ public class TestAdmin1 {
     TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY).close();
     tables = this.admin.listTables();
     assertEquals(numTables + 1, tables.length);
-    assertTrue("Table must be enabled.", TEST_UTIL.getHBaseCluster()
-        .getMaster().getAssignmentManager().getTableStateManager().isTableState(
-            tableName, TableState.State.ENABLED));
+    assertTrue("Table must be enabled.",
+        TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getTableStateManager()
+            .isTableState(tableName, TableState.State.ENABLED));
     assertEquals(TableState.State.ENABLED, getStateFromMeta(tableName));
   }
 
@@ -434,11 +434,11 @@ public class TestAdmin1 {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(fam1);
     this.admin.createTable(htd);
-    HTable table = (HTable)TEST_UTIL.getConnection().getTable(htd.getTableName());
+    Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     long ts = this.admin.getLastMajorCompactionTimestamp(tableName);
     assertEquals(0, ts);
     Put p = new Put(Bytes.toBytes("row1"));
-    p.add(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
+    p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
     ts = this.admin.getLastMajorCompactionTimestamp(tableName);
     // no files written -> no data
@@ -449,12 +449,14 @@ public class TestAdmin1 {
     // still 0, we flushed a file, but no major compaction happened
     assertEquals(0, ts);
 
-    byte[] regionName =
-        table.getRegionLocator().getAllRegionLocations().get(0).getRegionInfo().getRegionName();
+    byte[] regionName;
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      regionName = l.getAllRegionLocations().get(0).getRegionInfo().getRegionName();
+    }
     long ts1 = this.admin.getLastMajorCompactionTimestampForRegion(regionName);
     assertEquals(ts, ts1);
     p = new Put(Bytes.toBytes("row2"));
-    p.add(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
+    p.addColumn(Bytes.toBytes("fam1"), Bytes.toBytes("fam1"), Bytes.toBytes("fam1"));
     table.put(p);
     this.admin.flush(tableName);
     ts = this.admin.getLastMajorCompactionTimestamp(tableName);
@@ -623,19 +625,19 @@ public class TestAdmin1 {
         "hbase.online.schema.update.enable", true);
   }
 
-  @SuppressWarnings("deprecation")
-  protected void verifyRoundRobinDistribution(HTable ht, int expectedRegions) throws IOException {
-    int numRS = ht.getConnection().getCurrentNrHRS();
-    Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
+  protected void verifyRoundRobinDistribution(ClusterConnection c, RegionLocator regionLocator, int
+      expectedRegions) throws IOException {
+    int numRS = c.getCurrentNrHRS();
+    List<HRegionLocation> regions = regionLocator.getAllRegionLocations();
     Map<ServerName, List<HRegionInfo>> server2Regions = new HashMap<ServerName, List<HRegionInfo>>();
-    for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
-      ServerName server = entry.getValue();
+    for (HRegionLocation loc : regions) {
+      ServerName server = loc.getServerName();
       List<HRegionInfo> regs = server2Regions.get(server);
       if (regs == null) {
         regs = new ArrayList<HRegionInfo>();
         server2Regions.put(server, regs);
       }
-      regs.add(entry.getKey());
+      regs.add(loc.getRegionInfo());
     }
     if (numRS >= 2) {
       // Ignore the master region server,
@@ -656,28 +658,29 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc);
-    HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
-    Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
-    assertEquals("Table should have only 1 region", 1, regions.size());
-    ht.close();
+    List<HRegionLocation> regions;
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Table should have only 1 region", 1, regions.size());
+    }
 
     TableName TABLE_2 = TableName.valueOf(tableName.getNameAsString() + "_2");
     desc = new HTableDescriptor(TABLE_2);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, new byte[][]{new byte[]{42}});
-    HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2);
-    regions = ht2.getRegionLocations();
-    assertEquals("Table should have only 2 region", 2, regions.size());
-    ht2.close();
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Table should have only 2 region", 2, regions.size());
+    }
 
     TableName TABLE_3 = TableName.valueOf(tableName.getNameAsString() + "_3");
     desc = new HTableDescriptor(TABLE_3);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, "a".getBytes(), "z".getBytes(), 3);
-    HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3);
-    regions = ht3.getRegionLocations();
-    assertEquals("Table should have only 3 region", 3, regions.size());
-    ht3.close();
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Table should have only 3 region", 3, regions.size());
+    }
 
     TableName TABLE_4 = TableName.valueOf(tableName.getNameAsString() + "_4");
     desc = new HTableDescriptor(TABLE_4);
@@ -692,11 +695,11 @@ public class TestAdmin1 {
     TableName TABLE_5 = TableName.valueOf(tableName.getNameAsString() + "_5");
     desc = new HTableDescriptor(TABLE_5);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
-    admin.createTable(desc, new byte[] {1}, new byte[] {127}, 16);
-    HTable ht5 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_5);
-    regions = ht5.getRegionLocations();
-    assertEquals("Table should have 16 region", 16, regions.size());
-    ht5.close();
+    admin.createTable(desc, new byte[] { 1 }, new byte[] { 127 }, 16);
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_5)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Table should have 16 region", 16, regions.size());
+    }
   }
 
   @Test (timeout=300000)
@@ -724,47 +727,53 @@ public class TestAdmin1 {
     boolean tableAvailable = admin.isTableAvailable(tableName, splitKeys);
     assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable);
 
-    HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
-    Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
-    assertEquals("Tried to create " + expectedRegions + " regions " +
-        "but only found " + regions.size(),
-        expectedRegions, regions.size());
-    System.err.println("Found " + regions.size() + " regions");
-
-    Iterator<HRegionInfo> hris = regions.keySet().iterator();
-    HRegionInfo hri = hris.next();
-    assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
-    assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
-    assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
-
-    verifyRoundRobinDistribution(ht, expectedRegions);
-    ht.close();
+    List<HRegionLocation> regions;
+    Iterator<HRegionLocation> hris;
+    HRegionInfo hri;
+    ClusterConnection conn = (ClusterConnection) TEST_UTIL.getConnection();
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      regions = l.getAllRegionLocations();
+
+      assertEquals("Tried to create " + expectedRegions + " regions " +
+              "but only found " + regions.size(), expectedRegions, regions.size());
+      System.err.println("Found " + regions.size() + " regions");
+
+      hris = regions.iterator();
+      hri = hris.next().getRegionInfo();
+      assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[0]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[0]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[1]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[1]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[2]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[2]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[3]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[3]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[4]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[4]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[5]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[5]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[6]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[6]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[7]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[7]));
+      assertTrue(Bytes.equals(hri.getEndKey(), splitKeys[8]));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), splitKeys[8]));
+      assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
+
+      verifyRoundRobinDistribution(conn, l, expectedRegions);
+    }
+
+
 
     // Now test using start/end with a number of regions
 
@@ -784,47 +793,46 @@ public class TestAdmin1 {
     admin = TEST_UTIL.getHBaseAdmin();
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
-    HTable ht2 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_2);
-    regions = ht2.getRegionLocations();
-    assertEquals("Tried to create " + expectedRegions + " regions " +
-        "but only found " + regions.size(),
-        expectedRegions, regions.size());
-    System.err.println("Found " + regions.size() + " regions");
-
-    hris = regions.keySet().iterator();
-    hri = hris.next();
-    assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {1,1,1,1,1,1,1,1,1,1}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {2,2,2,2,2,2,2,2,2,2}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {3,3,3,3,3,3,3,3,3,3}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {4,4,4,4,4,4,4,4,4,4}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {5,5,5,5,5,5,5,5,5,5}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {6,6,6,6,6,6,6,6,6,6}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {7,7,7,7,7,7,7,7,7,7}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {8,8,8,8,8,8,8,8,8,8}));
-    assertTrue(Bytes.equals(hri.getEndKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
-    hri = hris.next();
-    assertTrue(Bytes.equals(hri.getStartKey(), new byte [] {9,9,9,9,9,9,9,9,9,9}));
-    assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
-
-    verifyRoundRobinDistribution(ht2, expectedRegions);
-    ht2.close();
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_2)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Tried to create " + expectedRegions + " regions " +
+          "but only found " + regions.size(), expectedRegions, regions.size());
+      System.err.println("Found " + regions.size() + " regions");
+
+      hris = regions.iterator();
+      hri = hris.next().getRegionInfo();
+      assertTrue(hri.getStartKey() == null || hri.getStartKey().length == 0);
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 2, 2, 2, 2, 2, 2, 2, 2, 2, 2 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 3, 3, 3, 3, 3, 3, 3, 3, 3, 3 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 4, 4, 4, 4, 4, 4, 4, 4, 4, 4 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 5, 5, 5, 5, 5, 5, 5, 5, 5, 5 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 6, 6, 6, 6, 6, 6, 6, 6, 6, 6 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 7, 7, 7, 7, 7, 7, 7, 7, 7, 7 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 8, 8, 8, 8, 8, 8, 8, 8, 8, 8 }));
+      assertTrue(Bytes.equals(hri.getEndKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
+      hri = hris.next().getRegionInfo();
+      assertTrue(Bytes.equals(hri.getStartKey(), new byte[] { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 }));
+      assertTrue(hri.getEndKey() == null || hri.getEndKey().length == 0);
+
+      verifyRoundRobinDistribution(conn, l, expectedRegions);
+    }
 
     // Try once more with something that divides into something infinite
 
@@ -841,15 +849,14 @@ public class TestAdmin1 {
     admin.createTable(desc, startKey, endKey, expectedRegions);
 
 
-    HTable ht3 = (HTable) TEST_UTIL.getConnection().getTable(TABLE_3);
-    regions = ht3.getRegionLocations();
-    assertEquals("Tried to create " + expectedRegions + " regions " +
-        "but only found " + regions.size(),
-        expectedRegions, regions.size());
-    System.err.println("Found " + regions.size() + " regions");
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(TABLE_3)) {
+      regions = l.getAllRegionLocations();
+      assertEquals("Tried to create " + expectedRegions + " regions " +
+          "but only found " + regions.size(), expectedRegions, regions.size());
+      System.err.println("Found " + regions.size() + " regions");
 
-    verifyRoundRobinDistribution(ht3, expectedRegions);
-    ht3.close();
+      verifyRoundRobinDistribution(conn, l, expectedRegions);
+    }
 
 
     // Try an invalid case where there are duplicate split keys
@@ -962,20 +969,22 @@ public class TestAdmin1 {
     HTableDescriptor desc = new HTableDescriptor(tableName);
     desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
     admin.createTable(desc, splitKeys);
-    HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
-    Map<HRegionInfo, ServerName> regions = ht.getRegionLocations();
-    assertEquals("Tried to create " + expectedRegions + " regions "
-        + "but only found " + regions.size(), expectedRegions, regions.size());
-    // Disable table.
-    admin.disableTable(tableName);
-    // Enable table, use retain assignment to assign regions.
-    admin.enableTable(tableName);
-    Map<HRegionInfo, ServerName> regions2 = ht.getRegionLocations();
-
-    // Check the assignment.
-    assertEquals(regions.size(), regions2.size());
-    for (Map.Entry<HRegionInfo, ServerName> entry : regions.entrySet()) {
-      assertEquals(regions2.get(entry.getKey()), entry.getValue());
+
+    try (RegionLocator l = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      List<HRegionLocation> regions = l.getAllRegionLocations();
+
+      assertEquals(
+          "Tried to create " + expectedRegions + " regions " + "but only found " + regions.size(),
+          expectedRegions, regions.size());
+      // Disable table.
+      admin.disableTable(tableName);
+      // Enable table, use retain assignment to assign regions.
+      admin.enableTable(tableName);
+      List<HRegionLocation> regions2 = l.getAllRegionLocations();
+
+      // Check the assignment.
+      assertEquals(regions.size(), regions2.size());
+      assertTrue(regions2.containsAll(regions));
     }
   }
 
@@ -1022,133 +1031,132 @@ public class TestAdmin1 {
       sb.append("_").append(Integer.toString(rowCounts[i]));
     }
     assertFalse(admin.tableExists(tableName));
-    final HTable table = TEST_UTIL.createTable(tableName, familyNames,
+    try(final Table table = TEST_UTIL.createTable(tableName, familyNames,
       numVersions, blockSize);
+      final RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+
+      int rowCount = 0;
+      byte[] q = new byte[0];
+
+      // insert rows into column families. The number of rows that have values
+      // in a specific column family is decided by rowCounts[familyIndex]
+      for (int index = 0; index < familyNames.length; index++) {
+        ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
+        for (int i = 0; i < rowCounts[index]; i++) {
+          byte[] k = Bytes.toBytes(i);
+          Put put = new Put(k);
+          put.addColumn(familyNames[index], q, k);
+          puts.add(put);
+        }
+        table.put(puts);
 
-    int rowCount = 0;
-    byte[] q = new byte[0];
-
-    // insert rows into column families. The number of rows that have values
-    // in a specific column family is decided by rowCounts[familyIndex]
-    for (int index = 0; index < familyNames.length; index++) {
-      ArrayList<Put> puts = new ArrayList<Put>(rowCounts[index]);
-      for (int i = 0; i < rowCounts[index]; i++) {
-        byte[] k = Bytes.toBytes(i);
-        Put put = new Put(k);
-        put.add(familyNames[index], q, k);
-        puts.add(put);
+        if (rowCount < rowCounts[index]) {
+          rowCount = rowCounts[index];
+        }
       }
-      table.put(puts);
 
-      if ( rowCount < rowCounts[index] ) {
-        rowCount = rowCounts[index];
+      // get the initial layout (should just be one region)
+      List<HRegionLocation> m = locator.getAllRegionLocations();
+      LOG.info("Initial regions (" + m.size() + "): " + m);
+      assertTrue(m.size() == 1);
+
+      // Verify row count
+      Scan scan = new Scan();
+      ResultScanner scanner = table.getScanner(scan);
+      int rows = 0;
+      for (@SuppressWarnings("unused") Result result : scanner) {
+        rows++;
       }
-    }
-
-    // get the initial layout (should just be one region)
-    Map<HRegionInfo, ServerName> m = table.getRegionLocations();
-    LOG.info("Initial regions (" + m.size() + "): " + m);
-    assertTrue(m.size() == 1);
-
-    // Verify row count
-    Scan scan = new Scan();
-    ResultScanner scanner = table.getScanner(scan);
-    int rows = 0;
-    for(@SuppressWarnings("unused") Result result : scanner) {
-      rows++;
-    }
-    scanner.close();
-    assertEquals(rowCount, rows);
-
-    // Have an outstanding scan going on to make sure we can scan over splits.
-    scan = new Scan();
-    scanner = table.getScanner(scan);
-    // Scan first row so we are into first region before split happens.
-    scanner.next();
-
-    // Split the table
-    this.admin.split(tableName, splitPoint);
-
-    final AtomicInteger count = new AtomicInteger(0);
-    Thread t = new Thread("CheckForSplit") {
-      @Override
-      public void run() {
-        for (int i = 0; i < 45; i++) {
-          try {
-            sleep(1000);
-          } catch (InterruptedException e) {
-            continue;
+      scanner.close();
+      assertEquals(rowCount, rows);
+
+      // Have an outstanding scan going on to make sure we can scan over splits.
+      scan = new Scan();
+      scanner = table.getScanner(scan);
+      // Scan first row so we are into first region before split happens.
+      scanner.next();
+
+      // Split the table
+      this.admin.split(tableName, splitPoint);
+
+      final AtomicInteger count = new AtomicInteger(0);
+      Thread t = new Thread("CheckForSplit") {
+        @Override
+        public void run() {
+          for (int i = 0; i < 45; i++) {
+            try {
+              sleep(1000);
+            } catch (InterruptedException e) {
+              continue;
+            }
+            // check again
+            List<HRegionLocation> regions = null;
+            try {
+              regions = locator.getAllRegionLocations();
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+            if (regions == null)
+              continue;
+            count.set(regions.size());
+            if (count.get() >= 2) {
+              LOG.info("Found: " + regions);
+              break;
+            }
+            LOG.debug("Cycle waiting on split");
           }
-          // check again    table = new HTable(conf, tableName);
-          Map<HRegionInfo, ServerName> regions = null;
-          try {
-            regions = table.getRegionLocations();
-          } catch (IOException e) {
-            e.printStackTrace();
-          }
-          if (regions == null) continue;
-          count.set(regions.size());
-          if (count.get() >= 2) {
-            LOG.info("Found: " + regions);
-            break;
-          }
-          LOG.debug("Cycle waiting on split");
+          LOG.debug("CheckForSplit thread exited, current region count: " + count.get());
+        }
+      };
+      t.setPriority(Thread.NORM_PRIORITY - 2);
+      t.start();
+      t.join();
+
+      // Verify row count
+      rows = 1; // We counted one row above.
+      for (@SuppressWarnings("unused") Result result : scanner) {
+        rows++;
+        if (rows > rowCount) {
+          scanner.close();
+          assertTrue("Scanned more than expected (" + rowCount + ")", false);
         }
-        LOG.debug("CheckForSplit thread exited, current region count: " + count.get());
-      }
-    };
-    t.setPriority(Thread.NORM_PRIORITY - 2);
-    t.start();
-    t.join();
-
-    // Verify row count
-    rows = 1; // We counted one row above.
-    for (@SuppressWarnings("unused") Result result : scanner) {
-      rows++;
-      if (rows > rowCount) {
-        scanner.close();
-        assertTrue("Scanned more than expected (" + rowCount + ")", false);
       }
-    }
-    scanner.close();
-    assertEquals(rowCount, rows);
+      scanner.close();
+      assertEquals(rowCount, rows);
 
-    Map<HRegionInfo, ServerName> regions = null;
-    try {
-      regions = table.getRegionLocations();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    assertEquals(2, regions.size());
-    Set<HRegionInfo> hRegionInfos = regions.keySet();
-    HRegionInfo[] r = hRegionInfos.toArray(new HRegionInfo[hRegionInfos.size()]);
-    if (splitPoint != null) {
-      // make sure the split point matches our explicit configuration
-      assertEquals(Bytes.toString(splitPoint),
-          Bytes.toString(r[0].getEndKey()));
-      assertEquals(Bytes.toString(splitPoint),
-          Bytes.toString(r[1].getStartKey()));
-      LOG.debug("Properly split on " + Bytes.toString(splitPoint));
-    } else {
-      if (familyNames.length > 1) {
-        int splitKey = Bytes.toInt(r[0].getEndKey());
-        // check if splitKey is based on the largest column family
-        // in terms of it store size
-        int deltaForLargestFamily = Math.abs(rowCount/2 - splitKey);
-        LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily +
-          ", r=" + r[0]);
-        for (int index = 0; index < familyNames.length; index++) {
-          int delta = Math.abs(rowCounts[index]/2 - splitKey);
-          if (delta < deltaForLargestFamily) {
-            assertTrue("Delta " + delta + " for family " + index
-              + " should be at least deltaForLargestFamily " + deltaForLargestFamily,
-              false);
+      List<HRegionLocation> regions = null;
+      try {
+        regions = locator.getAllRegionLocations();
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      assertEquals(2, regions.size());
+      if (splitPoint != null) {
+        // make sure the split point matches our explicit configuration
+        assertEquals(Bytes.toString(splitPoint),
+            Bytes.toString(regions.get(0).getRegionInfo().getEndKey()));
+        assertEquals(Bytes.toString(splitPoint),
+            Bytes.toString(regions.get(1).getRegionInfo().getStartKey()));
+        LOG.debug("Properly split on " + Bytes.toString(splitPoint));
+      } else {
+        if (familyNames.length > 1) {
+          int splitKey = Bytes.toInt(regions.get(0).getRegionInfo().getEndKey());
+          // check if splitKey is based on the largest column family
+          // in terms of it store size
+          int deltaForLargestFamily = Math.abs(rowCount / 2 - splitKey);
+          LOG.debug("SplitKey=" + splitKey + "&deltaForLargestFamily=" + deltaForLargestFamily +
+              ", r=" + regions.get(0).getRegionInfo());
+          for (int index = 0; index < familyNames.length; index++) {
+            int delta = Math.abs(rowCounts[index] / 2 - splitKey);
+            if (delta < deltaForLargestFamily) {
+              assertTrue("Delta " + delta + " for family " + index + " should be at least "
+                      + "deltaForLargestFamily " + deltaForLargestFamily, false);
+            }
           }
         }
       }
+      TEST_UTIL.deleteTable(tableName);
     }
-    TEST_UTIL.deleteTable(tableName);
-    table.close();
   }
 
   @Test
@@ -1173,7 +1181,7 @@ public class TestAdmin1 {
       Thread.sleep(10);
     } while (oldRegions.size() != 9); //3 regions * 3 replicas
     // write some data to the table
-    HTable ht = (HTable) TEST_UTIL.getConnection().getTable(tableName);
+    Table ht = TEST_UTIL.getConnection().getTable(tableName);
     List<Put> puts = new ArrayList<Put>();
     byte[] qualifier = "c".getBytes();
     Put put = new Put(new byte[]{(byte)'1'});
@@ -1186,7 +1194,6 @@ public class TestAdmin1 {
     put.add(cf, qualifier, "100".getBytes());
     puts.add(put);
     ht.put(puts);
-    ht.flushCommits();
     ht.close();
     List<Pair<HRegionInfo, ServerName>> regions =
         MetaTableAccessor.getTableRegionsAndLocations(TEST_UTIL.getConnection(), tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index b121c37..5a95d61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -104,7 +104,7 @@ public class TestAdmin2 {
   @After
   public void tearDown() throws Exception {
     for (HTableDescriptor htd : this.admin.listTables()) {
-      TEST_UTIL.deleteTable(htd.getName());
+      TEST_UTIL.deleteTable(htd.getTableName());
     }
   }
 
@@ -693,15 +693,17 @@ public class TestAdmin2 {
 
     final TableName tableName = TableName.valueOf("testGetRegion");
     LOG.info("Started " + tableName);
-    HTable t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
-
-    HRegionLocation regionLocation = t.getRegionLocation("mmm");
-    HRegionInfo region = regionLocation.getRegionInfo();
-    byte[] regionName = region.getRegionName();
-    Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
-    assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
-    pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
-    assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
+    Table t = TEST_UTIL.createMultiRegionTable(tableName, HConstants.CATALOG_FAMILY);
+
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      HRegionLocation regionLocation = locator.getRegionLocation(Bytes.toBytes("mmm"));
+      HRegionInfo region = regionLocation.getRegionInfo();
+      byte[] regionName = region.getRegionName();
+      Pair<HRegionInfo, ServerName> pair = rawAdmin.getRegion(regionName);
+      assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
+      pair = rawAdmin.getRegion(region.getEncodedNameAsBytes());
+      assertTrue(Bytes.equals(regionName, pair.getFirst().getRegionName()));
+    }
   }
 
   @Test(timeout = 30000)

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
index ae8a449..b68381f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCheckAndMutate.java
@@ -57,7 +57,7 @@ public class TestCheckAndMutate {
     final TableName tableName = TableName.valueOf("TestPutWithDelete");
     final byte[] rowKey = Bytes.toBytes("12345");
     final byte[] family = Bytes.toBytes("cf");
-    HTable table = TEST_UTIL.createTable(tableName, family);
+    Table table = TEST_UTIL.createTable(tableName, family);
     TEST_UTIL.waitTableAvailable(tableName.getName(), 5000);
     try {
       // put one row
@@ -79,11 +79,11 @@ public class TestCheckAndMutate {
       // put the same row again with C column deleted
       RowMutations rm = new RowMutations(rowKey);
       put = new Put(rowKey);
-      put.add(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
-      put.add(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
+      put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("a"));
+      put.addColumn(family, Bytes.toBytes("B"), Bytes.toBytes("b"));
       rm.add(put);
       Delete del = new Delete(rowKey);
-      del.deleteColumn(family, Bytes.toBytes("C"));
+      del.addColumn(family, Bytes.toBytes("C"));
       rm.add(del);
       boolean res = table.checkAndMutate(rowKey, family, Bytes.toBytes("A"), CompareFilter.CompareOp.EQUAL,
           Bytes.toBytes("a"), rm);


[2/6] hbase git commit: HBASE-13893 Replace HTable with Table in client tests (Jurriaan Mous)

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 6d1859c..bd6d452 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -196,7 +196,7 @@ public class TestHRegion {
   private final int MAX_VERSIONS = 2;
 
   // Test names
-  protected byte[] tableName;
+  protected TableName tableName;
   protected String method;
   protected final byte[] qual1 = Bytes.toBytes("qual1");
   protected final byte[] qual2 = Bytes.toBytes("qual2");
@@ -216,7 +216,7 @@ public class TestHRegion {
     CONF = TEST_UTIL.getConfiguration();
     dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
     method = name.getMethodName();
-    tableName = Bytes.toBytes(name.getMethodName());
+    tableName = TableName.valueOf(name.getMethodName());
   }
 
   @After
@@ -351,13 +351,13 @@ public class TestHRegion {
    * @param callingMethod a unique component for the path, probably the name of the test method.
    */
   private static WAL createWALCompatibleWithFaultyFileSystem(String callingMethod,
-      Configuration conf, byte[] tableName) throws IOException {
+      Configuration conf, TableName tableName) throws IOException {
     final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, logDir);
     return (new WALFactory(walConf,
         Collections.<WALActionsListener>singletonList(new MetricsWAL()), callingMethod))
-        .getWAL(tableName);
+        .getWAL(tableName.toBytes());
   }
 
   /**
@@ -888,7 +888,7 @@ public class TestHRegion {
     final WALFactory wals = new WALFactory(walConf, null, method);
     final WAL wal = wals.getWAL(tableName.getName());
 
-    this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
+    this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
     try {
       Path regiondir = region.getRegionFileSystem().getRegionDir();
@@ -1046,7 +1046,7 @@ public class TestHRegion {
     final WALFactory wals = new WALFactory(walConf, null, method);
     WAL wal = spy(wals.getWAL(tableName.getName()));
 
-    this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
+    this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
     try {
       int i = 0;
@@ -1089,7 +1089,7 @@ public class TestHRegion {
       }
 
       region.close();
-      this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
+      this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
       region.put(put);
 
@@ -1207,7 +1207,7 @@ public class TestHRegion {
    */
   @Test
   public void testWeirdCacheBehaviour() throws Exception {
-    byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
+    TableName TABLE = TableName.valueOf("testWeirdCacheBehaviour");
     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"),
         Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
     this.region = initHRegion(TABLE, getName(), CONF, FAMILIES);
@@ -1250,7 +1250,7 @@ public class TestHRegion {
 
   @Test
   public void testAppendWithReadOnlyTable() throws Exception {
-    byte[] TABLE = Bytes.toBytes("readOnlyTable");
+    TableName TABLE = TableName.valueOf("readOnlyTable");
     this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
     boolean exceptionCaught = false;
     Append append = new Append(Bytes.toBytes("somerow"));
@@ -1270,7 +1270,7 @@ public class TestHRegion {
 
   @Test
   public void testIncrWithReadOnlyTable() throws Exception {
-    byte[] TABLE = Bytes.toBytes("readOnlyTable");
+    TableName TABLE = TableName.valueOf("readOnlyTable");
     this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
     boolean exceptionCaught = false;
     Increment inc = new Increment(Bytes.toBytes("somerow"));
@@ -1362,11 +1362,11 @@ public class TestHRegion {
 
   @Test
   public void testFamilyWithAndWithoutColon() throws Exception {
-    byte[] b = Bytes.toBytes(getName());
+    TableName b = TableName.valueOf(getName());
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     this.region = initHRegion(b, getName(), CONF, cf);
     try {
-      Put p = new Put(b);
+      Put p = new Put(b.toBytes());
       byte[] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
       p.add(cfwithcolon, cfwithcolon, cfwithcolon);
       boolean exception = false;
@@ -1387,7 +1387,7 @@ public class TestHRegion {
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
+    this.region = initHRegion(TableName.valueOf(getName()), getName(), CONF, cf);
     MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
     try {
       long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
@@ -1428,7 +1428,7 @@ public class TestHRegion {
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
-    this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
+    this.region = initHRegion(TableName.valueOf(getName()), getName(), CONF, cf);
     MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
     try {
       long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
@@ -1526,7 +1526,7 @@ public class TestHRegion {
 
   @Test
   public void testBatchPutWithTsSlop() throws Exception {
-    byte[] b = Bytes.toBytes(getName());
+    TableName b = TableName.valueOf(getName());
     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
     byte[] qual = Bytes.toBytes("qual");
     byte[] val = Bytes.toBytes("val");
@@ -4049,7 +4049,7 @@ public class TestHRegion {
     HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE)
         .setBloomFilterType(BloomType.ROWCOL);
 
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+    HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     this.region = TEST_UTIL.createLocalHRegion(info, htd);
@@ -4154,7 +4154,7 @@ public class TestHRegion {
     HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE)
         .setBloomFilterType(BloomType.ROWCOL);
 
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
+    HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(hcd);
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
     this.region = TEST_UTIL.createLocalHRegion(info, htd);
@@ -4204,7 +4204,7 @@ public class TestHRegion {
     try {
       cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
       byte[][] families = { fam1, fam2 };
-      Table ht = htu.createTable(Bytes.toBytes(this.getName()), families);
+      Table ht = htu.createTable(TableName.valueOf(this.getName()), families);
 
       // Setting up region
       byte row[] = Bytes.toBytes("row1");
@@ -4646,7 +4646,7 @@ public class TestHRegion {
     FSUtils.setRootDir(walConf, logDir);
     final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
     final WAL wal = spy(wals.getWAL(tableName.getName()));
-    this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
+    this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, wal,
         new byte[][] { family });
 
@@ -4949,7 +4949,7 @@ public class TestHRegion {
   @Test
   public void testFlushResult() throws IOException {
     String method = name.getMethodName();
-    byte[] tableName = Bytes.toBytes(method);
+    TableName tableName = TableName.valueOf(method);
     byte[] family = Bytes.toBytes("family");
 
     this.region = initHRegion(tableName, method, family);
@@ -4961,7 +4961,7 @@ public class TestHRegion {
 
     // Flush enough files to get up to the threshold, doesn't need compactions
     for (int i = 0; i < 2; i++) {
-      Put put = new Put(tableName).add(family, family, tableName);
+      Put put = new Put(tableName.toBytes()).add(family, family, tableName.toBytes());
       region.put(put);
       fr = region.flush(true);
       assertTrue(fr.isFlushSucceeded());
@@ -4970,7 +4970,7 @@ public class TestHRegion {
 
     // Two flushes after the threshold, compactions are needed
     for (int i = 0; i < 2; i++) {
-      Put put = new Put(tableName).add(family, family, tableName);
+      Put put = new Put(tableName.toBytes()).add(family, family, tableName.toBytes());
       region.put(put);
       fr = region.flush(true);
       assertTrue(fr.isFlushSucceeded());
@@ -5007,20 +5007,6 @@ public class TestHRegion {
    */
   private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
       byte[]... families) throws IOException {
-    return initHRegion(tableName.getName(), null, null, callingMethod, conf, false, families);
-  }
-
-  /**
-   * @param tableName
-   * @param callingMethod
-   * @param conf
-   * @param families
-   * @throws IOException
-   * @return A region on which you must call
-   *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
-   */
-  private static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
-      byte[]... families) throws IOException {
     return initHRegion(tableName, null, null, callingMethod, conf, false, families);
   }
 
@@ -5034,16 +5020,16 @@ public class TestHRegion {
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
    */
-  private static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+  private static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
       boolean isReadOnly, byte[]... families) throws IOException {
     return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
   }
 
-  public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
+  public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
-    HRegionInfo hri = new HRegionInfo(TableName.valueOf(tableName), startKey, stopKey);
+    HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
     final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
     return initHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly,
         Durability.SYNC_WAL, wal, families);
@@ -5061,10 +5047,10 @@ public class TestHRegion {
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.
    */
-  public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
+  public static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
       String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
       WAL wal, byte[]... families) throws IOException {
-    return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf,
+    return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey,
         isReadOnly, durability, wal, families);
   }
 
@@ -5647,7 +5633,7 @@ public class TestHRegion {
 
   @Test (timeout=60000)
   public void testSplitRegionWithReverseScan() throws IOException {
-    byte [] tableName = Bytes.toBytes("testSplitRegionWithReverseScan");
+    TableName tableName = TableName.valueOf("testSplitRegionWithReverseScan");
     byte [] qualifier = Bytes.toBytes("qualifier");
     Configuration hc = initSplit();
     int numRows = 3;
@@ -6024,7 +6010,7 @@ public class TestHRegion {
   @Test (timeout=24000)
   public void testRegionTooBusy() throws IOException {
     String method = "testRegionTooBusy";
-    byte[] tableName = Bytes.toBytes(method);
+    TableName tableName = TableName.valueOf(method);
     byte[] family = Bytes.toBytes("family");
     long defaultBusyWaitDuration = CONF.getLong("hbase.busy.wait.duration",
       HRegion.DEFAULT_BUSY_WAIT_DURATION);
@@ -6198,7 +6184,7 @@ public class TestHRegion {
     }
   }
 
-  static HRegion initHRegion(byte[] tableName, String callingMethod,
+  static HRegion initHRegion(TableName tableName, String callingMethod,
       byte[]... families) throws IOException {
     return initHRegion(tableName, callingMethod, HBaseConfiguration.create(),
         families);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
index 9b8dabf..fa152ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -76,12 +77,17 @@ public class TestHRegionOnCluster {
 
       // Put data: r1->v1
       Log.info("Loading r1 to v1 into " + TABLENAME);
-      HTable table = (HTable) TEST_UTIL.getConnection().getTable(TABLENAME);
+      Table table = TEST_UTIL.getConnection().getTable(TABLENAME);
       putDataAndVerify(table, "r1", FAMILY, "v1", 1);
 
       TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
       // Move region to target server
-      HRegionInfo regionInfo = table.getRegionLocation("r1").getRegionInfo();
+
+      HRegionInfo regionInfo;
+      try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(TABLENAME)) {
+        regionInfo = locator.getRegionLocation(Bytes.toBytes("r1")).getRegionInfo();
+      }
+
       int originServerNum = cluster.getServerWith(regionInfo.getRegionName());
       HRegionServer originServer = cluster.getRegionServer(originServerNum);
       int targetServerNum = (originServerNum + 1) % NUM_RS;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
index 82689e4..c6aaf67 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java
@@ -110,7 +110,7 @@ public class TestPerColumnFamilyFlush {
   }
 
   // A helper function to verify edits.
-  void verifyEdit(int familyNum, int putNum, HTable table) throws IOException {
+  void verifyEdit(int familyNum, int putNum, Table table) throws IOException {
     Result r = table.get(createGet(familyNum, putNum));
     byte[] family = FAMILIES[familyNum - 1];
     byte[] qf = Bytes.toBytes("q" + familyNum);
@@ -342,7 +342,7 @@ public class TestPerColumnFamilyFlush {
       TEST_UTIL.startMiniCluster(numRegionServers);
       TEST_UTIL.getHBaseAdmin().createNamespace(
         NamespaceDescriptor.create(TABLENAME.getNamespaceAsString()).build());
-      HTable table = TEST_UTIL.createTable(TABLENAME, FAMILIES);
+      Table table = TEST_UTIL.createTable(TABLENAME, FAMILIES);
       HTableDescriptor htd = table.getTableDescriptor();
 
       for (byte[] family : FAMILIES) {
@@ -360,7 +360,6 @@ public class TestPerColumnFamilyFlush {
           table.put(createPut(3, i));
         }
       }
-      table.flushCommits();
       Thread.sleep(1000);
 
       Pair<Region, HRegionServer> desiredRegionAndServer = getRegionWithName(TABLENAME);
@@ -466,8 +465,7 @@ public class TestPerColumnFamilyFlush {
     final int numRegionServers = 1;
     TEST_UTIL.startMiniCluster(numRegionServers);
     try {
-      HTable table = null;
-      table = TEST_UTIL.createTable(tableName, FAMILIES);
+      Table table = TEST_UTIL.createTable(tableName, FAMILIES);
       // Force flush the namespace table so edits to it are not hanging around as oldest
       // edits. Otherwise, below, when we make maximum number of WAL files, then it will be
       // the namespace region that is flushed and not the below 'desiredRegion'.
@@ -489,7 +487,6 @@ public class TestPerColumnFamilyFlush {
         for (int j = 0; j < 100; j++) {
           table.put(createPut(1, i * 100 + j));
         }
-        table.flushCommits();
         // Roll the WAL. The log file count is less than maxLogs so no flush is triggered.
         int currentNumRolledLogFiles = getNumRolledLogFiles(desiredRegion);
         assertNull(getWAL(desiredRegion).rollWriter());
@@ -503,7 +500,6 @@ public class TestPerColumnFamilyFlush {
       assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize() < cfFlushSizeLowerBound);
       assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize() < cfFlushSizeLowerBound);
       table.put(createPut(1, 12345678));
-      table.flushCommits();
       // Make numRolledLogFiles greater than maxLogs
       desiredRegionAndServer.getSecond().walRoller.requestRollAll();
       // Wait for some time till the flush caused by log rolling happens.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java
index e924c4c..6dbdfb5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionFavoredNodes.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -53,7 +54,7 @@ import org.junit.experimental.categories.Category;
 public class TestRegionFavoredNodes {
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static HTable table;
+  private static Table table;
   private static final TableName TABLE_NAME =
       TableName.valueOf("table");
   private static final byte[] COLUMN_FAMILY = Bytes.toBytes("family");

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
index 85a8cd2..9ee0a76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.TableName;
@@ -68,7 +69,7 @@ public class TestRegionReplicas {
   private static final Log LOG = LogFactory.getLog(TestRegionReplicas.class);
 
   private static final int NB_SERVERS = 1;
-  private static HTable table;
+  private static Table table;
   private static final byte[] row = "TestRegionReplicas".getBytes();
 
   private static HRegionInfo hriPrimary;
@@ -91,7 +92,9 @@ public class TestRegionReplicas {
     // Create table then get the single region for our new table.
     table = HTU.createTable(tableName, f);
 
-    hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
+    try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
+      hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
+    }
 
     // mock a secondary region info to open
     hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 5498d66..c164363 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -337,7 +337,7 @@ public class TestRegionServerMetrics {
       p.add(cf, qualifier, val);
       puts.add(p);
     }
-    try (HTable t = TEST_UTIL.createTable(tableName, cf)) {
+    try (Table t = TEST_UTIL.createTable(tableName, cf)) {
       t.put(puts);
 
       Scan s = new Scan();
@@ -387,7 +387,7 @@ public class TestRegionServerMetrics {
       p.add(cf, qualifier, val);
       puts.add(p);
     }
-    try (HTable t = TEST_UTIL.createTable(tableName, cf)) {
+    try (Table t = TEST_UTIL.createTable(tableName, cf)) {
       t.put(puts);
 
       Scan s = new Scan();

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index d150e01..592bf1e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -58,7 +60,7 @@ import com.google.protobuf.ServiceException;
 public class TestRegionServerNoMaster {
 
   private static final int NB_SERVERS = 1;
-  private static HTable table;
+  private static Table table;
   private static final byte[] row = "ee".getBytes();
 
   private static HRegionInfo hri;
@@ -78,7 +80,9 @@ public class TestRegionServerNoMaster {
     p.add(HConstants.CATALOG_FAMILY, row, row);
     table.put(p);
 
-    hri = table.getRegionLocation(row, false).getRegionInfo();
+    try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
+      hri = locator.getRegionLocation(row, false).getRegionInfo();
+    }
     regionName = hri.getRegionName();
 
     stopMasterAndAssignMeta(HTU);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
index 9c6ee1a..9a8a4d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerOnlineConfigChange.java
@@ -28,7 +28,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -52,14 +55,14 @@ public class TestRegionServerOnlineConfigChange {
   private static HBaseTestingUtility hbaseTestingUtility = new HBaseTestingUtility();
   private static Configuration conf = null;
 
-  private static HTable t1 = null;
+  private static Table t1 = null;
   private static HRegionServer rs1 = null;
   private static byte[] r1name = null;
   private static Region r1 = null;
 
   private final static String table1Str = "table1";
   private final static String columnFamily1Str = "columnFamily1";
-  private final static byte[] TABLE1 = Bytes.toBytes(table1Str);
+  private final static TableName TABLE1 = TableName.valueOf(table1Str);
   private final static byte[] COLUMN_FAMILY1 = Bytes.toBytes(columnFamily1Str);
 
 
@@ -68,12 +71,13 @@ public class TestRegionServerOnlineConfigChange {
     conf = hbaseTestingUtility.getConfiguration();
     hbaseTestingUtility.startMiniCluster(1,1);
     t1 = hbaseTestingUtility.createTable(TABLE1, COLUMN_FAMILY1);
-    @SuppressWarnings("deprecation")
-    HRegionInfo firstHRI = t1.getRegionLocations().keySet().iterator().next();
-    r1name = firstHRI.getRegionName();
-    rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer(
-        hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
-    r1 = rs1.getRegion(r1name);
+    try (RegionLocator locator = hbaseTestingUtility.getConnection().getRegionLocator(TABLE1)) {
+      HRegionInfo firstHRI = locator.getAllRegionLocations().get(0).getRegionInfo();
+      r1name = firstHRI.getRegionName();
+      rs1 = hbaseTestingUtility.getHBaseCluster().getRegionServer(
+          hbaseTestingUtility.getHBaseCluster().getServerWith(r1name));
+      r1 = rs1.getRegion(r1name);
+    }
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
index 86515a6..ef7f105 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -83,7 +84,9 @@ public class TestScannerWithBulkload {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
     final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
-    bulkload.doBulkLoad(hfilePath, (HTable) table);
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      bulkload.doBulkLoad(hfilePath, admin, table, locator);
+    }
     ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     result = scanAfterBulkLoad(scanner, result, "version2");
@@ -168,8 +171,8 @@ public class TestScannerWithBulkload {
   private Table init(HBaseAdmin admin, long l, Scan scan, TableName tableName) throws Exception {
     Table table = TEST_UTIL.getConnection().getTable(tableName);
     Put put0 = new Put(Bytes.toBytes("row1"));
-    put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l, Bytes
-        .toBytes("version0")));
+    put0.add(new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
+        Bytes.toBytes("version0")));
     table.put(put0);
     admin.flush(tableName);
     Put put1 = new Put(Bytes.toBytes("row2"));
@@ -195,9 +198,9 @@ public class TestScannerWithBulkload {
 
   @Test
   public void testBulkLoadWithParallelScan() throws Exception {
-    TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan");
+    final TableName tableName = TableName.valueOf("testBulkLoadWithParallelScan");
       final long l = System.currentTimeMillis();
-    HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+    final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
     createTable(admin, tableName);
     Scan scan = createScan();
     final Table table = init(admin, l, scan, tableName);
@@ -217,7 +220,9 @@ public class TestScannerWithBulkload {
           put1.add(new KeyValue(Bytes.toBytes("row5"), Bytes.toBytes("col"), Bytes.toBytes("q"), l,
               Bytes.toBytes("version0")));
           table.put(put1);
-          bulkload.doBulkLoad(hfilePath, (HTable) table);
+          try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+            bulkload.doBulkLoad(hfilePath, admin, table, locator);
+          }
           latch.countDown();
         } catch (TableNotFoundException e) {
         } catch (IOException e) {
@@ -231,7 +236,6 @@ public class TestScannerWithBulkload {
     scanAfterBulkLoad(scanner, result, "version1");
     scanner.close();
     table.close();
-
   }
 
   @Test
@@ -248,7 +252,9 @@ public class TestScannerWithBulkload {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setBoolean("hbase.mapreduce.bulkload.assign.sequenceNumbers", true);
     final LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
-    bulkload.doBulkLoad(hfilePath, (HTable) table);
+    try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
+      bulkload.doBulkLoad(hfilePath, admin, table, locator);
+    }
     ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     // We had 'version0', 'version1' for 'row1,col:q' in the table.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java
index 5ce4456..6578d74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerCustomProtocol.java
@@ -156,18 +156,18 @@ public class TestServerCustomProtocol {
   @Before
   public void before()  throws Exception {
     final byte[][] SPLIT_KEYS = new byte[][] { ROW_B, ROW_C };
-    HTable table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
+    Table table = util.createTable(TEST_TABLE, TEST_FAMILY, SPLIT_KEYS);
 
     Put puta = new Put( ROW_A );
-    puta.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
+    puta.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
     table.put(puta);
 
     Put putb = new Put( ROW_B );
-    putb.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
+    putb.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
     table.put(putb);
 
     Put putc = new Put( ROW_C );
-    putc.add(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
+    putc.addColumn(TEST_FAMILY, Bytes.toBytes("col1"), Bytes.toBytes(1));
     table.put(putc);
   }
 
@@ -306,8 +306,8 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testSingleMethod() throws Throwable {
-    try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
-      RegionLocator locator = table.getRegionLocator();
+    try (Table table = util.getConnection().getTable(TEST_TABLE);
+        RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
       Map<byte [], String> results = table.coprocessorService(PingProtos.PingService.class,
         null, ROW_A,
         new Batch.Call<PingProtos.PingService, String>() {
@@ -335,10 +335,11 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testRowRange() throws Throwable {
-    try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
-      RegionLocator locator = table.getRegionLocator();
-      for (Entry<HRegionInfo, ServerName> e: table.getRegionLocations().entrySet()) {
-        LOG.info("Region " + e.getKey().getRegionNameAsString() + ", servername=" + e.getValue());
+    try (Table table = util.getConnection().getTable(TEST_TABLE);
+        RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
+      for (HRegionLocation e: locator.getAllRegionLocations()) {
+        LOG.info("Region " + e.getRegionInfo().getRegionNameAsString()
+            + ", servername=" + e.getServerName());
       }
       // Here are what regions looked like on a run:
       //
@@ -355,7 +356,7 @@ public class TestServerCustomProtocol {
       results = ping(table, ROW_BC, null);
       assertEquals(2, results.size());
       // should contain last 2 regions
-      HRegionLocation loc = table.getRegionLocation(ROW_A, true);
+      HRegionLocation loc = locator.getRegionLocation(ROW_A, true);
       assertNull("Should be missing region for row aaa (prior to start row)",
         results.get(loc.getRegionInfo().getRegionName()));
       verifyRegionResults(locator, results, ROW_B);
@@ -367,7 +368,7 @@ public class TestServerCustomProtocol {
       assertEquals(2, results.size());
       verifyRegionResults(locator, results, ROW_A);
       verifyRegionResults(locator, results, ROW_B);
-      loc = table.getRegionLocation(ROW_C, true);
+      loc = locator.getRegionLocation(ROW_C, true);
       assertNull("Should be missing region for row ccc (past stop row)",
           results.get(loc.getRegionInfo().getRegionName()));
   
@@ -377,7 +378,7 @@ public class TestServerCustomProtocol {
       assertEquals(2, results.size());
       verifyRegionResults(locator, results, ROW_A);
       verifyRegionResults(locator, results, ROW_B);
-      loc = table.getRegionLocation(ROW_C, true);
+      loc = locator.getRegionLocation(ROW_C, true);
       assertNull("Should be missing region for row ccc (past stop row)",
           results.get(loc.getRegionInfo().getRegionName()));
   
@@ -386,10 +387,10 @@ public class TestServerCustomProtocol {
       // should only contain region bbb
       assertEquals(1, results.size());
       verifyRegionResults(locator, results, ROW_B);
-      loc = table.getRegionLocation(ROW_A, true);
+      loc = locator.getRegionLocation(ROW_A, true);
       assertNull("Should be missing region for row aaa (prior to start)",
           results.get(loc.getRegionInfo().getRegionName()));
-      loc = table.getRegionLocation(ROW_C, true);
+      loc = locator.getRegionLocation(ROW_C, true);
       assertNull("Should be missing region for row ccc (past stop row)",
           results.get(loc.getRegionInfo().getRegionName()));
     }
@@ -415,8 +416,8 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testCompoundCall() throws Throwable {
-    try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
-      RegionLocator locator = table.getRegionLocator();
+    try (Table table = util.getConnection().getTable(TEST_TABLE);
+        RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
       Map<byte [], String> results = compoundOfHelloAndPing(table, ROW_A, ROW_C);
       verifyRegionResults(locator, results, "Hello, pong", ROW_A);
       verifyRegionResults(locator, results, "Hello, pong", ROW_B);
@@ -426,8 +427,8 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testNullCall() throws Throwable {
-    try(HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
-      RegionLocator locator = table.getRegionLocator();
+    try (Table table = util.getConnection().getTable(TEST_TABLE);
+        RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
       Map<byte[],String> results = hello(table, null, ROW_A, ROW_C);
       verifyRegionResults(locator, results, "Who are you?", ROW_A);
       verifyRegionResults(locator, results, "Who are you?", ROW_B);
@@ -437,8 +438,8 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testNullReturn() throws Throwable {
-    try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
-      RegionLocator locator = table.getRegionLocator();
+    try (Table table = util.getConnection().getTable(TEST_TABLE);
+        RegionLocator locator = util.getConnection().getRegionLocator(TEST_TABLE)) {
       Map<byte[],String> results = hello(table, "nobody", ROW_A, ROW_C);
       verifyRegionResults(locator, results, null, ROW_A);
       verifyRegionResults(locator, results, null, ROW_B);
@@ -448,7 +449,7 @@ public class TestServerCustomProtocol {
 
   @Test
   public void testEmptyReturnType() throws Throwable {
-    try (HTable table = (HTable) util.getConnection().getTable(TEST_TABLE)) {
+    try (Table table = util.getConnection().getTable(TEST_TABLE)) {
       Map<byte[],String> results = noop(table, ROW_A, ROW_C);
       assertEquals("Should have results from three regions", 3, results.size());
       // all results should be null

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 86d196e..2fe5654 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -298,7 +298,7 @@ public class TestSplitTransactionOnCluster {
         TableName.valueOf("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
 
     // Create table then get the single region for our new table.
-    HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
+    Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
     List<HRegion> regions = cluster.getRegions(tableName);
     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
 
@@ -343,7 +343,7 @@ public class TestSplitTransactionOnCluster {
         TableName.valueOf("testExistingZnodeBlocksSplitAndWeRollback");
 
     // Create table then get the single region for our new table.
-    HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
+    Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
     List<HRegion> regions = cluster.getRegions(tableName);
     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
 
@@ -401,7 +401,7 @@ public class TestSplitTransactionOnCluster {
         TableName.valueOf("testShutdownFixupWhenDaughterHasSplit");
 
     // Create table then get the single region for our new table.
-    HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
+    Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
     List<HRegion> regions = cluster.getRegions(tableName);
     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
 
@@ -567,7 +567,7 @@ public class TestSplitTransactionOnCluster {
         .valueOf("testMasterRestartAtRegionSplitPendingCatalogJanitor");
 
     // Create table then get the single region for our new table.
-    HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
+    Table t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
     List<HRegion> regions = cluster.getRegions(tableName);
     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
 
@@ -683,8 +683,7 @@ public class TestSplitTransactionOnCluster {
     htd.setRegionReplication(2);
     htd.addCoprocessor(SlowMeCopro.class.getName());
     // Create table then get the single region for our new table.
-    Table t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")},
-        TESTING_UTIL.getConfiguration());
+    Table t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")}, null);
     List<HRegion> oldRegions;
     do {
       oldRegions = cluster.getRegions(tableName);
@@ -965,7 +964,7 @@ public class TestSplitTransactionOnCluster {
       desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
       admin.createTable(desc);
       Connection connection = ConnectionFactory.createConnection(cluster.getConfiguration());
-      HTable hTable = (HTable) connection.getTable(desc.getTableName());
+      Table hTable = connection.getTable(desc.getTableName());
       for(int i = 1; i < 5; i++) {
         Put p1 = new Put(("r"+i).getBytes());
         p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
@@ -1266,9 +1265,9 @@ public class TestSplitTransactionOnCluster {
     return regions;
   }
 
-  private HTable createTableAndWait(TableName tableName, byte[] cf) throws IOException,
+  private Table createTableAndWait(TableName tableName, byte[] cf) throws IOException,
       InterruptedException {
-    HTable t = TESTING_UTIL.createTable(tableName, cf);
+    Table t = TESTING_UTIL.createTable(tableName, cf);
     awaitTableRegions(tableName);
     assertTrue("Table not online: " + tableName,
       cluster.getRegions(tableName).size() != 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java
index b2f2898..93c8ebc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactionWithThroughputController.java
@@ -87,7 +87,7 @@ public class TestCompactionWithThroughputController {
       admin.disableTable(tableName);
       admin.deleteTable(tableName);
     }
-    HTable table = TEST_UTIL.createTable(tableName, family);
+    Table table = TEST_UTIL.createTable(tableName, family);
     Random rand = new Random();
     for (int i = 0; i < 10; i++) {
       for (int j = 0; j < 10; j++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
index 6a6cf21..de93305 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
@@ -55,7 +55,7 @@ public class TestReplicationKillRS extends TestReplicationBase {
     Thread killer = killARegionServer(util, 5000, rsToKill1);
 
     LOG.info("Start loading table");
-    int initialCount = utility1.loadTable((HTable)htable1, famName);
+    int initialCount = utility1.loadTable(htable1, famName);
     LOG.info("Done loading table");
     killer.join(5000);
     LOG.info("Done waiting for threads");

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 930ffba..fe21070 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.BaseWALObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -86,7 +88,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
   private static final int NB_SERVERS = 2;
   private static TableName tableName = TableName.valueOf(
     TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName());
-  private static HTable table;
+  private static Table table;
   private static final byte[] row = "TestRegionReplicaReplicator".getBytes();
 
   private static HRegionServer rs0;
@@ -117,10 +119,12 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     HTU.startMiniCluster(NB_SERVERS);
 
     // Create table then get the single region for our new table.
-    HTableDescriptor htd = HTU.createTableDescriptor(tableName.toString());
-    table = HTU.createTable(htd, new byte[][]{f}, HTU.getConfiguration());
+    HTableDescriptor htd = HTU.createTableDescriptor(tableName.getNameAsString());
+    table = HTU.createTable(htd, new byte[][]{f}, null);
 
-    hriPrimary = table.getRegionLocation(row, false).getRegionInfo();
+    try (RegionLocator locator = HTU.getConnection().getRegionLocator(tableName)) {
+      hriPrimary = locator.getRegionLocation(row, false).getRegionInfo();
+    }
 
     // mock a secondary region info to open
     hriSecondary = new HRegionInfo(hriPrimary.getTable(), hriPrimary.getStartKey(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 77f9bc7..6d7a2ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -955,10 +955,12 @@ public class TestAccessController extends SecureTestUtil {
 
 
       try (Connection conn = ConnectionFactory.createConnection(conf);
-           HTable table = (HTable)conn.getTable(tableName)) {
+           Admin admin = conn.getAdmin();
+           RegionLocator locator = conn.getRegionLocator(tableName);
+           Table table = conn.getTable(tableName)) {
         TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
         LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
-        loader.doBulkLoad(loadPath, table);
+        loader.doBulkLoad(loadPath, admin, table, locator);
       }
     }
 
@@ -1964,7 +1966,7 @@ public class TestAccessController extends SecureTestUtil {
     HRegionLocation location = regions.get(0);
     final HRegionInfo hri = location.getRegionInfo();
     final ServerName server = location.getServerName();
-    try (HTable table = (HTable) systemUserConnection.getTable(TEST_TABLE2)) {
+    try (Table table = systemUserConnection.getTable(TEST_TABLE2)) {
       AccessTestAction moveAction = new AccessTestAction() {
         @Override
         public Object run() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index 0ff87af..a285eae 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -144,7 +145,7 @@ public class TestHBaseFsck {
   private static Admin admin;
 
   // for the instance, reset every test run
-  private HTable tbl;
+  private Table tbl;
   private final static byte[][] SPLITS = new byte[][] { Bytes.toBytes("A"),
     Bytes.toBytes("B"), Bytes.toBytes("C") };
   // one row per region.
@@ -155,7 +156,7 @@ public class TestHBaseFsck {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
-      MasterSyncObserver.class.getName());
+        MasterSyncObserver.class.getName());
 
     conf.setInt("hbase.regionserver.handler.count", 2);
     conf.setInt("hbase.regionserver.metahandler.count", 30);
@@ -326,7 +327,8 @@ public class TestHBaseFsck {
   private void deleteRegion(Configuration conf, final HTableDescriptor htd,
       byte[] startKey, byte[] endKey, boolean unassign, boolean metaRow,
       boolean hdfs) throws IOException, InterruptedException {
-    deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false, HRegionInfo.DEFAULT_REPLICA_ID);
+    deleteRegion(conf, htd, startKey, endKey, unassign, metaRow, hdfs, false,
+        HRegionInfo.DEFAULT_REPLICA_ID);
   }
 
   /**
@@ -344,7 +346,11 @@ public class TestHBaseFsck {
     LOG.info("** Before delete:");
     dumpMeta(htd.getTableName());
 
-    List<HRegionLocation> locations = tbl.getAllRegionLocations();
+    List<HRegionLocation> locations;
+    try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+      locations = rl.getAllRegionLocations();
+    }
+
     for (HRegionLocation location : locations) {
       HRegionInfo hri = location.getRegionInfo();
       ServerName hsa = location.getServerName();
@@ -426,15 +432,14 @@ public class TestHBaseFsck {
     desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
     createTable(TEST_UTIL, desc, SPLITS);
 
-    tbl = (HTable) connection.getTable(tablename, tableExecutorService);
+    tbl = connection.getTable(tablename, tableExecutorService);
     List<Put> puts = new ArrayList<Put>();
     for (byte[] row : ROWKEYS) {
       Put p = new Put(row);
-      p.add(FAM, Bytes.toBytes("val"), row);
+      p.addColumn(FAM, Bytes.toBytes("val"), row);
       puts.add(p);
     }
     tbl.put(puts);
-    tbl.flushCommits();
   }
 
   /**
@@ -683,13 +688,12 @@ public class TestHBaseFsck {
       TEST_UTIL.assertRegionOnServer(hriDupe, server, REGION_ONLINE_TIMEOUT);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
-            ERROR_CODE.DUPE_STARTKEYS});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS });
       assertEquals(2, hbck.getOverlapGroups(table).size());
       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
 
       // fix the degenerate region.
-      doFsck(conf,true);
+      doFsck(conf, true);
 
       // check that the degenerate region is gone and no data loss
       HBaseFsck hbck2 = doFsck(conf,false);
@@ -727,16 +731,16 @@ public class TestHBaseFsck {
       admin.flush(table);
       assertNoErrors(doFsck(conf, false));
       assertEquals(ROWKEYS.length, countRows());
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), true, false, false, false, 1); // unassign one replica
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+          false, false, false, 1); // unassign one replica
       // check that problem exists
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[]{ERROR_CODE.NOT_DEPLOYED});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED });
       // fix the problem
       hbck = doFsck(conf, true);
       // run hbck again to make sure we don't see any errors
       hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[]{});
+      assertErrors(hbck, new ERROR_CODE[] {});
     } finally {
       cleanupTable(table);
     }
@@ -868,13 +872,12 @@ public class TestHBaseFsck {
 
       // TODO why is dupe region different from dupe start keys?
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS,
-            ERROR_CODE.DUPE_STARTKEYS});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.DUPE_STARTKEYS, ERROR_CODE.DUPE_STARTKEYS });
       assertEquals(2, hbck.getOverlapGroups(table).size());
       assertEquals(ROWKEYS.length, countRows()); // seems like the "bigger" region won.
 
       // fix the degenerate region.
-      doFsck(conf,true);
+      doFsck(conf, true);
 
       // check that the degenerate region is gone and no data loss
       HBaseFsck hbck2 = doFsck(conf,false);
@@ -894,7 +897,7 @@ public class TestHBaseFsck {
     TableName table = TableName.valueOf("tableDegenerateRegions");
     try {
       setupTable(table);
-      assertNoErrors(doFsck(conf,false));
+      assertNoErrors(doFsck(conf, false));
       assertEquals(ROWKEYS.length, countRows());
 
       // Now let's mess it up, by adding a region with a duplicate startkey
@@ -913,7 +916,7 @@ public class TestHBaseFsck {
       assertEquals(ROWKEYS.length, countRows());
 
       // fix the degenerate region.
-      doFsck(conf,true);
+      doFsck(conf, true);
 
       // check that the degenerate region is gone and no data loss
       HBaseFsck hbck2 = doFsck(conf,false);
@@ -947,8 +950,7 @@ public class TestHBaseFsck {
       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.OVERLAP_IN_REGION_CHAIN });
       assertEquals(2, hbck.getOverlapGroups(table).size());
       assertEquals(ROWKEYS.length, countRows());
 
@@ -1073,8 +1075,8 @@ public class TestHBaseFsck {
 
       // Mess it up by creating an overlap in the metadata
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-          Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
+          true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
       admin.enableTable(table);
 
       HRegionInfo hriOverlap =
@@ -1086,9 +1088,9 @@ public class TestHBaseFsck {
       TEST_UTIL.assertRegionOnServer(hriOverlap, server, REGION_ONLINE_TIMEOUT);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-          ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck,
+          new ERROR_CODE[] { ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+              ERROR_CODE.HOLE_IN_REGION_CHAIN });
 
       // fix the problem.
       doFsck(conf, true);
@@ -1158,13 +1160,12 @@ public class TestHBaseFsck {
 
       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), true, true, true);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+          true, true);
       admin.enableTable(table);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
       // holes are separate from overlap groups
       assertEquals(0, hbck.getOverlapGroups(table).size());
 
@@ -1173,7 +1174,7 @@ public class TestHBaseFsck {
 
       // check that hole fixed
       assertNoErrors(doFsck(conf,false));
-      assertEquals(ROWKEYS.length - 2 , countRows()); // lost a region so lost a row
+      assertEquals(ROWKEYS.length - 2, countRows()); // lost a region so lost a row
     } finally {
       cleanupTable(table);
     }
@@ -1192,15 +1193,14 @@ public class TestHBaseFsck {
 
       // Mess it up by leaving a hole in the meta data
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+          true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
       admin.enableTable(table);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.ORPHAN_HDFS_REGION,
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-          ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck,
+          new ERROR_CODE[] { ERROR_CODE.ORPHAN_HDFS_REGION, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+              ERROR_CODE.HOLE_IN_REGION_CHAIN });
       // holes are separate from overlap groups
       assertEquals(0, hbck.getOverlapGroups(table).size());
 
@@ -1229,22 +1229,22 @@ public class TestHBaseFsck {
 
       // Mess it up by leaving a hole in the meta data
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), true, true, false); // don't rm from fs
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), true,
+          true, false); // don't rm from fs
       admin.enableTable(table);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck,
+          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
       // holes are separate from overlap groups
       assertEquals(0, hbck.getOverlapGroups(table).size());
 
       // fix hole
-      assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(doFsck(conf, true),
+          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
 
       // check that hole fixed
-      assertNoErrors(doFsck(conf,false));
+      assertNoErrors(doFsck(conf, false));
       assertEquals(ROWKEYS.length, countRows());
     } finally {
       cleanupTable(table);
@@ -1260,7 +1260,7 @@ public class TestHBaseFsck {
       desc.addFamily(new HColumnDescriptor(Bytes.toBytes("f")));
       createTable(TEST_UTIL, desc, null);
 
-      tbl = (HTable) connection.getTable(desc.getTableName());
+      tbl = connection.getTable(desc.getTableName());
       for (int i = 0; i < 5; i++) {
         Put p1 = new Put(("r" + i).getBytes());
         p1.add(Bytes.toBytes("f"), "q1".getBytes(), "v".getBytes());
@@ -1293,9 +1293,9 @@ public class TestHBaseFsck {
 
       // fix hole
       assertErrors(
-        doFsck(conf, false, true, false, false, false, false, false, false, false, false, null),
-        new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
+          doFsck(conf, false, true, false, false, false, false, false, false, false, false, null),
+          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+              ERROR_CODE.NOT_IN_META_OR_DEPLOYED });
 
       // check that hole fixed
       assertNoErrors(doFsck(conf, false));
@@ -1322,22 +1322,22 @@ public class TestHBaseFsck {
 
       // Mess it up by leaving a hole in the meta data
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), false, true, false); // don't rm from fs
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+          true, false); // don't rm from fs
       admin.enableTable(table);
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck,
+          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
       // holes are separate from overlap groups
       assertEquals(0, hbck.getOverlapGroups(table).size());
 
       // fix hole
-      assertErrors(doFsck(conf, true) , new ERROR_CODE[] {
-          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(doFsck(conf, true),
+          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
 
       // check that hole fixed
-      assertNoErrors(doFsck(conf,false));
+      assertNoErrors(doFsck(conf, false));
       assertEquals(ROWKEYS.length, countRows());
     } finally {
       cleanupTable(table);
@@ -1360,8 +1360,8 @@ public class TestHBaseFsck {
       admin.flush(table);
 
       // Mess it up by leaving a hole in the hdfs data
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), false, false, true); // don't rm meta
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+          false, true); // don't rm meta
 
       HBaseFsck hbck = doFsck(conf, false);
       assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
@@ -1407,17 +1407,17 @@ public class TestHBaseFsck {
       admin.flush(table);
 
       // Mess it up by leaving a hole in the hdfs data
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-          Bytes.toBytes("C"), false, false, true); // don't rm meta
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+          false, true); // don't rm meta
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
 
       // fix hole
       doFsck(conf, true);
 
       // check that hole fixed
-      assertNoErrors(doFsck(conf,false));
+      assertNoErrors(doFsck(conf, false));
       assertEquals(ROWKEYS.length - 2, countRows());
 
       // the following code checks whether the old primary/secondary has
@@ -1633,20 +1633,18 @@ public class TestHBaseFsck {
       // make sure data in regions, if in wal only there is no data loss
       admin.flush(table2);
       // Mess them up by leaving a hole in the hdfs data
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
-        Bytes.toBytes("C"), false, false, true); // don't rm meta
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"), Bytes.toBytes("C"), false,
+          false, true); // don't rm meta
 
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS});
+      assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS, ERROR_CODE.NOT_IN_HDFS });
 
       // fix hole in table 1
       doFsck(conf, true, table1);
       // check that hole in table 1 fixed
       assertNoErrors(doFsck(conf, false, table1));
       // check that hole in table 2 still there
-      assertErrors(doFsck(conf, false, table2),
-        new ERROR_CODE[] {ERROR_CODE.NOT_IN_HDFS});
+      assertErrors(doFsck(conf, false, table2), new ERROR_CODE[] { ERROR_CODE.NOT_IN_HDFS });
 
       // fix hole in table 2
       doFsck(conf, true, table2);
@@ -1672,7 +1670,11 @@ public class TestHBaseFsck {
 
       // make sure data in regions, if in wal only there is no data loss
       admin.flush(table);
-      HRegionLocation location = tbl.getRegionLocation("B");
+
+      HRegionLocation location;
+      try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+        location = rl.getRegionLocation(Bytes.toBytes("B"));
+      }
 
       // Delete one region from meta, but not hdfs, unassign it.
       deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("B"),
@@ -1752,33 +1754,38 @@ public class TestHBaseFsck {
 
       // make sure data in regions, if in wal only there is no data loss
       admin.flush(table);
-      HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
 
-      meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
-      HRegionInfo hri = location.getRegionInfo();
+      try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+        HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
 
-      // do a regular split
-      byte[] regionName = location.getRegionInfo().getRegionName();
-      admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
-      TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
+        meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
+        HRegionInfo hri = location.getRegionInfo();
 
-      // TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
-      // for some time until children references are deleted. HBCK erroneously sees this as
-      // overlapping regions
-      HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, false, false, null);
-      assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
+        // do a regular split
+        byte[] regionName = location.getRegionInfo().getRegionName();
+        admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
+        TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
 
-      // assert that the split hbase:meta entry is still there.
-      Get get = new Get(hri.getRegionName());
-      Result result = meta.get(get);
-      assertNotNull(result);
-      assertNotNull(MetaTableAccessor.getHRegionInfo(result));
+        // TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
+        // for some time until children references are deleted. HBCK erroneously sees this as
+        // overlapping regions
+        HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, false,
+            false, null);
+        assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
 
-      assertEquals(ROWKEYS.length, countRows());
+        // assert that the split hbase:meta entry is still there.
+        Get get = new Get(hri.getRegionName());
+        Result result = meta.get(get);
+        assertNotNull(result);
+        assertNotNull(MetaTableAccessor.getHRegionInfo(result));
 
-      // assert that we still have the split regions
-      assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
-      assertNoErrors(doFsck(conf, false));
+        assertEquals(ROWKEYS.length, countRows());
+
+        // assert that we still have the split regions
+        assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions
+        // pre-split.
+        assertNoErrors(doFsck(conf, false));
+      }
     } finally {
       cleanupTable(table);
       IOUtils.closeQuietly(meta);
@@ -1799,56 +1806,68 @@ public class TestHBaseFsck {
 
       // make sure data in regions, if in wal only there is no data loss
       admin.flush(table);
-      HRegionLocation location = tbl.getRegionLocation(Bytes.toBytes("B"));
-
-      HRegionInfo hri = location.getRegionInfo();
-
-      // do a regular split
-      byte[] regionName = location.getRegionInfo().getRegionName();
-      admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
-      TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
-
-      PairOfSameType<HRegionInfo> daughters =
-          MetaTableAccessor.getDaughterRegions(meta.get(new Get(regionName)));
-
-      // Delete daughter regions from meta, but not hdfs, unassign it.
-      Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
-      undeployRegion(connection, hris.get(daughters.getFirst()), daughters.getFirst());
-      undeployRegion(connection, hris.get(daughters.getSecond()), daughters.getSecond());
-
-      List<Delete> deletes = new ArrayList<>();
-      deletes.add(new Delete(daughters.getFirst().getRegionName()));
-      deletes.add(new Delete(daughters.getSecond().getRegionName()));
-      meta.delete(deletes);
-
-      // Remove daughters from regionStates
-      RegionStates regionStates = TEST_UTIL.getMiniHBaseCluster().getMaster().
-          getAssignmentManager().getRegionStates();
-      regionStates.deleteRegion(daughters.getFirst());
-      regionStates.deleteRegion(daughters.getSecond());
-
-      HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck,
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-              ERROR_CODE.HOLE_IN_REGION_CHAIN }); //no LINGERING_SPLIT_PARENT
-
-      // now fix it. The fix should not revert the region split, but add daughters to META
-      hbck = doFsck(conf, true, true, false, false, false, false, false, false, false, false, null);
-      assertErrors(hbck,
-          new ERROR_CODE[] { ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
-              ERROR_CODE.HOLE_IN_REGION_CHAIN });
-
-      // assert that the split hbase:meta entry is still there.
-      Get get = new Get(hri.getRegionName());
-      Result result = meta.get(get);
-      assertNotNull(result);
-      assertNotNull(MetaTableAccessor.getHRegionInfo(result));
-
-      assertEquals(ROWKEYS.length, countRows());
 
-      // assert that we still have the split regions
-      assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
-      assertNoErrors(doFsck(conf, false)); //should be fixed by now
+      try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+        HRegionLocation location = rl.getRegionLocation(Bytes.toBytes("B"));
+
+        HRegionInfo hri = location.getRegionInfo();
+
+        // do a regular split
+        byte[] regionName = location.getRegionInfo().getRegionName();
+        admin.splitRegion(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
+        TestEndToEndSplitTransaction.blockUntilRegionSplit(conf, 60000, regionName, true);
+
+        PairOfSameType<HRegionInfo> daughters = MetaTableAccessor.getDaughterRegions(
+            meta.get(new Get(regionName)));
+
+        // Delete daughter regions from meta, but not hdfs, unassign it.
+
+        ServerName firstSN =
+            rl.getRegionLocation(daughters.getFirst().getStartKey()).getServerName();
+        ServerName secondSN =
+            rl.getRegionLocation(daughters.getSecond().getStartKey()).getServerName();
+
+        undeployRegion(connection, firstSN, daughters.getFirst());
+        undeployRegion(connection, secondSN, daughters.getSecond());
+
+        List<Delete> deletes = new ArrayList<>();
+        deletes.add(new Delete(daughters.getFirst().getRegionName()));
+        deletes.add(new Delete(daughters.getSecond().getRegionName()));
+        meta.delete(deletes);
+
+        // Remove daughters from regionStates
+        RegionStates regionStates = TEST_UTIL.getMiniHBaseCluster().getMaster().
+            getAssignmentManager().getRegionStates();
+        regionStates.deleteRegion(daughters.getFirst());
+        regionStates.deleteRegion(daughters.getSecond());
+
+        HBaseFsck hbck = doFsck(conf, false);
+        assertErrors(hbck, new ERROR_CODE[] {
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.HOLE_IN_REGION_CHAIN }); //no LINGERING_SPLIT_PARENT
+
+        // now fix it. The fix should not revert the region split, but add daughters to META
+        hbck = doFsck(conf, true, true, false, false, false, false, false, false, false,
+            false, null);
+        assertErrors(hbck, new ERROR_CODE[] {
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+            ERROR_CODE.HOLE_IN_REGION_CHAIN });
+
+        // assert that the split hbase:meta entry is still there.
+        Get get = new Get(hri.getRegionName());
+        Result result = meta.get(get);
+        assertNotNull(result);
+        assertNotNull(MetaTableAccessor.getHRegionInfo(result));
+
+        assertEquals(ROWKEYS.length, countRows());
+
+        // assert that we still have the split regions
+        assertEquals(rl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions
+        // pre-split.
+        assertNoErrors(doFsck(conf, false)); //should be fixed by now
+      }
     } finally {
       meta.close();
       cleanupTable(table);
@@ -1952,13 +1971,13 @@ public class TestHBaseFsck {
       assertEquals(ROWKEYS.length, countRows());
 
       // Mess it up by closing a region
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-        Bytes.toBytes("B"), true, false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
+          false, false, false, HRegionInfo.DEFAULT_REPLICA_ID);
 
       // verify there is no other errors
       HBaseFsck hbck = doFsck(conf, false);
-      assertErrors(hbck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(hbck,
+          new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
 
       // verify that noHdfsChecking report the same errors
       HBaseFsck fsck = new HBaseFsck(conf, hbfsckExecutorService);
@@ -1967,8 +1986,8 @@ public class TestHBaseFsck {
       fsck.setTimeLag(0);
       fsck.setCheckHdfs(false);
       fsck.onlineHbck();
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(fsck,
+          new ERROR_CODE[] { ERROR_CODE.NOT_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN });
       fsck.close();
 
       // verify that fixAssignments works fine with noHdfsChecking
@@ -2062,8 +2081,8 @@ public class TestHBaseFsck {
 
       // Mess it up by creating an overlap in the metadata
       admin.disableTable(table);
-      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"),
-        Bytes.toBytes("B"), true, true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
+      deleteRegion(conf, tbl.getTableDescriptor(), Bytes.toBytes("A"), Bytes.toBytes("B"), true,
+          true, false, true, HRegionInfo.DEFAULT_REPLICA_ID);
       admin.enableTable(table);
 
       HRegionInfo hriOverlap =
@@ -2086,8 +2105,7 @@ public class TestHBaseFsck {
       fsck.setTimeLag(0);
       fsck.setCheckHdfs(false);
       fsck.onlineHbck();
-      assertErrors(fsck, new ERROR_CODE[] {
-        ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
       fsck.close();
 
       // verify that fixHdfsHoles doesn't work with noHdfsChecking
@@ -2101,7 +2119,7 @@ public class TestHBaseFsck {
       fsck.setFixHdfsOrphans(true);
       fsck.onlineHbck();
       assertFalse(fsck.shouldRerun());
-      assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN});
+      assertErrors(fsck, new ERROR_CODE[] { ERROR_CODE.HOLE_IN_REGION_CHAIN });
       fsck.close();
     } finally {
       if (admin.isTableDisabled(table)) {
@@ -2229,7 +2247,8 @@ public class TestHBaseFsck {
     final FileSystem fs = FileSystem.get(conf);
     HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
       @Override
-      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
+      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
+          throws IOException {
         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
           AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
           @Override
@@ -2259,7 +2278,8 @@ public class TestHBaseFsck {
     final FileSystem fs = FileSystem.get(conf);
     HBaseFsck hbck = new HBaseFsck(conf, hbfsckExecutorService) {
       @Override
-      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles) throws IOException {
+      public HFileCorruptionChecker createHFileCorruptionChecker(boolean sidelineCorruptHFiles)
+          throws IOException {
         return new HFileCorruptionChecker(conf, executor, sidelineCorruptHFiles) {
           AtomicBoolean attemptedFirstHFile = new AtomicBoolean(false);
           @Override
@@ -2529,16 +2549,20 @@ public class TestHBaseFsck {
     Threads.sleep(300); // wait some more to ensure writeLock.acquire() is called
 
     hbck = doFsck(conf, false);
-    assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK}); // still one expired, one not-expired
+    assertErrors(hbck, new ERROR_CODE[] {
+        ERROR_CODE.EXPIRED_TABLE_LOCK}); // still one expired, one not-expired
 
     edge.incrementTime(conf.getLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT,
         TableLockManager.DEFAULT_TABLE_LOCK_EXPIRE_TIMEOUT_MS)); // let table lock expire
 
     hbck = doFsck(conf, false);
-    assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK, ERROR_CODE.EXPIRED_TABLE_LOCK}); // both are expired
+    assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.EXPIRED_TABLE_LOCK,
+        ERROR_CODE.EXPIRED_TABLE_LOCK}); // both are expired
+
+    conf.setLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT, 1);
+    // reaping from ZKInterProcessWriteLock uses znode cTime,
+    // which is not injectable through EnvironmentEdge
 
-    conf.setLong(TableLockManager.TABLE_LOCK_EXPIRE_TIMEOUT, 1); // reaping from ZKInterProcessWriteLock uses znode cTime,
-                                                                 // which is not injectable through EnvironmentEdge
     Threads.sleep(10);
     hbck = doFsck(conf, true); // now fix both cases
 
@@ -2616,7 +2640,7 @@ public class TestHBaseFsck {
       HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toString(FAM));
       desc.addFamily(hcd); // If a table has no CF's it doesn't get checked
       createTable(TEST_UTIL, desc, null);
-      tbl = (HTable) connection.getTable(table, tableExecutorService);
+      tbl = connection.getTable(table, tableExecutorService);
 
       // Mess it up by leaving a hole in the assignment, meta, and hdfs data
       deleteRegion(conf, tbl.getTableDescriptor(), HConstants.EMPTY_START_ROW,
@@ -2648,35 +2672,36 @@ public class TestHBaseFsck {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
 
-      // make sure data in regions, if in wal only there is no data loss
-      admin.flush(table);
-      HRegionInfo region1 = tbl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
-      HRegionInfo region2 = tbl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
+      try(RegionLocator rl = connection.getRegionLocator(tbl.getName())) {
+        // make sure data in regions, if in wal only there is no data loss
+        admin.flush(table);
+        HRegionInfo region1 = rl.getRegionLocation(Bytes.toBytes("A")).getRegionInfo();
+        HRegionInfo region2 = rl.getRegionLocation(Bytes.toBytes("B")).getRegionInfo();
 
-      int regionCountBeforeMerge = tbl.getRegionLocations().size();
+        int regionCountBeforeMerge = rl.getAllRegionLocations().size();
 
-      assertNotEquals(region1, region2);
+        assertNotEquals(region1, region2);
 
-      // do a region merge
-      admin.mergeRegions(region1.getEncodedNameAsBytes(),
-          region2.getEncodedNameAsBytes(), false);
+        // do a region merge
+        admin.mergeRegions(region1.getEncodedNameAsBytes(), region2.getEncodedNameAsBytes(), false);
 
-      // wait until region merged
-      long timeout = System.currentTimeMillis() + 30 * 1000;
-      while (true) {
-        if (tbl.getRegionLocations().size() < regionCountBeforeMerge) {
-          break;
-        } else if (System.currentTimeMillis() > timeout) {
-          fail("Time out waiting on region " + region1.getEncodedName()
-              + " and " + region2.getEncodedName() + " be merged");
+        // wait until region merged
+        long timeout = System.currentTimeMillis() + 30 * 1000;
+        while (true) {
+          if (rl.getAllRegionLocations().size() < regionCountBeforeMerge) {
+            break;
+          } else if (System.currentTimeMillis() > timeout) {
+            fail("Time out waiting on region " + region1.getEncodedName() + " and " + region2
+                .getEncodedName() + " be merged");
+          }
+          Thread.sleep(10);
         }
-        Thread.sleep(10);
-      }
 
-      assertEquals(ROWKEYS.length, countRows());
+        assertEquals(ROWKEYS.length, countRows());
 
-      HBaseFsck hbck = doFsck(conf, false);
-      assertNoErrors(hbck); // no errors
+        HBaseFsck hbck = doFsck(conf, false);
+        assertNoErrors(hbck); // no errors
+      }
 
     } finally {
       TEST_UTIL.getHBaseCluster().getMaster().setCatalogJanitorEnabled(true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b5b58530/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java
index 513d538..085ddf5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestProcessBasedCluster.java
@@ -58,7 +58,7 @@ public class TestProcessBasedCluster {
     cluster.startMiniDFS();
     cluster.startHBase();
     try {
-      TEST_UTIL.createRandomTable(HTestConst.DEFAULT_TABLE_STR,
+      TEST_UTIL.createRandomTable(HTestConst.DEFAULT_TABLE,
           HTestConst.DEFAULT_CF_STR_SET,
           HColumnDescriptor.DEFAULT_VERSIONS, COLS_PER_ROW, FLUSHES, NUM_REGIONS,
           ROWS_PER_FLUSH);