You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2014/09/09 08:40:48 UTC

[5/8] HBASE-11679 Replace HTable with HTableInterface where backwards-compatible (Carter)

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
index 6997ef5..bb16add 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverBypass.java
@@ -29,17 +29,16 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-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.Result;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -93,7 +92,7 @@ public class TestRegionObserverBypass {
    */
   @Test
   public void testSimple() throws Exception {
-    HTable t = new HTable(util.getConfiguration(), tableName);
+    Table t = new HTable(util.getConfiguration(), tableName);
     Put p = new Put(row1);
     p.add(test,dummy,dummy);
     // before HBASE-4331, this would throw an exception
@@ -112,7 +111,7 @@ public class TestRegionObserverBypass {
     //previous deletes will eclipse successive puts having the same timestamp
     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
 
-    HTable t = new HTable(util.getConfiguration(), tableName);
+    Table t = new HTable(util.getConfiguration(), tableName);
     List<Put> puts = new ArrayList<Put>();
     Put p = new Put(row1);
     p.add(dummy,dummy,dummy);
@@ -197,7 +196,7 @@ public class TestRegionObserverBypass {
     EnvironmentEdgeManager.reset();
   }
 
-  private void checkRowAndDelete(HTable t, byte[] row, int count) throws IOException {
+  private void checkRowAndDelete(Table t, byte[] row, int count) throws IOException {
     Get g = new Get(row);
     Result r = t.get(g);
     assertEquals(count, r.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 61c1721..d7ec064 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -114,7 +115,7 @@ public class TestRegionObserverInterface {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRegionObserver");
     // recreate table every time in order to reset the status of the
     // coprocessor.
-    HTable 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", "hadDelete", "hadPostStartRegionOperation",
@@ -176,7 +177,7 @@ public class TestRegionObserverInterface {
   @Test
   public void testRowMutation() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testRowMutation");
-    HTable 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",
@@ -213,7 +214,7 @@ public class TestRegionObserverInterface {
   @Test
   public void testIncrementHook() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testIncrementHook");
-    HTable 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);
@@ -241,7 +242,7 @@ public class TestRegionObserverInterface {
   public void testCheckAndPutHooks() throws IOException {
     TableName tableName =
         TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndPutHooks");
-    HTable table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
     try {
       Put p = new Put(Bytes.toBytes(0));
       p.add(A, A, A);
@@ -272,7 +273,7 @@ public class TestRegionObserverInterface {
   public void testCheckAndDeleteHooks() throws IOException {
     TableName tableName =
         TableName.valueOf(TEST_TABLE.getNameAsString() + ".testCheckAndDeleteHooks");
-    HTable table = util.createTable(tableName, new byte[][] {A, B, C});
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
     try {
       Put p = new Put(Bytes.toBytes(0));
       p.add(A, A, A);
@@ -302,7 +303,7 @@ public class TestRegionObserverInterface {
   @Test
   public void testAppendHook() throws IOException {
     TableName tableName = TableName.valueOf(TEST_TABLE.getNameAsString() + ".testAppendHook");
-    HTable 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);
@@ -341,7 +342,7 @@ public class TestRegionObserverInterface {
         new Boolean[] {false, false, false, false}
     );
 
-    HTable table = new HTable(util.getConfiguration(), tableName);
+    Table table = new HTable(util.getConfiguration(), tableName);
     Put put = new Put(ROW);
     put.add(A, A, A);
     table.put(put);
@@ -391,7 +392,7 @@ public class TestRegionObserverInterface {
         new Boolean[] {false, false}
     );
 
-    HTable table = new HTable(util.getConfiguration(), tableName);
+    Table table = new HTable(util.getConfiguration(), tableName);
     Put put = new Put(ROW);
     put.add(A, A, A);
     table.put(put);
@@ -498,7 +499,7 @@ public class TestRegionObserverInterface {
     htd.addCoprocessor(EvenOnlyCompactor.class.getName());
     admin.createTable(htd);
 
-    HTable table = new HTable(util.getConfiguration(), compactTable);
+    Table table = new HTable(util.getConfiguration(), compactTable);
     for (long i=1; i<=10; i++) {
       byte[] iBytes = Bytes.toBytes(i);
       Put put = new Put(iBytes);
@@ -560,7 +561,7 @@ 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});
+    Table table = util.createTable(tableName, new byte[][] {A, B, C});
     try {
       verifyMethodResult(SimpleRegionObserver.class,
           new String[] {"hadPreBulkLoadHFile", "hadPostBulkLoadHFile"},

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 00f7c49..3365a95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Put;
 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.filter.FilterBase;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -257,7 +258,7 @@ public class TestRegionObserverScannerOpenHook {
     Admin admin = UTIL.getHBaseAdmin();
     admin.createTable(desc);
 
-    HTable table = new HTable(conf, desc.getTableName());
+    Table table = new HTable(conf, desc.getTableName());
 
     // put a row and flush it to disk
     Put put = new Put(ROW);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
index 5920bb5..f9d15c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerObserver.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -74,7 +75,7 @@ public class TestRegionServerObserver {
     // Start the cluster
     HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
     TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
-    HBaseAdmin admin = new HBaseAdmin(conf);
+    Admin admin = new HBaseAdmin(conf);
     try {
       MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
       HRegionServer regionServer = cluster.getRegionServer(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
index 085348e..76612ce 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRowProcessorEndpoint.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -107,7 +107,7 @@ public class TestRowProcessorEndpoint {
   private static volatile int expectedCounter = 0;
   private static int rowSize, row2Size;
 
-  private volatile static HTable table = null;
+  private volatile static Table table = null;
   private volatile static boolean swapped = false;
   private volatile CountDownLatch startSignal;
   private volatile CountDownLatch doneSignal;
@@ -196,7 +196,7 @@ public class TestRowProcessorEndpoint {
     }
   }
 
-  private int incrementCounter(HTable table) throws Throwable {
+  private int incrementCounter(Table table) throws Throwable {
     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
     RowProcessorEndpoint.IncrementCounterProcessor processor =
         new RowProcessorEndpoint.IncrementCounterProcessor(ROW);
@@ -259,7 +259,7 @@ public class TestRowProcessorEndpoint {
     }
   }
 
-  private void swapRows(HTable table) throws Throwable {
+  private void swapRows(Table table) throws Throwable {
     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
     RowProcessorEndpoint.RowSwapProcessor processor =
         new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
index 20135dd..c501149 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnRangeFilter.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.filter;
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -30,14 +29,12 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.*;
-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.Durability;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.After;
@@ -161,7 +158,7 @@ public class TestColumnRangeFilter {
   public void TestColumnRangeFilterClient() throws Exception {
     String family = "Family";
     String table = "TestColumnRangeFilterClient";
-    HTable ht = TEST_UTIL.createTable(Bytes.toBytes(table),
+    Table ht = TEST_UTIL.createTable(Bytes.toBytes(table),
         Bytes.toBytes(family), Integer.MAX_VALUE);
 
     List<String> rows = generateRandomWords(10, 8);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
index a35d5c5..3955d36 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
@@ -49,6 +49,7 @@ 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.ScannerCallable;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -85,7 +86,7 @@ public class TestFilterWithScanLimits {
 
       // add filter after batch defined
       scan.setFilter(filter);
-      HTable table = new HTable(conf, name);
+      Table table = new HTable(conf, name);
       ResultScanner scanner = table.getScanner(scan);
       // Expect to get following row
       // row2 => <f1:c1, 2_c1>, <f1:c2, 2_c2>,
@@ -111,7 +112,7 @@ public class TestFilterWithScanLimits {
 
   private static void prepareData() {
     try {
-      HTable table = new HTable(TestFilterWithScanLimits.conf, name);
+      Table table = new HTable(TestFilterWithScanLimits.conf, name);
       assertTrue("Fail to create the table", admin.tableExists(name));
       List<Put> puts = new ArrayList<Put>();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
index 9587aa3..46a7042 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -43,6 +42,7 @@ 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.util.Bytes;
 
 import org.junit.AfterClass;
@@ -83,7 +83,7 @@ public class TestFilterWrapper {
       FilterList filter = new FilterList(fs);
 
       scan.setFilter(filter);
-      HTable table = new HTable(conf, name);
+      Table table = new HTable(conf, name);
       ResultScanner scanner = table.getScanner(scan);
 
       // row2 (c1-c4) and row3(c1-c4) are returned
@@ -110,7 +110,7 @@ public class TestFilterWrapper {
 
   private static void prepareData() {
     try {
-      HTable table = new HTable(TestFilterWrapper.conf, name);
+      Table table = new HTable(TestFilterWrapper.conf, name);
       assertTrue("Fail to create the table", admin.tableExists(name));
       List<Put> puts = new ArrayList<Put>();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
index 1f853ac..d64a5ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowAndColumnRangeFilter.java
@@ -29,11 +29,11 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MediumTests;
 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.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.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
@@ -88,7 +88,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
   public void Test() throws Exception {
     String cf = "f";
     String table = "TestFuzzyAndColumnRangeFilterClient";
-    HTable ht = TEST_UTIL.createTable(Bytes.toBytes(table),
+    Table ht = TEST_UTIL.createTable(Bytes.toBytes(table),
             Bytes.toBytes(cf), Integer.MAX_VALUE);
 
     // 10 byte row key - (2 bytes 4 bytes 4 bytes)
@@ -128,7 +128,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
     runTest(ht, 1, 8);
   }
 
-  private void runTest(HTable hTable, int cqStart, int expectedSize) throws IOException {
+  private void runTest(Table hTable, int cqStart, int expectedSize) throws IOException {
     // [0, 2, ?, ?, ?, ?, 0, 0, 0, 1]
     byte[] fuzzyKey = new byte[10];
     ByteBuffer buf = ByteBuffer.wrap(fuzzyKey);
@@ -150,7 +150,7 @@ public class TestFuzzyRowAndColumnRangeFilter {
     runScanner(hTable, expectedSize, columnRangeFilter, fuzzyRowFilter);
   }
 
-  private void runScanner(HTable hTable, int expectedSize, Filter... filters) throws IOException {
+  private void runScanner(Table hTable, int expectedSize, Filter... filters) throws IOException {
     String cf = "f";
     Scan scan = new Scan();
     scan.addFamily(cf.getBytes());

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
index 49e3465..f83fbf3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
-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.HLogUtil;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -259,7 +259,7 @@ public class TestBlockReorder {
     // We use the regionserver file system & conf as we expect it to have the hook.
     conf = targetRs.getConfiguration();
     HFileSystem rfs = (HFileSystem) targetRs.getFileSystem();
-    HTable h = htu.createTable("table".getBytes(), sb);
+    Table h = htu.createTable("table".getBytes(), sb);
 
     // Now, we have 4 datanodes and a replication count of 3. So we don't know if the datanode
     // with the same node will be used. We can't really stop an existing datanode, this would

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
index 2b1f6f6..b7bb02c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestChangingEncoding.java
@@ -42,6 +42,7 @@ 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.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
@@ -136,7 +137,7 @@ public class TestChangingEncoding {
   static void writeTestDataBatch(Configuration conf, String tableName,
       int batchId) throws Exception {
     LOG.debug("Writing test data batch " + batchId);
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     table.setAutoFlushTo(false);
     for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
       Put put = new Put(getRowKey(batchId, i));
@@ -154,7 +155,7 @@ public class TestChangingEncoding {
   static void verifyTestDataBatch(Configuration conf, String tableName,
       int batchId) throws Exception {
     LOG.debug("Verifying test data batch " + batchId);
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     for (int i = 0; i < NUM_ROWS_PER_BATCH; ++i) {
       Get get = new Get(getRowKey(batchId, i));
       Result result = table.get(get);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java
index d19437e..710aed9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTree.java
@@ -34,6 +34,7 @@ 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.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -79,7 +80,7 @@ public class TestPrefixTree {
   @Test
   public void testHBASE11728() throws Exception {
     TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    HTable table = null;
+    Table table = null;
     try {
       Admin hBaseAdmin = TEST_UTIL.getHBaseAdmin();
       HColumnDescriptor colDesc = new HColumnDescriptor(fam);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
index 88ffd9f..1cdde7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableInputFormat.java
@@ -35,11 +35,11 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.*;
-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.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -88,8 +88,8 @@ public class TestTableInputFormat {
    * @return
    * @throws IOException
    */
-  public static HTable createTable(byte[] tableName) throws IOException {
-    HTable table = UTIL.createTable(tableName, FAMILY);
+  public static Table createTable(byte[] tableName) throws IOException {
+    Table table = UTIL.createTable(tableName, FAMILY);
     Put p = new Put("aaa".getBytes());
     p.add(FAMILY, null, "value aaa".getBytes());
     table.put(p);
@@ -124,7 +124,7 @@ public class TestTableInputFormat {
    * @param table
    * @throws IOException
    */
-  static void runTestMapred(HTable table) throws IOException {
+  static void runTestMapred(Table table) throws IOException {
     org.apache.hadoop.hbase.mapred.TableRecordReader trr = 
         new org.apache.hadoop.hbase.mapred.TableRecordReader();
     trr.setStartRow("aaa".getBytes());
@@ -157,7 +157,7 @@ public class TestTableInputFormat {
    * @throws IOException
    * @throws InterruptedException
    */
-  static void runTestMapreduce(HTable table) throws IOException,
+  static void runTestMapreduce(Table table) throws IOException,
       InterruptedException {
     org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl trr = 
         new org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl();
@@ -194,7 +194,7 @@ public class TestTableInputFormat {
    * 
    * @throws IOException
    */
-  static HTable createIOEScannerTable(byte[] name, final int failCnt)
+  static Table createIOEScannerTable(byte[] name, final int failCnt)
       throws IOException {
     // build up a mock scanner stuff to fail the first time
     Answer<ResultScanner> a = new Answer<ResultScanner>() {
@@ -218,7 +218,7 @@ public class TestTableInputFormat {
       }
     };
 
-    HTable htable = spy(createTable(name));
+    Table htable = spy(createTable(name));
     doAnswer(a).when(htable).getScanner((Scan) anyObject());
     return htable;
   }
@@ -229,7 +229,7 @@ public class TestTableInputFormat {
    * 
    * @throws IOException
    */
-  static HTable createDNRIOEScannerTable(byte[] name, final int failCnt)
+  static Table createDNRIOEScannerTable(byte[] name, final int failCnt)
       throws IOException {
     // build up a mock scanner stuff to fail the first time
     Answer<ResultScanner> a = new Answer<ResultScanner>() {
@@ -256,7 +256,7 @@ public class TestTableInputFormat {
       }
     };
 
-    HTable htable = spy(createTable(name));
+    Table htable = spy(createTable(name));
     doAnswer(a).when(htable).getScanner((Scan) anyObject());
     return htable;
   }
@@ -268,7 +268,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReader() throws IOException {
-    HTable table = createTable("table1".getBytes());
+    Table table = createTable("table1".getBytes());
     runTestMapred(table);
   }
 
@@ -279,7 +279,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReaderScannerFail() throws IOException {
-    HTable htable = createIOEScannerTable("table2".getBytes(), 1);
+    Table htable = createIOEScannerTable("table2".getBytes(), 1);
     runTestMapred(htable);
   }
 
@@ -290,7 +290,7 @@ public class TestTableInputFormat {
    */
   @Test(expected = IOException.class)
   public void testTableRecordReaderScannerFailTwice() throws IOException {
-    HTable htable = createIOEScannerTable("table3".getBytes(), 2);
+    Table htable = createIOEScannerTable("table3".getBytes(), 2);
     runTestMapred(htable);
   }
 
@@ -302,7 +302,7 @@ public class TestTableInputFormat {
    */
   @Test
   public void testTableRecordReaderScannerTimeout() throws IOException {
-    HTable htable = createDNRIOEScannerTable("table4".getBytes(), 1);
+    Table htable = createDNRIOEScannerTable("table4".getBytes(), 1);
     runTestMapred(htable);
   }
 
@@ -314,7 +314,7 @@ public class TestTableInputFormat {
    */
   @Test(expected = org.apache.hadoop.hbase.DoNotRetryIOException.class)
   public void testTableRecordReaderScannerTimeoutTwice() throws IOException {
-    HTable htable = createDNRIOEScannerTable("table5".getBytes(), 2);
+    Table htable = createDNRIOEScannerTable("table5".getBytes(), 2);
     runTestMapred(htable);
   }
 
@@ -327,7 +327,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderMapreduce() throws IOException,
       InterruptedException {
-    HTable table = createTable("table1-mr".getBytes());
+    Table table = createTable("table1-mr".getBytes());
     runTestMapreduce(table);
   }
 
@@ -340,7 +340,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderScannerFailMapreduce() throws IOException,
       InterruptedException {
-    HTable htable = createIOEScannerTable("table2-mr".getBytes(), 1);
+    Table htable = createIOEScannerTable("table2-mr".getBytes(), 1);
     runTestMapreduce(htable);
   }
 
@@ -353,7 +353,7 @@ public class TestTableInputFormat {
   @Test(expected = IOException.class)
   public void testTableRecordReaderScannerFailMapreduceTwice() throws IOException,
       InterruptedException {
-    HTable htable = createIOEScannerTable("table3-mr".getBytes(), 2);
+    Table htable = createIOEScannerTable("table3-mr".getBytes(), 2);
     runTestMapreduce(htable);
   }
 
@@ -367,7 +367,7 @@ public class TestTableInputFormat {
   @Test
   public void testTableRecordReaderScannerTimeoutMapreduce()
       throws IOException, InterruptedException {
-    HTable htable = createDNRIOEScannerTable("table4-mr".getBytes(), 1);
+    Table htable = createDNRIOEScannerTable("table4-mr".getBytes(), 1);
     runTestMapreduce(htable);
   }
 
@@ -381,7 +381,7 @@ public class TestTableInputFormat {
   @Test(expected = org.apache.hadoop.hbase.DoNotRetryIOException.class)
   public void testTableRecordReaderScannerTimeoutMapreduceTwice()
       throws IOException, InterruptedException {
-    HTable htable = createDNRIOEScannerTable("table5-mr".getBytes(), 2);
+    Table htable = createDNRIOEScannerTable("table5-mr".getBytes(), 2);
     runTestMapreduce(htable);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
index 99fc48d..e527f97 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapred/TestTableMapReduceUtil.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LargeTests;
-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.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.JobClient;
@@ -61,7 +61,7 @@ public class TestTableMapReduceUtil {
   private static final Log LOG = LogFactory
       .getLog(TestTableMapReduceUtil.class);
 
-  private static HTable presidentsTable;
+  private static Table presidentsTable;
   private static final String TABLE_NAME = "People";
 
   private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info");
@@ -104,13 +104,13 @@ public class TestTableMapReduceUtil {
     LOG.info("before done");
   }
 
-  public static HTable createAndFillTable(byte[] tableName) throws IOException {
-    HTable table = UTIL.createTable(tableName, COLUMN_FAMILY);
+  public static Table createAndFillTable(byte[] tableName) throws IOException {
+    Table table = UTIL.createTable(tableName, COLUMN_FAMILY);
     createPutCommand(table);
     return table;
   }
 
-  private static void createPutCommand(HTable table) throws IOException {
+  private static void createPutCommand(Table table) throws IOException {
     for (String president : presidentsRowKeys) {
       if (presidentNames.hasNext()) {
         Put p = new Put(Bytes.toBytes(president));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 1ccc57d..72f48b1 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,8 +25,8 @@ import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LargeTests;
-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.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
 import org.apache.hadoop.mapreduce.Job;
@@ -80,7 +80,7 @@ public class TestCellCounter {
   public void testCellCounter() throws Exception {
     String sourceTable = "sourceTable";
     byte[][] families = { FAMILY_A, FAMILY_B };
-    HTable t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t = UTIL.createTable(Bytes.toBytes(sourceTable), families);
     try{
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER, now, Bytes.toBytes("Data11"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 6163bb9..8d171a6 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
@@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 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.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
 import org.apache.hadoop.mapreduce.Job;
@@ -84,8 +84,8 @@ public class TestCopyTable {
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] COLUMN1 = Bytes.toBytes("c1");
 
-    HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
-    HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
+    Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
+    Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
 
     // put rows into the first table
     for (int i = 0; i < 10; i++) {
@@ -125,8 +125,8 @@ public class TestCopyTable {
     final byte[] ROW1 = Bytes.toBytes("row1");
     final byte[] ROW2 = Bytes.toBytes("row2");
 
-    HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
-    HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
+    Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
+    Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
 
     // put rows into the first table
     Put p = new Put(ROW0);
@@ -176,8 +176,8 @@ public class TestCopyTable {
 
     byte[][] families = { FAMILY_A, FAMILY_B };
 
-    HTable t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families);
-    HTable t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families);
+    Table t = TEST_UTIL.createTable(Bytes.toBytes(sourceTable), families);
+    Table t2 = TEST_UTIL.createTable(Bytes.toBytes(targetTable), families);
     Put p = new Put(ROW1);
     p.add(FAMILY_A, QUALIFIER,  Bytes.toBytes("Data11"));
     p.add(FAMILY_B, QUALIFIER,  Bytes.toBytes("Data12"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 a46660e..c257cf3 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
@@ -58,9 +58,11 @@ import org.apache.hadoop.hbase.TableName;
 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;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -476,7 +478,7 @@ public class TestHFileOutputFormat  {
   }
 
   /**
-   * Test for {@link HFileOutputFormat#configureCompression(HTable,
+   * Test for {@link HFileOutputFormat#configureCompression(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat#createFamilyCompressionMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -490,7 +492,7 @@ public class TestHFileOutputFormat  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, Compression.Algorithm> familyToCompression =
           getMockColumnFamiliesForCompression(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForCompression(table, familyToCompression);
       HFileOutputFormat.configureCompression(table, conf);
 
@@ -508,7 +510,7 @@ public class TestHFileOutputFormat  {
     }
   }
 
-  private void setupMockColumnFamiliesForCompression(HTable table,
+  private void setupMockColumnFamiliesForCompression(Table table,
       Map<String, Compression.Algorithm> familyToCompression) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
@@ -546,7 +548,7 @@ public class TestHFileOutputFormat  {
 
 
   /**
-   * Test for {@link HFileOutputFormat#configureBloomType(HTable,
+   * Test for {@link HFileOutputFormat#configureBloomType(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat#createFamilyBloomTypeMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -560,7 +562,7 @@ public class TestHFileOutputFormat  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, BloomType> familyToBloomType =
           getMockColumnFamiliesForBloomType(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForBloomType(table,
           familyToBloomType);
       HFileOutputFormat.configureBloomType(table, conf);
@@ -581,7 +583,7 @@ public class TestHFileOutputFormat  {
     }
   }
 
-  private void setupMockColumnFamiliesForBloomType(HTable table,
+  private void setupMockColumnFamiliesForBloomType(Table table,
       Map<String, BloomType> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, BloomType> entry : familyToDataBlockEncoding.entrySet()) {
@@ -617,7 +619,7 @@ public class TestHFileOutputFormat  {
   }
 
   /**
-   * Test for {@link HFileOutputFormat#configureBlockSize(HTable,
+   * Test for {@link HFileOutputFormat#configureBlockSize(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat#createFamilyBlockSizeMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -631,7 +633,7 @@ public class TestHFileOutputFormat  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, Integer> familyToBlockSize =
           getMockColumnFamiliesForBlockSize(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForBlockSize(table,
           familyToBlockSize);
       HFileOutputFormat.configureBlockSize(table, conf);
@@ -653,7 +655,7 @@ public class TestHFileOutputFormat  {
     }
   }
 
-  private void setupMockColumnFamiliesForBlockSize(HTable table,
+  private void setupMockColumnFamiliesForBlockSize(Table table,
       Map<String, Integer> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, Integer> entry : familyToDataBlockEncoding.entrySet()) {
@@ -693,7 +695,7 @@ public class TestHFileOutputFormat  {
   }
 
     /**
-   * Test for {@link HFileOutputFormat#configureDataBlockEncoding(HTable,
+   * Test for {@link HFileOutputFormat#configureDataBlockEncoding(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat#createFamilyDataBlockEncodingMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -707,7 +709,7 @@ public class TestHFileOutputFormat  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, DataBlockEncoding> familyToDataBlockEncoding =
           getMockColumnFamiliesForDataBlockEncoding(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForDataBlockEncoding(table,
           familyToDataBlockEncoding);
       HFileOutputFormat.configureDataBlockEncoding(table, conf);
@@ -728,7 +730,7 @@ public class TestHFileOutputFormat  {
     }
   }
 
-  private void setupMockColumnFamiliesForDataBlockEncoding(HTable table,
+  private void setupMockColumnFamiliesForDataBlockEncoding(Table table,
       Map<String, DataBlockEncoding> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {
@@ -767,7 +769,7 @@ public class TestHFileOutputFormat  {
     return familyToDataBlockEncoding;
   }
 
-  private void setupMockStartKeys(HTable table) throws IOException {
+  private void setupMockStartKeys(RegionLocator table) throws IOException {
     byte[][] mockKeys = new byte[][] {
         HConstants.EMPTY_BYTE_ARRAY,
         Bytes.toBytes("aaa"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 76db299..edfd339 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,9 +56,11 @@ import org.apache.hadoop.hbase.TableName;
 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;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -475,7 +477,7 @@ public class TestHFileOutputFormat2  {
   }
 
   /**
-   * Test for {@link HFileOutputFormat2#configureCompression(HTable,
+   * Test for {@link HFileOutputFormat2#configureCompression(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat2#createFamilyCompressionMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -489,7 +491,7 @@ public class TestHFileOutputFormat2  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, Compression.Algorithm> familyToCompression =
           getMockColumnFamiliesForCompression(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForCompression(table, familyToCompression);
       HFileOutputFormat2.configureCompression(table, conf);
 
@@ -507,7 +509,7 @@ public class TestHFileOutputFormat2  {
     }
   }
 
-  private void setupMockColumnFamiliesForCompression(HTable table,
+  private void setupMockColumnFamiliesForCompression(Table table,
       Map<String, Compression.Algorithm> familyToCompression) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
@@ -546,7 +548,7 @@ public class TestHFileOutputFormat2  {
 
 
   /**
-   * Test for {@link HFileOutputFormat2#configureBloomType(HTable,
+   * Test for {@link HFileOutputFormat2#configureBloomType(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat2#createFamilyBloomTypeMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -560,7 +562,7 @@ public class TestHFileOutputFormat2  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, BloomType> familyToBloomType =
           getMockColumnFamiliesForBloomType(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForBloomType(table,
           familyToBloomType);
       HFileOutputFormat2.configureBloomType(table, conf);
@@ -581,7 +583,7 @@ public class TestHFileOutputFormat2  {
     }
   }
 
-  private void setupMockColumnFamiliesForBloomType(HTable table,
+  private void setupMockColumnFamiliesForBloomType(Table table,
       Map<String, BloomType> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, BloomType> entry : familyToDataBlockEncoding.entrySet()) {
@@ -617,7 +619,7 @@ public class TestHFileOutputFormat2  {
   }
 
   /**
-   * Test for {@link HFileOutputFormat2#configureBlockSize(HTable,
+   * Test for {@link HFileOutputFormat2#configureBlockSize(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat2#createFamilyBlockSizeMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -631,7 +633,7 @@ public class TestHFileOutputFormat2  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, Integer> familyToBlockSize =
           getMockColumnFamiliesForBlockSize(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForBlockSize(table,
           familyToBlockSize);
       HFileOutputFormat2.configureBlockSize(table, conf);
@@ -653,7 +655,7 @@ public class TestHFileOutputFormat2  {
     }
   }
 
-  private void setupMockColumnFamiliesForBlockSize(HTable table,
+  private void setupMockColumnFamiliesForBlockSize(Table table,
       Map<String, Integer> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, Integer> entry : familyToDataBlockEncoding.entrySet()) {
@@ -693,7 +695,7 @@ public class TestHFileOutputFormat2  {
   }
 
     /**
-   * Test for {@link HFileOutputFormat2#configureDataBlockEncoding(HTable,
+   * Test for {@link HFileOutputFormat2#configureDataBlockEncoding(org.apache.hadoop.hbase.client.Table,
    * Configuration)} and {@link HFileOutputFormat2#createFamilyDataBlockEncodingMap
    * (Configuration)}.
    * Tests that the compression map is correctly serialized into
@@ -707,7 +709,7 @@ public class TestHFileOutputFormat2  {
       Configuration conf = new Configuration(this.util.getConfiguration());
       Map<String, DataBlockEncoding> familyToDataBlockEncoding =
           getMockColumnFamiliesForDataBlockEncoding(numCfs);
-      HTable table = Mockito.mock(HTable.class);
+      Table table = Mockito.mock(HTable.class);
       setupMockColumnFamiliesForDataBlockEncoding(table,
           familyToDataBlockEncoding);
       HFileOutputFormat2.configureDataBlockEncoding(table, conf);
@@ -728,7 +730,7 @@ public class TestHFileOutputFormat2  {
     }
   }
 
-  private void setupMockColumnFamiliesForDataBlockEncoding(HTable table,
+  private void setupMockColumnFamiliesForDataBlockEncoding(Table table,
       Map<String, DataBlockEncoding> familyToDataBlockEncoding) throws IOException {
     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
     for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {
@@ -767,7 +769,7 @@ public class TestHFileOutputFormat2  {
     return familyToDataBlockEncoding;
   }
 
-  private void setupMockStartKeys(HTable table) throws IOException {
+  private void setupMockStartKeys(RegionLocator table) throws IOException {
     byte[][] mockKeys = new byte[][] {
         HConstants.EMPTY_BYTE_ARRAY,
         Bytes.toBytes("aaa"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index 7852fc5..f89808f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -53,6 +53,7 @@ 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.Filter;
 import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
@@ -159,7 +160,7 @@ public class TestImportExport {
   @Test
   public void testSimpleCase() throws Exception {
     String EXPORT_TABLE = "exportSimpleCase";
-    HTable t = UTIL.createTable(Bytes.toBytes(EXPORT_TABLE), FAMILYA, 3);
+    Table t = UTIL.createTable(Bytes.toBytes(EXPORT_TABLE), FAMILYA, 3);
     Put p = new Put(ROW1);
     p.add(FAMILYA, QUAL, now, QUAL);
     p.add(FAMILYA, QUAL, now+1, QUAL);
@@ -222,7 +223,7 @@ public class TestImportExport {
     fs.copyFromLocalFile(importPath, new Path(FQ_OUTPUT_DIR + Path.SEPARATOR
         + "exportedTableIn94Format"));
     String IMPORT_TABLE = "importTableExportedFrom94";
-    HTable t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), Bytes.toBytes("f1"), 3);
+    Table t = UTIL.createTable(Bytes.toBytes(IMPORT_TABLE), Bytes.toBytes("f1"), 3);
     String[] args = new String[] {
         "-Dhbase.import.version=0.94" ,
         IMPORT_TABLE, FQ_OUTPUT_DIR
@@ -252,7 +253,7 @@ public class TestImportExport {
         .setMaxVersions(1)
     );
     UTIL.getHBaseAdmin().createTable(desc);
-    HTable t = new HTable(UTIL.getConfiguration(), BATCH_TABLE);
+    Table t = new HTable(UTIL.getConfiguration(), BATCH_TABLE);
 
     Put p = new Put(ROW1);
     p.add(FAMILYA, QUAL, now, QUAL);
@@ -283,7 +284,7 @@ public class TestImportExport {
         .setKeepDeletedCells(true)
     );
     UTIL.getHBaseAdmin().createTable(desc);
-    HTable t = new HTable(UTIL.getConfiguration(), EXPORT_TABLE);
+    Table t = new HTable(UTIL.getConfiguration(), EXPORT_TABLE);
 
     Put p = new Put(ROW1);
     p.add(FAMILYA, QUAL, now, QUAL);
@@ -349,7 +350,7 @@ public class TestImportExport {
     HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(EXPORT_TABLE));
     desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
     UTIL.getHBaseAdmin().createTable(desc);
-    HTable exportTable = new HTable(UTIL.getConfiguration(), EXPORT_TABLE);
+    Table exportTable = new HTable(UTIL.getConfiguration(), EXPORT_TABLE);
 
     Put p = new Put(ROW1);
     p.add(FAMILYA, QUAL, now, QUAL);
@@ -376,7 +377,7 @@ public class TestImportExport {
     desc.addFamily(new HColumnDescriptor(FAMILYA).setMaxVersions(5));
     UTIL.getHBaseAdmin().createTable(desc);
 
-    HTable importTable = new HTable(UTIL.getConfiguration(), IMPORT_TABLE);
+    Table importTable = new HTable(UTIL.getConfiguration(), IMPORT_TABLE);
     args = new String[] { "-D" + Import.FILTER_CLASS_CONF_KEY + "=" + PrefixFilter.class.getName(),
         "-D" + Import.FILTER_ARGS_CONF_KEY + "=" + Bytes.toString(ROW1), IMPORT_TABLE, FQ_OUTPUT_DIR,
         "1000" };
@@ -410,7 +411,7 @@ public class TestImportExport {
    * @return
    * @throws IOException
    */
-  private int getCount(HTable table, Filter filter) throws IOException {
+  private int getCount(Table table, Filter filter) throws IOException {
     Scan scan = new Scan();
     scan.setFilter(filter);
     ResultScanner results = table.getScanner(scan);
@@ -542,7 +543,7 @@ public class TestImportExport {
   public void testDurability() throws IOException, InterruptedException, ClassNotFoundException {
     // Create an export table.
     String exportTableName = "exporttestDurability";
-    HTable exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3);
+    Table exportTable = UTIL.createTable(Bytes.toBytes(exportTableName), FAMILYA, 3);
 
     // Insert some data
     Put put = new Put(ROW1);
@@ -563,7 +564,7 @@ public class TestImportExport {
 
     // Create the table for import
     String importTableName = "importTestDurability1";
-    HTable importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3);
+    Table importTable = UTIL.createTable(Bytes.toBytes(importTableName), FAMILYA, 3);
 
     // Register the hlog listener for the import table
     TableWALActionListener walListener = new TableWALActionListener(importTableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 27f4acb..1c13468 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
@@ -45,6 +46,7 @@ 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.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -96,7 +98,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
     conf.set("hbase.coprocessor.master.classes", OperationAttributesTestController.class.getName());
     conf.set("hbase.coprocessor.region.classes", OperationAttributesTestController.class.getName());
     util.startMiniCluster();
-    HBaseAdmin admin = new HBaseAdmin(util.getConfiguration());
+    Admin admin = new HBaseAdmin(util.getConfiguration());
     util.startMiniMapReduceCluster();
   }
 
@@ -192,7 +194,7 @@ public class TestImportTSVWithOperationAttributes implements Configurable {
       int valueMultiplier, boolean dataAvailable) throws IOException {
 
     LOG.debug("Validating table.");
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     boolean verified = false;
     long pause = conf.getLong("hbase.client.pause", 5 * 1000);
     int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 0c483aa..9494fa4 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
@@ -42,12 +42,14 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.client.Admin;
 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.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.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
@@ -118,7 +120,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
     // Wait for the labels table to become available
     util.waitTableEnabled(VisibilityConstants.LABELS_TABLE_NAME.getName(), 50000);
     createLabels();
-    HBaseAdmin admin = new HBaseAdmin(util.getConfiguration());
+    Admin admin = new HBaseAdmin(util.getConfiguration());
     util.startMiniMapReduceCluster();
   }
 
@@ -181,7 +183,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
 
   private void issueDeleteAndVerifyData(String tableName) throws IOException {
     LOG.debug("Validating table after delete.");
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     boolean verified = false;
     long pause = conf.getLong("hbase.client.pause", 5 * 1000);
     int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@@ -365,7 +367,7 @@ public class TestImportTSVWithVisibilityLabels implements Configurable {
       int valueMultiplier) throws IOException {
 
     LOG.debug("Validating table.");
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     boolean verified = false;
     long pause = conf.getLong("hbase.client.pause", 5 * 1000);
     int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 075c0ab..8e64f77 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.HTable;
 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.util.Bytes;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Utils.OutputFileUtils.OutputFilesFilter;
@@ -302,7 +303,7 @@ public class TestImportTsv implements Configurable {
       String family, int valueMultiplier) throws IOException {
 
     LOG.debug("Validating table.");
-    HTable table = new HTable(conf, tableName);
+    Table table = new HTable(conf, tableName);
     boolean verified = false;
     long pause = conf.getLong("hbase.client.pause", 5 * 1000);
     int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
index 27c809a..2695706 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
@@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -249,7 +249,7 @@ public class TestLoadIncrementalHFiles {
     String [] args= {dir.toString(), tableName.toString()};
     loader.run(args);
 
-    HTable table = new HTable(util.getConfiguration(), tableName);
+    Table table = new HTable(util.getConfiguration(), tableName);
     try {
       assertEquals(expectedRows, util.countRows(table));
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 491c2c6..c89bd51 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.client.HTable;
 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.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
@@ -229,7 +230,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
    * @throws IOException 
    */
   void assertExpectedTable(String table, int count, int value) throws IOException {
-    HTable t = null;
+    Table t = null;
     try {
       assertEquals(util.getHBaseAdmin().listTables(table).length, 1);
       t = new HTable(util.getConfiguration(), table);
@@ -341,7 +342,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
     LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(
         util.getConfiguration()) {
 
-      protected void bulkLoadPhase(final HTable htable, final HConnection conn,
+      protected void bulkLoadPhase(final Table htable, final HConnection conn,
           ExecutorService pool, Deque<LoadQueueItem> queue,
           final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
         int i = attemptedCalls.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 34bca43..93dbbeb 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
@@ -34,6 +34,7 @@ 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.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.Job;
@@ -160,7 +161,7 @@ public class TestMultithreadedTableMapper {
   }
 
   private void verify(String tableName) throws IOException {
-    HTable table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
+    Table table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
     boolean verified = false;
     long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
     int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@@ -193,7 +194,7 @@ public class TestMultithreadedTableMapper {
    * @throws IOException
    * @throws NullPointerException if we failed to find a cell value
    */
-  private void verifyAttempt(final HTable table)
+  private void verifyAttempt(final Table table)
       throws IOException, NullPointerException {
     Scan scan = new Scan();
     scan.addFamily(INPUT_FAMILY);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 f5bbddc..09aa652 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
@@ -32,8 +32,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.MediumTests;
-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.mapreduce.RowCounter.RowCounterMapper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
@@ -67,7 +67,7 @@ public class TestRowCounter {
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniCluster();
     TEST_UTIL.startMiniMapReduceCluster();
-    HTable table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME),
+    Table table = TEST_UTIL.createTable(Bytes.toBytes(TABLE_NAME),
         Bytes.toBytes(COL_FAM));
     writeRows(table);
     table.close();
@@ -164,7 +164,7 @@ public class TestRowCounter {
    * @param table
    * @throws IOException
    */
-  private static void writeRows(HTable table) throws IOException {
+  private static void writeRows(Table table) throws IOException {
     final byte[] family = Bytes.toBytes(COL_FAM);
     final byte[] value = Bytes.toBytes("abcd");
     final byte[] col1 = Bytes.toBytes(COL1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 8a45f14..789c874 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
@@ -37,6 +37,7 @@ 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.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
@@ -132,7 +133,7 @@ public abstract class TestTableMapReduceBase {
   }
 
   protected void verify(String tableName) throws IOException {
-    HTable table = new HTable(UTIL.getConfiguration(), tableName);
+    Table table = new HTable(UTIL.getConfiguration(), tableName);
     boolean verified = false;
     long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
     int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
@@ -163,7 +164,7 @@ public abstract class TestTableMapReduceBase {
    * @throws IOException
    * @throws NullPointerException if we failed to find a cell value
    */
-  private void verifyAttempt(final HTable table) throws IOException, NullPointerException {
+  private void verifyAttempt(final Table table) throws IOException, NullPointerException {
     Scan scan = new Scan();
     TableInputFormat.addColumns(scan, columns);
     ResultScanner scanner = table.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
index a5c0b92..e6520df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -39,6 +40,7 @@ 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.Durability;
+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.MapWritable;
@@ -57,7 +59,7 @@ public class TestTimeRangeMapRed {
   private final static Log log = LogFactory.getLog(TestTimeRangeMapRed.class);
   private static final HBaseTestingUtility UTIL =
     new HBaseTestingUtility();
-  private HBaseAdmin admin;
+  private Admin admin;
 
   private static final byte [] KEY = Bytes.toBytes("row1");
   private static final NavigableMap<Long, Boolean> TIMESTAMP =
@@ -103,7 +105,7 @@ public class TestTimeRangeMapRed {
   implements Configurable {
 
     private Configuration conf = null;
-    private HTable table = null;
+    private Table table = null;
 
     @Override
     public void map(ImmutableBytesWritable key, Result result,
@@ -147,13 +149,13 @@ public class TestTimeRangeMapRed {
     col.setMaxVersions(Integer.MAX_VALUE);
     desc.addFamily(col);
     admin.createTable(desc);
-    HTable table = new HTable(UTIL.getConfiguration(), desc.getTableName());
+    Table table = new HTable(UTIL.getConfiguration(), desc.getTableName());
     prepareTest(table);
     runTestOnTable();
     verify(table);
   }
 
-  private void prepareTest(final HTable table) throws IOException {
+  private void prepareTest(final Table table) throws IOException {
     for (Map.Entry<Long, Boolean> entry : TIMESTAMP.entrySet()) {
       Put put = new Put(KEY);
       put.setDurability(Durability.SKIP_WAL);
@@ -190,7 +192,7 @@ public class TestTimeRangeMapRed {
     }
   }
 
-  private void verify(final HTable table) throws IOException {
+  private void verify(final Table table) throws IOException {
     Scan scan = new Scan();
     scan.addColumn(FAMILY_NAME, COLUMN_NAME);
     scan.setMaxVersions(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
index 89a46d8..a586af5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
@@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 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.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.WALPlayer.HLogKeyValueMapper;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -92,8 +92,8 @@ public class TestWALPlayer {
     final byte[] COLUMN1 = Bytes.toBytes("c1");
     final byte[] COLUMN2 = Bytes.toBytes("c2");
     final byte[] ROW = Bytes.toBytes("row");
-    HTable t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
-    HTable t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
+    Table t1 = TEST_UTIL.createTable(TABLENAME1, FAMILY);
+    Table t2 = TEST_UTIL.createTable(TABLENAME2, FAMILY);
 
     // put a row into the first table
     Put p = new Put(ROW);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index 5fb67e5..383baf8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName;
 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.Table;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
@@ -203,7 +204,7 @@ public class TestAssignmentListener {
       assertEquals(0, listener.getCloseCount());
 
       // Add some data
-      HTable table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME);
+      Table table = new HTable(TEST_UTIL.getConfiguration(), TABLE_NAME);
       try {
         for (int i = 0; i < 10; ++i) {
           byte[] key = Bytes.toBytes("row-" + i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/4995ed8a/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 a5a707c..4b5556e 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -174,7 +175,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -218,7 +219,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       final HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -427,7 +428,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -476,7 +477,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -522,7 +523,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -564,7 +565,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -649,7 +650,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -702,7 +703,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -775,7 +776,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -850,7 +851,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -891,7 +892,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -970,7 +971,7 @@ public class TestAssignmentManagerOnCluster {
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
 
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri = new HRegionInfo(
         desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);
@@ -1045,7 +1046,7 @@ public class TestAssignmentManagerOnCluster {
       HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(table));
       desc.addFamily(new HColumnDescriptor(FAMILY));
       admin.createTable(desc);
-      HTable meta = new HTable(conf, TableName.META_TABLE_NAME);
+      Table meta = new HTable(conf, TableName.META_TABLE_NAME);
       HRegionInfo hri =
           new HRegionInfo(desc.getTableName(), Bytes.toBytes("A"), Bytes.toBytes("Z"));
       MetaTableAccessor.addRegionToMeta(meta, hri);