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:53 UTC

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

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);