You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2017/11/08 10:37:33 UTC

[3/3] phoenix git commit: PHOENIX-4303 Replace HTableInterface, HConnection with Table, Connection interfaces respectively(Rajeshbabu)

PHOENIX-4303 Replace HTableInterface,HConnection with Table,Connection interfaces respectively(Rajeshbabu)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 113904275d0689755aea100aaeb43aed9bd9cc9d
Parents: 136c7a6
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed Nov 8 16:07:14 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Wed Nov 8 16:07:14 2017 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   4 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   4 +-
 .../end2end/ColumnProjectionOptimizationIT.java |   4 +-
 .../phoenix/end2end/DisableLocalIndexIT.java    |   4 +-
 .../apache/phoenix/end2end/DynamicColumnIT.java |   6 +-
 .../apache/phoenix/end2end/DynamicFamilyIT.java |   6 +-
 .../phoenix/end2end/MappingTableDataTypeIT.java |   9 +-
 .../phoenix/end2end/MetaDataEndPointIT.java     |   4 +-
 .../phoenix/end2end/NativeHBaseTypesIT.java     |  10 +-
 .../phoenix/end2end/PhoenixRuntimeIT.java       |  16 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |   4 +-
 .../end2end/RebuildIndexConnectionPropsIT.java  |  10 +-
 .../phoenix/end2end/StatsCollectorIT.java       |   8 +-
 .../UpdateCacheAcrossDifferentClientsIT.java    |   4 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |   7 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   4 +-
 .../phoenix/end2end/index/BaseIndexIT.java      |   4 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |   4 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |   4 +-
 .../end2end/index/PartialIndexRebuilderIT.java  |  36 ++--
 .../phoenix/tx/FlappingTransactionIT.java       |   4 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |   8 +-
 .../apache/phoenix/cache/ServerCacheClient.java |   9 +-
 .../apache/phoenix/compile/FromCompiler.java    |   4 +-
 .../DelegateRegionCoprocessorEnvironment.java   |  15 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  14 +-
 .../coprocessor/MetaDataRegionObserver.java     |   6 +-
 .../apache/phoenix/execute/DelegateHTable.java  | 185 ++++++++-----------
 .../apache/phoenix/execute/MutationState.java   |  18 +-
 .../index/table/CoprocessorHTableFactory.java   |   6 +-
 .../hbase/index/table/HTableFactory.java        |   6 +-
 .../hbase/index/write/IndexWriterUtils.java     |  19 +-
 .../write/ParallelWriterIndexCommitter.java     |   4 +-
 .../TrackingParallelWriterIndexCommitter.java   |   4 +-
 .../index/PhoenixIndexFailurePolicy.java        |   6 +-
 .../index/PhoenixTransactionalIndexer.java      |   2 +-
 .../phoenix/iterate/BaseResultIterators.java    |   2 +-
 .../apache/phoenix/iterate/SnapshotScanner.java |   4 +-
 .../phoenix/iterate/TableResultIterator.java    |   6 +-
 .../phoenix/mapreduce/PhoenixRecordReader.java  |   3 +-
 .../mapreduce/index/DirectHTableWriter.java     |  13 +-
 .../phoenix/mapreduce/index/IndexTool.java      |   8 +-
 .../phoenix/query/ConnectionQueryServices.java  |   7 +-
 .../query/ConnectionQueryServicesImpl.java      |  65 +++----
 .../query/ConnectionlessQueryServicesImpl.java  |   6 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/GuidePostsCache.java   |   4 +-
 .../phoenix/query/HConnectionFactory.java       |  10 +-
 .../org/apache/phoenix/query/HTableFactory.java |  11 +-
 .../stats/DefaultStatisticsCollector.java       |   4 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |   4 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |  12 +-
 .../transaction/OmidTransactionTable.java       |  34 +---
 .../transaction/PhoenixTransactionalTable.java  |  22 +--
 .../transaction/TephraTransactionTable.java     | 101 +++++-----
 .../phoenix/transaction/TransactionFactory.java |   6 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |  19 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  14 +-
 .../org/apache/phoenix/util/ServerUtil.java     |  21 +--
 .../apache/phoenix/util/TransactionUtil.java    |   4 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  93 +++++-----
 .../hbase/index/write/FakeTableFactory.java     |  12 +-
 .../hbase/index/write/TestIndexWriter.java      |   8 +-
 .../index/write/TestParalleIndexWriter.java     |  12 +-
 .../write/TestParalleWriterIndexCommitter.java  |  12 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |  11 +-
 .../hive/mapreduce/PhoenixInputFormat.java      |   7 +-
 67 files changed, 488 insertions(+), 516 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 6b57148..aeb892e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -34,7 +34,7 @@ import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -747,7 +747,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
-            HTableInterface htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
+            Table htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
             assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
             assertFalse(phoenixConn.getTable(new PTableKey(null, baseTableName)).isTransactional());
             assertFalse(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 478b234..b024d03 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -33,10 +33,10 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.apache.phoenix.jdbc.PhoenixConnection;
@@ -85,7 +85,7 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
         // Confirm that dropping the view also deletes the rows in the index
         if (saltBuckets == null) {
             try (Connection conn = DriverManager.getConnection(getUrl())) {
-                HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
+                Table htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
                 if(ScanUtil.isLocalIndex(scan)) {
                     ScanUtil.setLocalIndexAttributes(scan, 0, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, scan.getStartRow(), scan.getStopRow());
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
index 43dc302..08ecee6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
@@ -45,8 +45,8 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.types.PInteger;
@@ -238,7 +238,7 @@ public class ColumnProjectionOptimizationIT extends ParallelStatsDisabledIT {
         byte[] c1 = Bytes.toBytes("COL1");
         byte[] c2 = Bytes.toBytes("COL2");
         byte[] c3 = Bytes.toBytes("COL3");
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = conn2.getQueryServices().getTable(htableName);
             Put put = new Put(PInteger.INSTANCE.toBytes(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
index 8eab9e2..01fc24c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
@@ -27,7 +27,7 @@ import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -58,7 +58,7 @@ public class DisableLocalIndexIT extends ParallelStatsDisabledIT {
         assertFalse(admin.tableExists(Bytes.toBytes(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + tableName)));
         admin.close();
         try {
-            HTableInterface t = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + tableName));
+            Table t = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + tableName));
             t.getTableDescriptor(); // Exception no longer thrown by getTable, but instead need to force an RPC
             fail("Local index table should not have been created");
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
index 70c56a0..714f80a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.ConnectionQueryServices;
@@ -76,7 +76,7 @@ public class DynamicColumnIT extends ParallelStatsDisabledIT {
                 admin.createTable(htd);
             }
 
-            try (HTableInterface hTable = services.getTable(Bytes.toBytes(tableName))) {
+            try (Table hTable = services.getTable(Bytes.toBytes(tableName))) {
                 // Insert rows using standard HBase mechanism with standard HBase "types"
                 List<Row> mutations = new ArrayList<Row>();
                 byte[] dv = Bytes.toBytes("DV");
@@ -96,7 +96,7 @@ public class DynamicColumnIT extends ParallelStatsDisabledIT {
                 put.addColumn(FAMILY_NAME_B, f2v2, Bytes.toBytes("f2value2"));
                 mutations.add(put);
 
-                hTable.batch(mutations);
+                hTable.batch(mutations, null);
 
                 // Create Phoenix table after HBase table was created through the native APIs
                 // The timestamp of the table creation must be later than the timestamp of the data

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
index acae6ee..84c00ba 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicFamilyIT.java
@@ -33,9 +33,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.exception.PhoenixParserException;
@@ -109,7 +109,7 @@ public class DynamicFamilyIT extends ParallelStatsDisabledIT {
     @SuppressWarnings("deprecation")
     private static void initTableValues() throws Exception {
         ConnectionQueryServices services = driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
-        HTableInterface hTable = services.getTable(SchemaUtil.getTableNameAsBytes(WEB_STATS_SCHEMA_NAME,WEB_STATS));
+        Table hTable = services.getTable(SchemaUtil.getTableNameAsBytes(WEB_STATS_SCHEMA_NAME,WEB_STATS));
         try {
             // Insert rows using standard HBase mechanism with standard HBase "types"
             Put put;
@@ -136,7 +136,7 @@ public class DynamicFamilyIT extends ParallelStatsDisabledIT {
             put.addColumn(B_CF, ByteUtil.concat(LAST_LOGIN_TIME_DYNCOL_PREFIX, USER_ID3_BYTES), PTime.INSTANCE.toBytes(ENTRY3_USER_ID3_LOGIN_TIME));
             mutations.add(put);
 
-            hTable.batch(mutations);
+            hTable.batch(mutations, null);
 
         } finally {
             hTable.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
index 546c133..5173fe4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
@@ -38,11 +38,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -66,7 +66,7 @@ public class MappingTableDataTypeIT extends ParallelStatsDisabledIT {
             descriptor.addFamily(columnDescriptor1);
             descriptor.addFamily(columnDescriptor2);
             admin.createTable(descriptor);
-            HTableInterface t = conn.getQueryServices().getTable(Bytes.toBytes(mtest));
+            Table t = conn.getQueryServices().getTable(Bytes.toBytes(mtest));
             insertData(tableName.getName(), admin, t);
             t.close();
             // create phoenix table that maps to existing HBase table
@@ -104,14 +104,13 @@ public class MappingTableDataTypeIT extends ParallelStatsDisabledIT {
         }
     }
 
-    private void insertData(final byte[] tableName, HBaseAdmin admin, HTableInterface t) throws IOException,
+    private void insertData(final byte[] tableName, HBaseAdmin admin, Table t) throws IOException,
             InterruptedException {
         Put p = new Put(Bytes.toBytes("row"));
         p.addColumn(Bytes.toBytes("cf1"), Bytes.toBytes("q1"), Bytes.toBytes("value1"));
         p.addColumn(Bytes.toBytes("cf2"), Bytes.toBytes("q2"), Bytes.toBytes("value2"));
         t.put(p);
-        t.flushCommits();
-        admin.flush(tableName);
+        admin.flush(TableName.valueOf(tableName));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
index 08b8cc6..2f39ec8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndPointIT.java
@@ -22,7 +22,7 @@ import static org.junit.Assert.assertEquals;
 import java.sql.Connection;
 import java.sql.DriverManager;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -44,7 +44,7 @@ public class MetaDataEndPointIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + "(k INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER) COLUMN_ENCODED_BYTES = 0, STORE_NULLS=true, GUIDE_POSTS_WIDTH=1000");
             conn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + fullTableName + " (v1) INCLUDE (v2)");
             conn.commit();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             MutationCode code = IndexUtil.updateIndexState(fullIndexName1, 0L, metaTable, PIndexState.DISABLE).getMutationCode();
             assertEquals(MutationCode.TABLE_ALREADY_EXISTS, code);
             long ts = EnvironmentEdgeManager.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
index 50563d4..3b17ad1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
@@ -38,10 +38,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 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.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Row;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -84,7 +84,7 @@ public class NativeHBaseTypesIT extends ParallelStatsDisabledIT {
         }
         
         ConnectionQueryServices services = driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
-        HTableInterface hTable = services.getTable(tableBytes);
+        Table hTable = services.getTable(tableBytes);
         try {
             // Insert rows using standard HBase mechanism with standard HBase "types"
             List<Row> mutations = new ArrayList<Row>();
@@ -132,7 +132,7 @@ public class NativeHBaseTypesIT extends ParallelStatsDisabledIT {
             put.addColumn(family, ulongCol, HConstants.LATEST_TIMESTAMP, Bytes.toBytes(40000L));
             mutations.add(put);
             
-            hTable.batch(mutations);
+            hTable.batch(mutations, null);
             
             Result r = hTable.get(new Get(bKey));
             assertFalse(r.isEmpty());
@@ -273,7 +273,7 @@ public class NativeHBaseTypesIT extends ParallelStatsDisabledIT {
         String tableName = initTableValues();
         String query = "SELECT string_key FROM " + tableName + " WHERE uint_key > 100000";
         PhoenixConnection conn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
-        HTableInterface hTable = conn.getQueryServices().getTable(tableName.getBytes());
+        Table hTable = conn.getQueryServices().getTable(tableName.getBytes());
         
         List<Row> mutations = new ArrayList<Row>();
         byte[] family = Bytes.toBytes("1");
@@ -290,7 +290,7 @@ public class NativeHBaseTypesIT extends ParallelStatsDisabledIT {
         put.addColumn(family, ulongCol, HConstants.LATEST_TIMESTAMP, Bytes.toBytes(100L));
         put.addColumn(family, QueryConstants.EMPTY_COLUMN_BYTES, HConstants.LATEST_TIMESTAMP, ByteUtil.EMPTY_BYTE_ARRAY);
         mutations.add(put);
-        hTable.batch(mutations);
+        hTable.batch(mutations, null);
     
         // Demonstrates weakness of HBase Bytes serialization. Negative numbers
         // show up as bigger than positive numbers

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index 72ff21e..5652b58 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -29,10 +29,10 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -55,7 +55,7 @@ import org.junit.Test;
 import com.google.common.collect.Sets;
 
 public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
-    private static void assertTenantIds(Expression e, HTableInterface htable, Filter filter, String[] tenantIds) throws IOException {
+    private static void assertTenantIds(Expression e, Table htable, Filter filter, String[] tenantIds) throws IOException {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         Scan scan = new Scan();
         scan.setFilter(filter);
@@ -109,17 +109,17 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         Connection tsconn = DriverManager.getConnection(getUrl(), props);
         tsconn.createStatement().execute("CREATE SEQUENCE " + sequenceName);
         Expression e1 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
-        HTableInterface htable1 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
+        Table htable1 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
         assertTenantIds(e1, htable1, new FirstKeyOnlyFilter(), new String[] {"", t1} );
 
         String viewName = generateUniqueName();
         tsconn.createStatement().execute("CREATE VIEW " + viewName + "(V1 VARCHAR) AS SELECT * FROM " + tableName);
         Expression e2 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
-        HTableInterface htable2 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+        Table htable2 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
         assertTenantIds(e2, htable2, getUserTableAndViewsFilter(), new String[] {"", t1} );
         
         Expression e3 = PhoenixRuntime.getTenantIdExpression(conn, tableName);
-        HTableInterface htable3 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
+        Table htable3 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
         assertTenantIds(e3, htable3, new FirstKeyOnlyFilter(), new String[] {t1, t2} );
 
         String basTableName = generateUniqueName();
@@ -130,13 +130,13 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         String indexName1 = generateUniqueName();
         tsconn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + viewName + "(V1)");
         Expression e5 = PhoenixRuntime.getTenantIdExpression(tsconn, indexName1);
-        HTableInterface htable5 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX + tableName));
+        Table htable5 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX + tableName));
         assertTenantIds(e5, htable5, new FirstKeyOnlyFilter(), new String[] {t1} );
 
         String indexName2 = generateUniqueName();
         conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON " + tableName + "(k2)");
         Expression e6 = PhoenixRuntime.getTenantIdExpression(conn, indexName2);
-        HTableInterface htable6 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName2));
+        Table htable6 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName2));
         assertTenantIds(e6, htable6, new FirstKeyOnlyFilter(), new String[] {t1, t2} );
         
         tableName = generateUniqueName() + "BAR_" + (isSalted ? "SALTED" : "UNSALTED");
@@ -144,7 +144,7 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('" + t1 + "','x')");
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('" + t2 + "','y')");
         Expression e7 = PhoenixRuntime.getFirstPKColumnExpression(conn, tableName);
-        HTableInterface htable7 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
+        Table htable7 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
         assertTenantIds(e7, htable7, new FirstKeyOnlyFilter(), new String[] {t1, t2} );
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index 6d675f5..af5a52a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
@@ -863,7 +863,7 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
                 // expected to fail b/c table is read-only
             }
 
-            HTableInterface htable =
+            Table htable =
                     pconn.getQueryServices()
                             .getTable(SchemaUtil.getTableNameAsBytes(schemaName, tableName));
             Put put = new Put(Bytes.toBytes("0"));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
index 33a891f..363b657 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RebuildIndexConnectionPropsIT.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.phoenix.coprocessor.MetaDataRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDriver;
@@ -103,11 +103,11 @@ public class RebuildIndexConnectionPropsIT extends BaseUniqueNamesOwnClusterIT {
                     Long.toString(NUM_RPC_RETRIES),
                     rebuildQueryServicesConfig.get(HConstants.HBASE_CLIENT_RETRIES_NUMBER));
                 ConnectionQueryServices rebuildQueryServices = rebuildIndexConnection.getQueryServices();
-                HConnection rebuildIndexHConnection =
-                        (HConnection) Whitebox.getInternalState(rebuildQueryServices,
+                Connection rebuildIndexHConnection =
+                        (Connection) Whitebox.getInternalState(rebuildQueryServices,
                             "connection");
-                HConnection regularHConnection =
-                        (HConnection) Whitebox.getInternalState(
+                Connection regularHConnection =
+                        (Connection) Whitebox.getInternalState(
                             regularConnection.getQueryServices(), "connection");
                 // assert that a new HConnection was created
                 assertFalse(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index da8e78d..e18552a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -41,10 +41,10 @@ import java.util.Random;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -459,7 +459,7 @@ public abstract class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         Scan scan = new Scan();
         scan.setRaw(true);
         PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
-        try (HTableInterface htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
             ResultScanner scanner = htable.getScanner(scan);
             Result result;
             while ((result = scanner.next())!=null) {
@@ -472,7 +472,7 @@ public abstract class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         scan = new Scan();
         scan.setRaw(true);
         phxConn = conn.unwrap(PhoenixConnection.class);
-        try (HTableInterface htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
             ResultScanner scanner = htable.getScanner(scan);
             Result result;
             while ((result = scanner.next())!=null) {
@@ -709,7 +709,7 @@ public abstract class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPDATE STATISTICS " + tableName);
             ConnectionQueryServices queryServices =
                     conn.unwrap(PhoenixConnection.class).getQueryServices();
-            try (HTableInterface statsHTable =
+            try (Table statsHTable =
                     queryServices.getTable(
                         SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES,
                             queryServices.getProps()).getName())) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java
index 25e2367..4c85a0c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java
@@ -21,7 +21,7 @@ import java.sql.ResultSet;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixDriver;
@@ -128,7 +128,7 @@ public class UpdateCacheAcrossDifferentClientsIT extends BaseUniqueNamesOwnClust
             String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
             conn1.createStatement().execute("CREATE TABLE " + fullTableName + "(k INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER) COLUMN_ENCODED_BYTES = 0, STORE_NULLS=true");
             conn1.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
-            HTableInterface metaTable = conn2.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn2.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, 0, metaTable, PIndexState.DISABLE);
             conn2.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES(1,2,3)");
             conn2.commit();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 551247f..14c5d8a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -47,9 +47,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RowMutations;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -626,7 +626,7 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
 
         try (PhoenixConnection conn =
                 (DriverManager.getConnection(getUrl())).unwrap(PhoenixConnection.class)) {
-            try (HTableInterface htable =
+            try (Table htable =
                     conn.getQueryServices().getTable(
                         Bytes.toBytes(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME))) {
                 RowMutations mutations = new RowMutations(rowKey);
@@ -731,13 +731,12 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
     private void putUnlockKVInSysMutex(byte[] row) throws Exception {
         try (Connection conn = getConnection(false, null)) {
             ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-            try (HTableInterface sysMutexTable = services.getTable(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME_BYTES)) {
+            try (Table sysMutexTable = services.getTable(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME_BYTES)) {
                 byte[] family = PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES;
                 byte[] qualifier = UPGRADE_MUTEX;
                 Put put = new Put(row);
                 put.addColumn(family, qualifier, UPGRADE_MUTEX_UNLOCKED);
                 sysMutexTable.put(put);
-                sysMutexTable.flushCommits();
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index e89cc58..8365ca0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -37,10 +37,10 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -662,7 +662,7 @@ public class UpsertValuesIT extends ParallelStatsDisabledIT {
         }
         // Issue a raw hbase scan and assert that key values have the expected column qualifiers.
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            HTableInterface table = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullTableName));
+            Table table = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullTableName));
             ResultScanner scanner = table.getScanner(new Scan());
             Result next = scanner.next();
             assertTrue(next.containsColumn(Bytes.toBytes("CF1"), PInteger.INSTANCE.toBytes(1)));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
index 049416c..c1f0628 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
@@ -46,10 +46,10 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
@@ -954,7 +954,7 @@ public abstract class BaseIndexIT extends ParallelStatsDisabledIT {
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
             PTable index = pconn.getTable(new PTableKey(null, fullIndexName));
             byte[] physicalIndexTable = index.getPhysicalName().getBytes();
-            try (HTableInterface hIndex = pconn.getQueryServices().getTable(physicalIndexTable)) {
+            try (Table hIndex = pconn.getQueryServices().getTable(physicalIndexTable)) {
                 Scan scan = new Scan();
                 scan.setRaw(true);
                 if (this.transactional) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 48221ab..6ea96a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -46,10 +46,10 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 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.HTableInterface;
 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.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
@@ -577,7 +577,7 @@ public class LocalIndexIT extends BaseLocalIndexIT {
     public void testLocalIndexAutomaticRepair() throws Exception {
         if (isNamespaceMapped) { return; }
         PhoenixConnection conn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
-        try (HTableInterface metaTable = conn.getQueryServices().getTable(TableName.META_TABLE_NAME.getName());
+        try (Table metaTable = conn.getQueryServices().getTable(TableName.META_TABLE_NAME.getName());
                 HBaseAdmin admin = conn.getQueryServices().getAdmin();) {
             Statement statement = conn.createStatement();
             final String tableName = "T_AUTO_MATIC_REPAIR";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index e46a213..e1d0b31 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
@@ -658,7 +658,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
         TableName indexTable = TableName.valueOf(localIndex?tableName: indexName);
         admin.flush(indexTable);
         boolean merged = false;
-        HTableInterface table = connectionQueryServices.getTable(indexTable.getName());
+        Table table = connectionQueryServices.getTable(indexTable.getName());
         // merge regions until 1 left
         long numRegions = 0;
         while (true) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
index a1da339..61cca0b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/PartialIndexRebuilderIT.java
@@ -37,8 +37,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
@@ -202,7 +202,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
         final String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
         final String fullIndexName = SchemaUtil.getTableName(schemaName, indexName);
         Connection conn = DriverManager.getConnection(getUrl());
-        HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+        Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
         conn.createStatement().execute("CREATE TABLE " + fullTableName + "(k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 INTEGER, CONSTRAINT pk PRIMARY KEY (k1,k2)) STORE_NULLS=true, VERSIONS=1");
         conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + "(v1)");
         
@@ -337,7 +337,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.commit();
             clock.time += 100;
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName1, disableTS, metaTable, PIndexState.DISABLE);
             IndexUtil.updateIndexState(fullIndexName2, disableTS, metaTable, PIndexState.DISABLE);
             clock.time += 100;
@@ -367,7 +367,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
             mutateRandomly(conn, fullTableName, nRows);
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             boolean[] cancel = new boolean[1];
             try {
@@ -395,7 +395,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + "(k INTEGER PRIMARY KEY, v1 INTEGER, v2 INTEGER) COLUMN_ENCODED_BYTES = 0, STORE_NULLS=true");
             conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
             mutateRandomly(conn, fullTableName, nRows);
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             final boolean[] hasInactiveIndex = new boolean[1];
             final CountDownLatch doneSignal = new CountDownLatch(1);
             Runnable r = new Runnable() {
@@ -446,7 +446,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','bb')");
             conn.commit();
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','ccc')");
             conn.commit();
@@ -476,7 +476,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a')");
             conn.commit();
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a',null)");
             conn.commit();
@@ -506,7 +506,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a',null)");
             conn.commit();
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','bb')");
             conn.commit();
@@ -536,7 +536,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a',null)");
             conn.commit();
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','b')");
             conn.commit();
@@ -566,7 +566,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("DELETE FROM " + fullTableName);
             conn.commit();
             long disableTS = EnvironmentEdgeManager.currentTimeMillis();
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, disableTS, metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','b')");
             conn.commit();
@@ -622,7 +622,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, clock.currentTime(), metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','bb')");
             conn.commit();
@@ -658,7 +658,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a','0')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, 0L, metaTable, PIndexState.DISABLE);
             clock.time += 100;
             long disableTime = clock.currentTime();
@@ -705,7 +705,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a','0')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             long disableTime = clock.currentTime();
             IndexUtil.updateIndexState(fullIndexName, disableTime, metaTable, PIndexState.DISABLE);
             clock.time += 100;
@@ -752,7 +752,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             clock.time += 100;
             conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + fullTableName + " (v1, v2)");
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, 0L, metaTable, PIndexState.DISABLE);
             clock.time += 100;
             long disableTime = clock.currentTime();
@@ -801,7 +801,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a','0','x')");
             conn.commit();
             clock.time += 100;
-            try (HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES)) {
+            try (Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES)) {
                 // By using an INDEX_DISABLE_TIMESTAMP of 0, we prevent the partial index rebuilder from triggering
                 IndexUtil.updateIndexState(fullIndexName, 0L, metaTable, PIndexState.DISABLE);
                 clock.time += 100;
@@ -868,7 +868,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a','0')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
 
             long disableTime = clock.currentTime();
             // Simulates an index write failure
@@ -952,7 +952,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, clock.currentTime(), metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("DELETE FROM " + fullTableName + " WHERE k='a'");
             conn.commit();
@@ -986,7 +986,7 @@ public class PartialIndexRebuilderIT extends BaseUniqueNamesOwnClusterIT {
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','a')");
             conn.commit();
             clock.time += 100;
-            HTableInterface metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            Table metaTable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
             IndexUtil.updateIndexState(fullIndexName, clock.currentTime(), metaTable, PIndexState.DISABLE);
             conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES('a','ccc')");
             conn.commit();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
index 301768b..dca651e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/FlappingTransactionIT.java
@@ -34,9 +34,9 @@ import java.sql.Statement;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 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.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -214,7 +214,7 @@ public class FlappingTransactionIT extends ParallelStatsDisabledIT {
         
         Statement stmt = conn.createStatement();
         stmt.execute("CREATE TABLE " + fullTableName + "(K VARCHAR PRIMARY KEY, V1 VARCHAR, V2 VARCHAR) TRANSACTIONAL=true");
-        HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(fullTableName));
+        Table htable = pconn.getQueryServices().getTable(Bytes.toBytes(fullTableName));
         stmt.executeUpdate("upsert into " + fullTableName + " values('x', 'a', 'a')");
         conn.commit();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
index 897007d..042d915 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
@@ -39,8 +39,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
@@ -270,7 +270,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         String index = generateUniqueName();
         conn.createStatement().execute("CREATE INDEX " + index + " ON " + nonTxTableName + "(v)");
         // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
+        Table htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
         List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
         for (Put put : puts) {
             put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
@@ -331,7 +331,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute("UPSERT INTO \"SYSTEM\"." + nonTxTableName + " VALUES (1)");
         conn.commit();
         // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
+        Table htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
         Put put = new Put(PInteger.INSTANCE.toBytes(1));
         put.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
         htable.put(put);
@@ -373,7 +373,7 @@ public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute(ddl);
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
         PTable table = pconn.getTable(new PTableKey(null, t1));
-        HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
+        Table htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
         assertTrue(table.isTransactional());
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
index 26b6f01..e9b5b37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
@@ -43,6 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.BlockingRpcCallback;
@@ -256,7 +257,7 @@ public class ServerCacheClient {
                     servers.add(entry);
                     if (LOG.isDebugEnabled()) {LOG.debug(addCustomAnnotations("Adding cache entry to be sent for " + entry, connection));}
                     final byte[] key = getKeyInRegion(entry.getRegionInfo().getStartKey());
-                    final HTableInterface htable = services.getTable(cacheUsingTableRef.getTable().getPhysicalName().getBytes());
+                    final Table htable = services.getTable(cacheUsingTableRef.getTable().getPhysicalName().getBytes());
                     closeables.add(htable);
                     futures.add(executor.submit(new JobCallable<Boolean>() {
                         
@@ -330,7 +331,7 @@ public class ServerCacheClient {
      * @throws IllegalStateException if hashed table cannot be removed on any region server on which it was added
      */
     private void removeServerCache(final ServerCache cache, Set<HRegionLocation> remainingOnServers) throws SQLException {
-        HTableInterface iterateOverTable = null;
+        Table iterateOverTable = null;
         final byte[] cacheId = cache.getId();
         try {
             ConnectionQueryServices services = connection.getQueryServices();
@@ -431,7 +432,7 @@ public class ServerCacheClient {
 
     public boolean addServerCache(byte[] startkeyOfRegion, ServerCache cache, HashCacheFactory cacheFactory,
              byte[] txState, PTable pTable) throws Exception {
-        HTableInterface table = null;
+        Table table = null;
         boolean success = true;
         byte[] cacheId = cache.getId();
         try {
@@ -453,7 +454,7 @@ public class ServerCacheClient {
         }
     }
     
-    public boolean addServerCache(HTableInterface htable, byte[] key, final PTable cacheUsingTable, final byte[] cacheId,
+    public boolean addServerCache(Table htable, byte[] key, final PTable cacheUsingTable, final byte[] cacheId,
             final ImmutableBytesWritable cachePtr, final ServerCacheFactory cacheFactory, final byte[] txState)
             throws Exception {
         byte[] keyInRegion = getKeyInRegion(key);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index f88b34b..0d06f0a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -29,7 +29,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
@@ -187,7 +187,7 @@ public class FromCompiler {
                 boolean isNamespaceMapped = SchemaUtil.isNamespaceMappingEnabled(statement.getTableType(), connection.getQueryServices().getProps());
                 byte[] fullTableName = SchemaUtil.getPhysicalHBaseTableName(
                     baseTable.getSchemaName(), baseTable.getTableName(), isNamespaceMapped).getBytes();
-                HTableInterface htable = null;
+                Table htable = null;
                 try {
                     htable = services.getTable(fullTableName);
                 } catch (UnsupportedOperationException ignore) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
index 380212e..da5e7a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionCoprocessorEnvironment.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@@ -59,7 +59,6 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
     public Coprocessor getInstance() {
         return delegate.getInstance();
     }
-
     @Override
     public int getPriority() {
         return delegate.getPriority();
@@ -76,12 +75,12 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
     }
 
     @Override
-    public HTableInterface getTable(TableName tableName) throws IOException {
+    public Table getTable(TableName tableName) throws IOException {
         return delegate.getTable(tableName);
     }
 
     @Override
-    public HTableInterface getTable(TableName tableName, ExecutorService service)
+    public Table getTable(TableName tableName, ExecutorService service)
             throws IOException {
         return delegate.getTable(tableName, service);
     }
@@ -102,13 +101,13 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
     }
 
     @Override
-    public RegionServerServices getRegionServerServices() {
-        return delegate.getRegionServerServices();
+    public ConcurrentMap<String, Object> getSharedData() {
+        return delegate.getSharedData();
     }
 
     @Override
-    public ConcurrentMap<String, Object> getSharedData() {
-        return delegate.getSharedData();
+    public RegionServerServices getRegionServerServices() {
+        return delegate.getRegionServerServices();
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index fe13c09..a42e1b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -107,12 +107,12 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 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.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -1711,10 +1711,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // RegionScanner scanner = region.getScanner(scan);
         // The following *should* work, but doesn't due to HBASE-11837
         // TableName systemCatalogTableName = region.getTableDesc().getTableName();
-        // HTableInterface hTable = env.getTable(systemCatalogTableName);
+        // Table hTable = env.getTable(systemCatalogTableName);
         // These deprecated calls work around the issue
-        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
-            region.getTableDesc().getTableName().getName())) {
+        try (Table hTable = ServerUtil.getHTableForCoprocessorScan(env,
+            region.getTableDesc().getTableName())) {
             boolean allViewsInCurrentRegion = true;
             int numOfChildViews = 0;
             List<ViewInfo> viewInfoList = Lists.newArrayList();
@@ -1760,10 +1760,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // RegionScanner scanner = region.getScanner(scan);
         // The following *should* work, but doesn't due to HBASE-11837
         // TableName systemCatalogTableName = region.getTableDesc().getTableName();
-        // HTableInterface hTable = env.getTable(systemCatalogTableName);
+        // Table hTable = env.getTable(systemCatalogTableName);
         // These deprecated calls work around the issue
-        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
-            region.getTableDesc().getTableName().getName())) {
+        try (Table hTable = ServerUtil.getHTableForCoprocessorScan(env,
+            region.getTableDesc().getTableName())) {
             boolean allViewsInCurrentRegion = true;
             int numOfChildViews = 0;
             List<ViewInfo> viewInfoList = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11390427/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index e11ff14..e7b9a35 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -160,8 +160,8 @@ public class MetaDataRegionObserver implements RegionObserver,RegionCoprocessor
         Runnable r = new Runnable() {
             @Override
             public void run() {
-                HTableInterface metaTable = null;
-                HTableInterface statsTable = null;
+                Table metaTable = null;
+                Table statsTable = null;
                 try {
                     ReadOnlyProps props=new ReadOnlyProps(env.getConfiguration().iterator());
                     Thread.sleep(1000);
@@ -411,7 +411,7 @@ public class MetaDataRegionObserver implements RegionObserver,RegionCoprocessor
 						List<Pair<PTable,Long>> pairs = entry.getValue();
                         List<PTable> indexesToPartiallyRebuild = Lists.newArrayListWithExpectedSize(pairs.size());
 						try (
-                        HTableInterface metaTable = env.getTable(
+                        Table metaTable = env.getTable(
 								SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props))) {
 							long earliestDisableTimestamp = Long.MAX_VALUE;
                             long latestUpperBoundTimestamp = Long.MIN_VALUE;