You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2020/03/09 13:51:41 UTC

[phoenix] 02/02: PHOENIX-5721 Unify 4.x branches (add HBase 1.3)

This is an automated email from the ASF dual-hosted git repository.

stoty pushed a commit to branch 4.x
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit a0cff1744bb76a6fc1b79a3aa43859418adb8850
Author: Istvan Toth <st...@apache.org>
AuthorDate: Thu Feb 27 10:08:23 2020 +0100

    PHOENIX-5721 Unify 4.x branches (add HBase 1.3)
    
    adds HBase 1.3 compatibility
---
 phoenix-core/pom.xml                               |  3 +
 .../ipc/controller/ClientRpcControllerFactory.java | 19 ++---
 .../hbase/ipc/controller/IndexRpcController.java   |  8 +--
 ...InterRegionServerIndexRpcControllerFactory.java | 21 +++---
 ...erRegionServerMetadataRpcControllerFactory.java | 19 ++---
 .../ipc/controller/MetadataRpcController.java      |  8 +--
 .../regionserver/IndexHalfStoreFileReader.java     | 22 ++++--
 .../regionserver/LocalIndexStoreFileScanner.java   |  4 +-
 .../DelegateRegionCoprocessorEnvironment.java      |  9 +--
 .../org/apache/phoenix/execute/DelegateHTable.java | 26 +------
 .../apache/phoenix/index/GlobalIndexChecker.java   | 15 ++--
 .../phoenix/iterate/ScanningResultIterator.java    |  6 +-
 .../apache/phoenix/iterate/SnapshotScanner.java    |  8 +--
 .../phoenix/monitoring/GlobalClientMetrics.java    | 43 +----------
 .../monitoring/GlobalClientMetricsRegistry.java    | 49 +++++++++++++
 .../phoenix/schema/stats/UpdateStatisticsTool.java | 14 ++--
 .../phoenix/transaction/OmidTransactionTable.java  |  8 +--
 .../java/org/apache/phoenix/query/BaseTest.java    | 26 +++++--
 phoenix-hbase-compat-1.3.0/pom.xml                 | 83 ++++++++++++++++++++++
 .../hbase/regionserver/CompatStoreFileScanner.java | 33 +++++++++
 .../phoenix/compat/hbase/CompatDelegateHTable.java | 28 ++++++++
 ...CompatDelegateRegionCoprocessorEnvironment.java | 27 +++++++
 .../compat/hbase/CompatDelegateRegionObserver.java | 29 ++++++++
 .../hbase/CompatDelegatingHBaseRpcController.java  | 57 +++++++++++++++
 .../compat/hbase/CompatHBaseRpcController.java     | 39 ++++++++++
 .../compat/hbase/CompatObserverContext.java        | 32 +++++++++
 ...CompatPhoenixMetaDataControllerEnvironment.java | 36 ++++++++++
 .../compat/hbase/CompatPhoenixRpcScheduler.java    | 24 +++++++
 .../hbase/CompatRegionCoprocessorEnvironment.java  | 23 ++++++
 .../compat/hbase/CompatRpcControllerFactory.java   | 48 +++++++++++++
 .../compat/hbase/CompatScanningResultIterator.java | 21 ++++++
 .../apache/phoenix/compat/hbase/CompatUtil.java    | 47 ++++++++++++
 .../CompatGlobalClientMetricsRegistry.java         | 40 +++++++++++
 .../hbase/regionserver/CompatStoreFileScanner.java | 33 +++++++++
 .../phoenix/compat/hbase/CompatDelegateHTable.java | 48 +++++++++++++
 ...CompatDelegateRegionCoprocessorEnvironment.java | 32 +++++++++
 .../hbase/CompatDelegatingHBaseRpcController.java  | 30 ++++++++
 .../compat/hbase/CompatHBaseRpcController.java     | 23 ++++++
 .../compat/hbase/CompatHBaseRpcControllerImpl.java | 41 +++++++++++
 .../hbase/CompatRegionCoprocessorEnvironment.java  | 29 ++++++++
 .../compat/hbase/CompatRpcControllerFactory.java   | 48 +++++++++++++
 .../apache/phoenix/compat/hbase/CompatUtil.java    | 36 ++++++++++
 .../CompatGlobalClientMetricsRegistry.java         | 62 ++++++++++++++++
 .../monitoring/GlobalMetricRegistriesAdapter.java  | 50 +++++++------
 .../hbase/regionserver/CompatStoreFileScanner.java | 33 +++++++++
 .../phoenix/compat/hbase/CompatDelegateHTable.java | 48 +++++++++++++
 ...CompatDelegateRegionCoprocessorEnvironment.java | 32 +++++++++
 .../hbase/CompatDelegatingHBaseRpcController.java  | 30 ++++++++
 .../compat/hbase/CompatHBaseRpcController.java     | 23 ++++++
 .../compat/hbase/CompatHBaseRpcControllerImpl.java | 41 +++++++++++
 .../hbase/CompatRegionCoprocessorEnvironment.java  | 29 ++++++++
 .../compat/hbase/CompatRpcControllerFactory.java   | 48 +++++++++++++
 .../apache/phoenix/compat/hbase/CompatUtil.java    | 36 ++++++++++
 .../CompatGlobalClientMetricsRegistry.java         | 62 ++++++++++++++++
 .../monitoring/GlobalMetricRegistriesAdapter.java  | 50 +++++++------
 pom.xml                                            | 26 +++++++
 56 files changed, 1576 insertions(+), 189 deletions(-)

diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index fdec241..215bc54 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -546,6 +546,9 @@
         </dependency>
       </dependencies>
     </profile>
+    <profile>
+      <id>phoenix-hbase-compat-1.3.0</id>
+    </profile>
   </profiles>
 
   <reporting>
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/ClientRpcControllerFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/ClientRpcControllerFactory.java
index 9d926b5..c9f7fb9 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/ClientRpcControllerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/ClientRpcControllerFactory.java
@@ -22,38 +22,39 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.phoenix.compat.hbase.CompatHBaseRpcController;
+import org.apache.phoenix.compat.hbase.CompatRpcControllerFactory;
 
 /**
  * {@link RpcControllerFactory} that sets the priority of metadata rpc calls to be processed
  * in its own queue.
  */
-public class ClientRpcControllerFactory extends RpcControllerFactory {
+public class ClientRpcControllerFactory extends CompatRpcControllerFactory {
 
     public ClientRpcControllerFactory(Configuration conf) {
         super(conf);
     }
 
     @Override
-    public HBaseRpcController newController() {
-        HBaseRpcController delegate = super.newController();
+    public CompatHBaseRpcController newController() {
+        CompatHBaseRpcController delegate = super.newController();
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(CellScanner cellScanner) {
-        HBaseRpcController delegate = super.newController(cellScanner);
+    public CompatHBaseRpcController newController(CellScanner cellScanner) {
+        CompatHBaseRpcController delegate = super.newController(cellScanner);
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(List<CellScannable> cellIterables) {
-        HBaseRpcController delegate = super.newController(cellIterables);
+    public CompatHBaseRpcController newController(List<CellScannable> cellIterables) {
+        CompatHBaseRpcController delegate = super.newController(cellIterables);
         return getController(delegate);
     }
     
-    private HBaseRpcController getController(HBaseRpcController delegate) {
+    private CompatHBaseRpcController getController(CompatHBaseRpcController delegate) {
         return new MetadataRpcController(delegate, conf);
     }
     
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/IndexRpcController.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/IndexRpcController.java
index b8976ce..e5aa82d 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/IndexRpcController.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/IndexRpcController.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.ipc.controller;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
+import org.apache.phoenix.compat.hbase.CompatDelegatingHBaseRpcController;
+import org.apache.phoenix.compat.hbase.CompatHBaseRpcController;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 
@@ -31,12 +31,12 @@ import com.google.protobuf.RpcController;
  * {@link RpcController} that sets the appropriate priority of RPC calls destined for Phoenix index
  * tables.
  */
-class IndexRpcController extends DelegatingHBaseRpcController {
+class IndexRpcController extends CompatDelegatingHBaseRpcController {
 
     private final int priority;
     private final String tracingTableName;
     
-    public IndexRpcController(HBaseRpcController delegate, Configuration conf) {
+    public IndexRpcController(CompatHBaseRpcController delegate, Configuration conf) {
         super(delegate);
         this.priority = PhoenixRpcSchedulerFactory.getIndexPriority(conf);
         this.tracingTableName = conf.get(QueryServices.TRACING_STATS_TABLE_NAME_ATTRIB,
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerIndexRpcControllerFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerIndexRpcControllerFactory.java
index c586233..5c07213 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerIndexRpcControllerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerIndexRpcControllerFactory.java
@@ -23,8 +23,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.phoenix.compat.hbase.CompatHBaseRpcController;
+import org.apache.phoenix.compat.hbase.CompatRpcControllerFactory;
 
 /**
  * {@link RpcControllerFactory} that should only be used when creating {@link HTable} for
@@ -32,33 +33,33 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
  * This controller factory shouldn't be globally configured anywhere and is meant to be used
  * only internally by Phoenix indexing code.
  */
-public class InterRegionServerIndexRpcControllerFactory extends RpcControllerFactory {
+public class InterRegionServerIndexRpcControllerFactory extends CompatRpcControllerFactory {
 
     public InterRegionServerIndexRpcControllerFactory(Configuration conf) {
         super(conf);
     }
 
     @Override
-    public HBaseRpcController newController() {
-        HBaseRpcController delegate = super.newController();
+    public CompatHBaseRpcController newController() {
+        CompatHBaseRpcController delegate = super.newController();
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(CellScanner cellScanner) {
-        HBaseRpcController delegate = super.newController(cellScanner);
+    public CompatHBaseRpcController newController(CellScanner cellScanner) {
+        CompatHBaseRpcController delegate = super.newController(cellScanner);
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(List<CellScannable> cellIterables) {
-        HBaseRpcController delegate = super.newController(cellIterables);
+    public CompatHBaseRpcController newController(List<CellScannable> cellIterables) {
+        CompatHBaseRpcController delegate = super.newController(cellIterables);
         return getController(delegate);
     }
 
-    private HBaseRpcController getController(HBaseRpcController delegate) {
+    private CompatHBaseRpcController getController(CompatHBaseRpcController delegate) {
         // construct a chain of controllers: metadata, index and standard controller
-      HBaseRpcController indexRpcController = new IndexRpcController(delegate, conf);
+        CompatHBaseRpcController indexRpcController = new IndexRpcController(delegate, conf);
         return new MetadataRpcController(indexRpcController, conf);
     }
 
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerMetadataRpcControllerFactory.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerMetadataRpcControllerFactory.java
index 98c5ecf..bf961b4 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerMetadataRpcControllerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/InterRegionServerMetadataRpcControllerFactory.java
@@ -23,38 +23,39 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.phoenix.compat.hbase.CompatHBaseRpcController;
+import org.apache.phoenix.compat.hbase.CompatRpcControllerFactory;
 
 /**
  * {@link RpcControllerFactory} that should only be used when creating {@link HTable} for
  * making remote RPCs to the region servers hosting Phoenix SYSTEM tables.
  */
-public class InterRegionServerMetadataRpcControllerFactory extends RpcControllerFactory {
+public class InterRegionServerMetadataRpcControllerFactory extends CompatRpcControllerFactory {
 
     public InterRegionServerMetadataRpcControllerFactory(Configuration conf) {
         super(conf);
     }
 
     @Override
-    public HBaseRpcController newController() {
-        HBaseRpcController delegate = super.newController();
+    public CompatHBaseRpcController newController() {
+        CompatHBaseRpcController delegate = super.newController();
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(CellScanner cellScanner) {
-        HBaseRpcController delegate = super.newController(cellScanner);
+    public CompatHBaseRpcController newController(CellScanner cellScanner) {
+        CompatHBaseRpcController delegate = super.newController(cellScanner);
         return getController(delegate);
     }
 
     @Override
-    public HBaseRpcController newController(List<CellScannable> cellIterables) {
-        HBaseRpcController delegate = super.newController(cellIterables);
+    public CompatHBaseRpcController newController(List<CellScannable> cellIterables) {
+        CompatHBaseRpcController delegate = super.newController(cellIterables);
         return getController(delegate);
     }
 
-    private HBaseRpcController getController(HBaseRpcController delegate) {
+    private CompatHBaseRpcController getController(CompatHBaseRpcController delegate) {
         return new MetadataRpcController(delegate, conf);
     }
 
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
index cbeabaa..e489ee1 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
@@ -21,9 +21,9 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
-import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
+import org.apache.phoenix.compat.hbase.CompatDelegatingHBaseRpcController;
+import org.apache.phoenix.compat.hbase.CompatHBaseRpcController;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -34,7 +34,7 @@ import com.google.protobuf.RpcController;
  * {@link RpcController} that sets the appropriate priority of RPC calls destined for Phoenix SYSTEM
  * tables
  */
-class MetadataRpcController extends DelegatingHBaseRpcController {
+class MetadataRpcController extends CompatDelegatingHBaseRpcController {
 
 	private int priority;
 	// list of system tables
@@ -53,7 +53,7 @@ class MetadataRpcController extends DelegatingHBaseRpcController {
                     .getNameAsString())
             .build();
 
-	public MetadataRpcController(HBaseRpcController delegate,
+	public MetadataRpcController(CompatHBaseRpcController delegate,
 			Configuration conf) {
 		super(delegate);
 		this.priority = PhoenixRpcSchedulerFactory.getMetadataPriority(conf);
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
index e2dff03..818c527 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
@@ -131,11 +131,23 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
         return top;
     }
 
-    @Override
-    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread, boolean isCompaction, long readPt,
-                                                long scannerOrder, boolean canOptimizeForNonNullColumn) {
-        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction), true,
-                getHFileReader().hasMVCCInfo(), readPt, scannerOrder, canOptimizeForNonNullColumn);
+    // These implement the 1.3, and 1.4+ versions of the getStoreFileScanner() interface
+    // The @Override is missing, since we do not know which version will be active at compile time
+
+    //The 1.4+ version
+    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread,
+            boolean isCompaction, long readPt,
+            long scannerOrder, boolean canOptimizeForNonNullColumn) {
+        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction),
+            true, getHFileReader().hasMVCCInfo(), readPt,
+            scannerOrder, canOptimizeForNonNullColumn);
+    }
+
+    //The 1.3 version
+    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread,
+           boolean isCompaction, long readPt) {
+        return new LocalIndexStoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction),
+                true, getHFileReader().hasMVCCInfo(), readPt, 0, false);
     }
 
     @Override
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexStoreFileScanner.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexStoreFileScanner.java
index df279d7..fece89e 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexStoreFileScanner.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexStoreFileScanner.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
@@ -34,7 +32,7 @@ import org.apache.phoenix.index.IndexMaintainer;
 
 import static org.apache.hadoop.hbase.KeyValue.ROW_LENGTH_SIZE;
 
-public class LocalIndexStoreFileScanner extends StoreFileScanner{
+public class LocalIndexStoreFileScanner extends CompatStoreFileScanner {
 
     private IndexHalfStoreFileReader reader;
     private boolean changeBottomKeys;
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 71ed20e..94484ad 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
@@ -27,9 +27,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.phoenix.compat.hbase.CompatDelegateRegionCoprocessorEnvironment;
 import org.apache.phoenix.hbase.index.table.HTableFactory;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.util.ServerUtil;
@@ -40,10 +40,9 @@ import org.apache.phoenix.util.ServerUtil.ConnectionType;
  * clone the configuration provided by the HBase coprocessor environment before modifying it. So
  * this class comes in handy where we have to return our custom config.
  */
-public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEnvironment {
+public class DelegateRegionCoprocessorEnvironment extends CompatDelegateRegionCoprocessorEnvironment {
 
     private final Configuration config;
-    private RegionCoprocessorEnvironment delegate;
     private HTableFactory tableFactory;
 
     public DelegateRegionCoprocessorEnvironment(RegionCoprocessorEnvironment delegate, ConnectionType connectionType) {
@@ -118,8 +117,4 @@ public class DelegateRegionCoprocessorEnvironment implements RegionCoprocessorEn
         return delegate.getSharedData();
     }
 
-    @Override
-    public MetricRegistry getMetricRegistryForRegionServer() {
-        return delegate.getMetricRegistryForRegionServer();
-    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateHTable.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateHTable.java
index 8ca8a91..82e0884 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateHTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateHTable.java
@@ -40,17 +40,17 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.phoenix.compat.hbase.CompatDelegateHTable;
 
 import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 
-public class DelegateHTable implements Table {
-    protected final Table delegate;
+public class DelegateHTable extends CompatDelegateHTable implements Table {
 
     public DelegateHTable(Table delegate) {
-        this.delegate = delegate;
+        super(delegate);
     }
 
     @Override
@@ -268,24 +268,4 @@ public class DelegateHTable implements Table {
 		return delegate.checkAndDelete(row, family, qualifier, compareOp, value, delete);
 	}
 
-  @Override
-  public int getReadRpcTimeout() {
-    return delegate.getReadRpcTimeout();
-  }
-
-  @Override
-  public void setReadRpcTimeout(int readRpcTimeout) {
-    delegate.setReadRpcTimeout(readRpcTimeout);
-  }
-
-  @Override
-  public int getWriteRpcTimeout() {
-    return delegate.getWriteRpcTimeout();
-  }
-
-  @Override
-  public void setWriteRpcTimeout(int writeRpcTimeout) {
-    delegate.setWriteRpcTimeout(writeRpcTimeout);
-  }
-
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java b/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
index 7eb95b5..b65c714 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/GlobalIndexChecker.java
@@ -25,6 +25,9 @@ import static org.apache.phoenix.hbase.index.IndexRegionObserver.VERIFIED_BYTES;
 import static org.apache.phoenix.index.IndexMaintainer.getIndexMaintainer;
 import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 
+import static org.apache.phoenix.compat.hbase.CompatUtil.*;
+
+
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
@@ -297,8 +300,8 @@ public class GlobalIndexChecker extends BaseRegionObserver {
             // Rebuild the index row from the corresponding the row in the the data table
             // Get the data row key from the index row key
             byte[] dataRowKey = indexMaintainer.buildDataRowKey(new ImmutableBytesWritable(indexRowKey), viewConstants);
-            buildIndexScan.withStartRow(dataRowKey, true);
-            buildIndexScan.withStopRow(dataRowKey, true);
+            setStartRow(buildIndexScan, dataRowKey, true);
+            setStopRow(buildIndexScan, dataRowKey, true);
             buildIndexScan.setTimeRange(0, maxTimestamp);
             // Pass the index row key to the partial index builder which will rebuild the index row and check if the
             // row key of this rebuilt index row matches with the passed index row key
@@ -330,7 +333,7 @@ public class GlobalIndexChecker extends BaseRegionObserver {
                 // Delete the unverified row from index if it is old enough
                 deleteRowIfAgedEnough(indexRowKey, row, ts, false);
                 // Open a new scanner starting from the row after the current row
-                indexScan.withStartRow(indexRowKey, false);
+                setStartRow(indexScan, indexRowKey, false);
                 scanner = region.getScanner(indexScan);
                 // Skip this unverified row (i.e., do not return it to the client). Just retuning empty row is
                 // sufficient to do that
@@ -339,7 +342,7 @@ public class GlobalIndexChecker extends BaseRegionObserver {
             }
             // code == RebuildReturnCode.INDEX_ROW_EXISTS.getValue()
             // Open a new scanner starting from the current row
-            indexScan.withStartRow(indexRowKey, true);
+            setStartRow(indexScan, indexRowKey, true);
             scanner = region.getScanner(indexScan);
             scanner.next(row);
             if (row.isEmpty()) {
@@ -378,8 +381,8 @@ public class GlobalIndexChecker extends BaseRegionObserver {
                 // Now we will do a single row scan to retrieve the verified index row built from the data table row.
                 // Note we cannot read all versions in one scan as the max number of row versions for an index table
                 // can be 1. In that case, we will get only one (i.e., the most recent) version instead of all versions
-                singleRowIndexScan.withStartRow(indexRowKey, true);
-                singleRowIndexScan.withStopRow(indexRowKey, true);
+                setStartRow(singleRowIndexScan, indexRowKey, true);
+                setStopRow(singleRowIndexScan, indexRowKey, true);
                 singleRowIndexScan.setTimeRange(minTimestamp, ts);
                 RegionScanner singleRowScanner = region.getScanner(singleRowIndexScan);
                 row.clear();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
index d86a27a..76db0d4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
@@ -59,12 +59,14 @@ import org.apache.phoenix.util.ServerUtil;
 
 public class ScanningResultIterator implements ResultIterator {
     private final ResultScanner scanner;
+    private final Scan scan;
     private final ScanMetricsHolder scanMetricsHolder;
     boolean scanMetricsUpdated;
     boolean scanMetricsEnabled;
 
     public ScanningResultIterator(ResultScanner scanner, Scan scan, ScanMetricsHolder scanMetricsHolder) {
         this.scanner = scanner;
+        this.scan = scan;
         this.scanMetricsHolder = scanMetricsHolder;
         scanMetricsUpdated = false;
         scanMetricsEnabled = scan.isScanMetricsEnabled();
@@ -92,7 +94,9 @@ public class ScanningResultIterator implements ResultIterator {
     private void updateMetrics() {
 
         if (scanMetricsEnabled && !scanMetricsUpdated) {
-            ScanMetrics scanMetrics = scanner.getScanMetrics();
+            ScanMetrics scanMetrics = scan.getScanMetrics();
+            if (scanMetrics == null)
+                return; // See PHOENIX-5345
             Map<String, Long> scanMetricsMap = scanMetrics.getMetricsMap();
             scanMetricsHolder.setScanMetricMap(scanMetricsMap);
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
index 450d9d3..6b0dee7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/SnapshotScanner.java
@@ -34,11 +34,11 @@ 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.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.phoenix.compat.hbase.CompatRegionCoprocessorEnvironment;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -148,7 +148,7 @@ public class SnapshotScanner extends AbstractClientScanner {
   }
 
   private RegionCoprocessorEnvironment getSnapshotContextEnvironment(final Configuration conf) {
-    return new RegionCoprocessorEnvironment() {
+    return new CompatRegionCoprocessorEnvironment() {
       @Override
       public Region getRegion() {
         return region;
@@ -215,10 +215,6 @@ public class SnapshotScanner extends AbstractClientScanner {
         throw new UnsupportedOperationException();
       }
 
-      @Override
-      public MetricRegistry getMetricRegistryForRegionServer() {
-        throw new UnsupportedOperationException();
-      }
     };
   }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
index 810278d..f41ab52 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
@@ -59,10 +59,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.hadoop.hbase.metrics.Gauge;
-import org.apache.hadoop.hbase.metrics.MetricRegistries;
-import org.apache.hadoop.hbase.metrics.MetricRegistry;
-import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
 import org.apache.phoenix.query.QueryServicesOptions;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -121,11 +117,7 @@ public enum GlobalClientMetrics {
 
     static {
         initPhoenixGlobalClientMetrics();
-        if (isGlobalMetricsEnabled) {
-            MetricRegistry metricRegistry = createMetricRegistry();
-            registerPhoenixMetricsToRegistry(metricRegistry);
-            GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry);
-        }
+        GlobalClientMetricsRegistry.register();
     }
 
     private static void initPhoenixGlobalClientMetrics() {
@@ -135,47 +127,14 @@ public enum GlobalClientMetrics {
         }
     }
 
-    private static void registerPhoenixMetricsToRegistry(MetricRegistry metricRegistry) {
-        for (GlobalClientMetrics globalMetric : GlobalClientMetrics.values()) {
-            metricRegistry.register(globalMetric.metricType.columnName(),
-                    new PhoenixGlobalMetricGauge(globalMetric.metric));
-        }
-    }
-
-    private static MetricRegistry createMetricRegistry() {
-        LOGGER.info("Creating Metric Registry for Phoenix Global Metrics");
-        MetricRegistryInfo registryInfo = new MetricRegistryInfo("PHOENIX", "Phoenix Client Metrics",
-                "phoenix", "Phoenix,sub=CLIENT", true);
-        return MetricRegistries.global().create(registryInfo);
-    }
-
-    /**
-     * Class to convert Phoenix Metric objects into HBase Metric objects (Gauge)
-     */
-    private static class PhoenixGlobalMetricGauge implements Gauge<Long> {
-
-        private final GlobalMetric metric;
-
-        public PhoenixGlobalMetricGauge(GlobalMetric metric) {
-            this.metric = metric;
-        }
-
-        @Override
-        public Long getValue() {
-            return metric.getValue();
-        }
-    }
-
     public void update(long value) {
         metric.change(value);
     }
 
-    @VisibleForTesting
     public GlobalMetric getMetric() {
         return metric;
     }
 
-    @VisibleForTesting
     public MetricType getMetricType() {
         return metricType;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetricsRegistry.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetricsRegistry.java
new file mode 100644
index 0000000..60220f8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetricsRegistry.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.monitoring;
+
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GlobalClientMetricsRegistry extends CompatGlobalClientMetricsRegistry {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(GlobalClientMetrics.class);
+
+    public static void register() {
+        if (GlobalClientMetrics.isMetricsEnabled()) {
+            createRegistry();
+            registerPhoenixMetricsToRegistry();
+            registerMetricsAdapter(QueryServicesOptions.withDefaults().getClientMetricTag());
+        }
+    }
+
+    private static void registerPhoenixMetricsToRegistry() {
+        for (final GlobalClientMetrics globalMetric : GlobalClientMetrics.values()) {
+            final GlobalMetric innerMetric = globalMetric.getMetric();
+            registerMetricToRegistry(globalMetric.getMetricType().columnName(),
+                new ValueProvider() {
+                    @Override
+                    public Long getValue() {
+                        return innerMetric.getValue();
+                    }
+                });
+        }
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/UpdateStatisticsTool.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/UpdateStatisticsTool.java
index ccb3376..26c6148 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/UpdateStatisticsTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/UpdateStatisticsTool.java
@@ -30,8 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.metrics.Gauge;
-import org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobPriority;
@@ -41,6 +39,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.htrace.SpanReceiver;
+import org.apache.phoenix.compat.hbase.CompatUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
@@ -215,10 +214,13 @@ public class UpdateStatisticsTool extends Configured implements Tool {
         job.setPriority(this.jobPriority);
 
         TableMapReduceUtil.addDependencyJars(job);
-        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), PhoenixConnection.class, Chronology.class,
-                CharStream.class, TransactionSystemClient.class, TransactionNotInProgressException.class,
-                ZKClient.class, DiscoveryServiceClient.class, ZKDiscoveryService.class,
-                Cancellable.class, TTransportException.class, SpanReceiver.class, TransactionProcessor.class, Gauge.class, MetricRegistriesImpl.class);
+        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
+            PhoenixConnection.class, Chronology.class, CharStream.class,
+            TransactionSystemClient.class, TransactionNotInProgressException.class, ZKClient.class,
+            DiscoveryServiceClient.class, ZKDiscoveryService.class, Cancellable.class,
+            TTransportException.class, SpanReceiver.class, TransactionProcessor.class);
+        TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
+            CompatUtil.getMrMetricsClasses());
         LOGGER.info("UpdateStatisticsTool running for: " + tableName
                 + " on snapshot: " + snapshotName + " with restore dir: " + restoreDir);
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
index 73c3b6d..3c77201 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
@@ -322,22 +322,22 @@ public class OmidTransactionTable implements Table {
         throw new UnsupportedOperationException();
     }
 
-    @Override
+    //No @Override for compatibility reasons
     public int getWriteRpcTimeout() {
         throw new UnsupportedOperationException();
     }
 
-    @Override
+    //No @Override for compatibility reasons
     public void setWriteRpcTimeout(int writeRpcTimeout) {
         throw new UnsupportedOperationException();
     }
 
-    @Override
+    //No @Override for compatibility reasons
     public int getReadRpcTimeout() {
         throw new UnsupportedOperationException();
     }
 
-    @Override
+    //No @Override for compatibility reasons
     public void setReadRpcTimeout(int readRpcTimeout) {
         throw new UnsupportedOperationException();
     }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index ee0683b..4b96107 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -132,6 +132,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -1921,7 +1922,19 @@ public abstract class BaseTest {
 
         splitTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_HBASE_TABLE_NAME, splitPoints);
     }
-    
+
+    // We don't need need this for 1.4+, but this works with 1.3, 1.4, and 1.5
+    private static int getRegionServerIndex(MiniHBaseCluster cluster, ServerName serverName) {
+        // we have a small number of region servers, this should be fine for now.
+        List<RegionServerThread> servers = cluster.getRegionServerThreads();
+        for (int i = 0; i < servers.size(); i++) {
+            if (servers.get(i).getRegionServer().getServerName().equals(serverName)) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
     /**
      * Ensures each region of SYSTEM.CATALOG is on a different region server
      */
@@ -1931,9 +1944,14 @@ public abstract class BaseTest {
         MiniHBaseCluster cluster = util.getHBaseCluster();
         HMaster master = cluster.getMaster();
         AssignmentManager am = master.getAssignmentManager();
-   
-        HRegionServer dstServer = util.getHBaseCluster().getRegionServer(dstServerName);
-        HRegionServer srcServer = util.getHBaseCluster().getRegionServer(srcServerName);
+
+        // The 1.4+ MiniCluster way
+        // HRegionServer dstServer = util.getHBaseCluster().getRegionServer(dstServerName);
+        // HRegionServer srcServer = util.getHBaseCluster().getRegionServer(srcServerName);
+        HRegionServer dstServer =
+                cluster.getRegionServer(getRegionServerIndex(cluster, dstServerName));
+        HRegionServer srcServer =
+                cluster.getRegionServer(getRegionServerIndex(cluster, srcServerName));
         byte[] encodedRegionNameInBytes = regionInfo.getEncodedNameAsBytes();
         admin.move(encodedRegionNameInBytes, Bytes.toBytes(dstServer.getServerName().getServerName()));
         while (dstServer.getOnlineRegion(regionInfo.getRegionName()) == null
diff --git a/phoenix-hbase-compat-1.3.0/pom.xml b/phoenix-hbase-compat-1.3.0/pom.xml
new file mode 100644
index 0000000..1e4b575
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/pom.xml
@@ -0,0 +1,83 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation=
+      "http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache</groupId>
+    <artifactId>apache</artifactId>
+    <version>21</version>
+  </parent>
+  <groupId>org.apache.phoenix</groupId>
+  <artifactId>phoenix-hbase-compat-1.3.0</artifactId>
+  <version>4.16.0-SNAPSHOT</version>
+  <name>Phoenix Hbase 1.3.0 compatibility</name>
+  <description>Compatibility module for HBase 1.3.0+</description>
+
+  <build>
+    <plugins>
+      <!-- Setup eclipse -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-eclipse-plugin</artifactId>
+        <version>2.9</version>
+        <configuration>
+          <buildcommands>
+            <buildcommand>org.eclipse.jdt.core.javabuilder</buildcommand>
+          </buildcommands>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>1.3.0</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+
+  <reporting>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-project-info-reports-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </reporting>
+</project>
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
new file mode 100644
index 0000000..e28d948
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+
+// This has to be in org.apache.hadoop.hbase.regionserver due to the default
+// visibility of some methods
+public abstract class CompatStoreFileScanner extends StoreFileScanner {
+
+    public CompatStoreFileScanner(Reader reader, HFileScanner hfs, boolean useMVCC, boolean hasMVCC,
+            long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
+        super(reader, hfs, useMVCC, hasMVCC, readPt);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
new file mode 100644
index 0000000..1a1921b
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.client.Table;
+
+public abstract class CompatDelegateHTable implements Table {
+    protected final Table delegate;
+
+    public CompatDelegateHTable(Table delegate) {
+        this.delegate = delegate;
+    }
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..cfa5bd1
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+public abstract class CompatDelegateRegionCoprocessorEnvironment
+        implements RegionCoprocessorEnvironment {
+
+    protected RegionCoprocessorEnvironment delegate;
+
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionObserver.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionObserver.java
new file mode 100644
index 0000000..36db2ce
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionObserver.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+
+public abstract class CompatDelegateRegionObserver implements RegionObserver {
+
+    protected final RegionObserver delegate;
+
+    public CompatDelegateRegionObserver(RegionObserver delegate) {
+        this.delegate = delegate;
+    }
+}
\ No newline at end of file
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
new file mode 100644
index 0000000..9c00a33
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+
+// We need to copy the HBase implementation, because we need to have CompatHBaseRpcController
+// as ancestor, so we cannot simply subclass the HBase Delegating* class
+public abstract class CompatDelegatingHBaseRpcController extends CompatHBaseRpcController {
+    private PayloadCarryingRpcController delegate;
+
+    public CompatDelegatingHBaseRpcController(CompatHBaseRpcController delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public CellScanner cellScanner() {
+        return delegate.cellScanner();
+    }
+
+    @Override
+    public void setCellScanner(final CellScanner cellScanner) {
+        delegate.setCellScanner(cellScanner);
+    }
+
+    @Override
+    public void setPriority(int priority) {
+        delegate.setPriority(priority);
+    }
+
+    @Override
+    public void setPriority(final TableName tn) {
+        delegate.setPriority(tn);
+    }
+
+    @Override
+    public int getPriority() {
+        return delegate.getPriority();
+    }
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
new file mode 100644
index 0000000..c8bfbe5
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+
+public class CompatHBaseRpcController extends PayloadCarryingRpcController {
+
+    public CompatHBaseRpcController() {
+        this((CellScanner) null);
+    }
+
+    public CompatHBaseRpcController(final CellScanner cellScanner) {
+        super(cellScanner);
+    }
+
+    public CompatHBaseRpcController(final List<CellScannable> cellIterables) {
+        super(cellIterables);
+    }
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatObserverContext.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatObserverContext.java
new file mode 100644
index 0000000..2dfb6b7
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatObserverContext.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.security.User;
+
+public class CompatObserverContext<E extends CoprocessorEnvironment> extends ObserverContext {
+
+    public CompatObserverContext(User caller) {
+        // We are silently dropping the argument, instead of not pre-computing it
+        // This has some performance impact, but I believe that it is negligible
+        super();
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixMetaDataControllerEnvironment.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixMetaDataControllerEnvironment.java
new file mode 100644
index 0000000..a0b53fc
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixMetaDataControllerEnvironment.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+public abstract class CompatPhoenixMetaDataControllerEnvironment extends CoprocessorHost.Environment
+        implements RegionCoprocessorEnvironment {
+
+    protected RegionCoprocessorEnvironment env;
+
+    public CompatPhoenixMetaDataControllerEnvironment(RegionCoprocessorEnvironment env,
+            Coprocessor instance, int priority, int sequence, Configuration conf) {
+        super(instance, priority, sequence, conf);
+        this.env = env;
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixRpcScheduler.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixRpcScheduler.java
new file mode 100644
index 0000000..18fa149
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatPhoenixRpcScheduler.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.ipc.RpcScheduler;
+
+public abstract class CompatPhoenixRpcScheduler extends RpcScheduler {
+    protected RpcScheduler delegate;
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..dd30029
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+
+public abstract class CompatRegionCoprocessorEnvironment implements RegionCoprocessorEnvironment {
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
new file mode 100644
index 0000000..7de11b4
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+
+public class CompatRpcControllerFactory extends RpcControllerFactory {
+
+    public CompatRpcControllerFactory(Configuration conf) {
+        super(conf);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController() {
+        return new CompatHBaseRpcController();
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final CellScanner cellScanner) {
+        return new CompatHBaseRpcController(cellScanner);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final List<CellScannable> cellIterables) {
+        return new CompatHBaseRpcController(cellIterables);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatScanningResultIterator.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatScanningResultIterator.java
new file mode 100644
index 0000000..6afc5f0
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatScanningResultIterator.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+public abstract class CompatScanningResultIterator {
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
new file mode 100644
index 0000000..4392455
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.hbase.client.Scan;
+
+public class CompatUtil {
+
+    public static Class[] getMrMetricsClasses() {
+        return new Class[] {};
+    }
+
+    public static Scan setStartRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        if (inclusive) {
+            return scan.setStartRow(indexRowKey);
+        } else {
+            byte[] nextIndexRowKey = new byte[indexRowKey.length + 1];
+            System.arraycopy(indexRowKey, 0, nextIndexRowKey, 0, indexRowKey.length);
+            nextIndexRowKey[indexRowKey.length] = 0;
+            return scan.setStartRow(nextIndexRowKey);
+        }
+    }
+
+    public static Scan setStopRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        if (inclusive) {
+            return scan.setStopRow(indexRowKey);
+        } else {
+            throw new NotImplementedException();
+        }
+    }
+}
diff --git a/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
new file mode 100644
index 0000000..1441a95
--- /dev/null
+++ b/phoenix-hbase-compat-1.3.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.monitoring;
+
+public class CompatGlobalClientMetricsRegistry {
+
+    protected static void createRegistry() {
+        // NOOP
+    }
+
+    protected static void registerMetricToRegistry(final String name,
+            final ValueProvider valueProvider) {
+        // NOOP
+    }
+
+    protected static void registerMetricsAdapter(String metricTag) {
+        // NOOP
+    }
+
+    // Glue interface to break dependency on org.apache.hadoop.hbase.metrics.Gauge
+    protected interface ValueProvider {
+        public Long getValue();
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
new file mode 100644
index 0000000..c935cdf
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+
+// This has to be in org.apache.hadoop.hbase.regionserver due to the default
+// visibility of some methods
+public abstract class CompatStoreFileScanner extends StoreFileScanner {
+
+    public CompatStoreFileScanner(Reader reader, HFileScanner hfs, boolean useMVCC, boolean hasMVCC,
+            long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
+        super(reader, hfs, useMVCC, hasMVCC, readPt, scannerOrder, canOptimizeForNonNullColumn);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
new file mode 100644
index 0000000..7baffd8
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.client.Table;
+
+public abstract class CompatDelegateHTable implements Table {
+    protected final Table delegate;
+
+    public CompatDelegateHTable(Table delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public int getReadRpcTimeout() {
+        return delegate.getReadRpcTimeout();
+    }
+
+    @Override
+    public void setReadRpcTimeout(int readRpcTimeout) {
+        delegate.setReadRpcTimeout(readRpcTimeout);
+    }
+
+    @Override
+    public int getWriteRpcTimeout() {
+        return delegate.getWriteRpcTimeout();
+    }
+
+    @Override
+    public void setWriteRpcTimeout(int writeRpcTimeout) {
+        delegate.setWriteRpcTimeout(writeRpcTimeout);
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..c89601f
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+
+public abstract class CompatDelegateRegionCoprocessorEnvironment
+        implements RegionCoprocessorEnvironment {
+
+    protected RegionCoprocessorEnvironment delegate;
+
+    @Override
+    public MetricRegistry getMetricRegistryForRegionServer() {
+        return delegate.getMetricRegistryForRegionServer();
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
new file mode 100644
index 0000000..c6b06f7
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
+
+// We need to copy the HBase implementation, because we need to have CompatHBaseRpcController
+// as ancestor, so we cannot simply subclass the HBase Delegating* class
+public abstract class CompatDelegatingHBaseRpcController extends DelegatingHBaseRpcController
+        implements CompatHBaseRpcController {
+
+    public CompatDelegatingHBaseRpcController(CompatHBaseRpcController delegate) {
+        super(delegate);
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
new file mode 100644
index 0000000..a4827f2
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+
+public interface CompatHBaseRpcController extends HBaseRpcController {
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java
new file mode 100644
index 0000000..3fc88e3
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
+
+public class CompatHBaseRpcControllerImpl extends HBaseRpcControllerImpl
+        implements CompatHBaseRpcController {
+
+    public CompatHBaseRpcControllerImpl() {
+        super();
+    }
+
+    public CompatHBaseRpcControllerImpl(final CellScanner cellScanner) {
+        super(cellScanner);
+    }
+
+    public CompatHBaseRpcControllerImpl(final List<CellScannable> cellIterables) {
+        super(cellIterables);
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..7b71ef8
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+
+public abstract class CompatRegionCoprocessorEnvironment implements RegionCoprocessorEnvironment {
+
+    @Override
+    public MetricRegistry getMetricRegistryForRegionServer() {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
new file mode 100644
index 0000000..22bd0eb
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+
+public abstract class CompatRpcControllerFactory extends RpcControllerFactory {
+
+    public CompatRpcControllerFactory(Configuration conf) {
+        super(conf);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController() {
+        return new CompatHBaseRpcControllerImpl();
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final CellScanner cellScanner) {
+        return new CompatHBaseRpcControllerImpl(cellScanner);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final List<CellScannable> cellIterables) {
+        return new CompatHBaseRpcControllerImpl(cellIterables);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
new file mode 100644
index 0000000..046e04e
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl;
+
+public abstract class CompatUtil {
+    public static Class[] getMrMetricsClasses() {
+        return new Class[] { Gauge.class, MetricRegistriesImpl.class };
+    }
+
+    public static Scan setStartRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        return scan.withStartRow(indexRowKey, inclusive);
+    }
+
+    public static Scan setStopRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        return scan.withStopRow(indexRowKey, inclusive);
+    }
+}
diff --git a/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
new file mode 100644
index 0000000..d0eb389
--- /dev/null
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.monitoring;
+
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompatGlobalClientMetricsRegistry {
+
+    private static final Logger LOGGER =
+            LoggerFactory.getLogger(CompatGlobalClientMetricsRegistry.class);
+
+    protected static MetricRegistry metricRegistry;
+
+    protected static void createRegistry() {
+        LOGGER.info("Creating Metric Registry for Phoenix Global Metrics");
+        MetricRegistryInfo registryInfo =
+                new MetricRegistryInfo("PHOENIX", "Phoenix Client Metrics", "phoenix",
+                        "Phoenix,sub=CLIENT", true);
+        metricRegistry = MetricRegistries.global().create(registryInfo);
+    }
+
+    protected static void registerMetricToRegistry(final String name,
+            final ValueProvider valueProvider) {
+        metricRegistry.register(name, new Gauge<Long>() {
+            @Override
+            public Long getValue() {
+                return valueProvider.getValue();
+            }
+        });
+    }
+
+    protected static void registerMetricsAdapter(String metricTag) {
+        GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry,
+            metricTag);
+    }
+
+    // Glue interface to break dependency on org.apache.hadoop.hbase.metrics.Gauge
+    protected interface ValueProvider {
+        public Long getValue();
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
similarity index 81%
copy from phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
copy to phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
index 27aa9b9..66556f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
+++ b/phoenix-hbase-compat-1.4.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
@@ -37,13 +37,12 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MutableHistogram;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Contents mostly copied from GlobalMetricRegistriesAdapter class from hbase-hadoop2-compat
- * The adapter attaches HBase's MetricRegistry to Hadoop's DefaultMetricsSystem
+ * Contents mostly copied from GlobalMetricRegistriesAdapter class from hbase-hadoop2-compat The
+ * adapter attaches HBase's MetricRegistry to Hadoop's DefaultMetricsSystem 
  * Note: This DOES NOT handle dynamic attach/detach of registries
  */
 public class GlobalMetricRegistriesAdapter {
@@ -61,13 +60,14 @@ public class GlobalMetricRegistriesAdapter {
         return INSTANCE;
     }
 
-    public void registerMetricRegistry(MetricRegistry registry) {
+    public void registerMetricRegistry(MetricRegistry registry, String metricTag) {
         if (registry == null) {
             LOGGER.warn("Registry cannot be registered with Hadoop Metrics 2 since it is null.");
             return;
         }
 
-        HBaseMetrics2HadoopMetricsAdapter adapter = new HBaseMetrics2HadoopMetricsAdapter(registry);
+        HBaseMetrics2HadoopMetricsAdapter adapter =
+                new HBaseMetrics2HadoopMetricsAdapter(registry, metricTag);
         adapter.registerToDefaultMetricsSystem();
     }
 
@@ -80,45 +80,49 @@ public class GlobalMetricRegistriesAdapter {
         private final MetricRegistry registry;
         private final String metricTag;
 
-        private HBaseMetrics2HadoopMetricsAdapter(MetricRegistry registry) {
+        private HBaseMetrics2HadoopMetricsAdapter(MetricRegistry registry, String tag) {
             this.registry = registry;
-            metricTag = QueryServicesOptions.withDefaults().getClientMetricTag();
+            this.metricTag = tag;
         }
 
         private void registerToDefaultMetricsSystem() {
             MetricRegistryInfo info = registry.getMetricRegistryInfo();
-            LOGGER.info("Registering " + info.getMetricsJmxContext() +
-                    " " + info.getMetricsDescription() + " into DefaultMetricsSystem");
-            DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(), info.getMetricsDescription(), this);
+            LOGGER.info("Registering " + info.getMetricsJmxContext() + " "
+                    + info.getMetricsDescription() + " into DefaultMetricsSystem");
+            DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(),
+                info.getMetricsDescription(), this);
         }
 
         private void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsCollector collector) {
             MetricRegistryInfo hbaseMetricRegistryInfo = metricRegistry.getMetricRegistryInfo();
-            MetricsInfo hadoopMetricsInfo = Interns.info(hbaseMetricRegistryInfo.getMetricsName(), hbaseMetricRegistryInfo.getMetricsDescription());
+            MetricsInfo hadoopMetricsInfo =
+                    Interns.info(hbaseMetricRegistryInfo.getMetricsName(),
+                        hbaseMetricRegistryInfo.getMetricsDescription());
             MetricsRecordBuilder builder = collector.addRecord(hadoopMetricsInfo);
             builder.setContext(hbaseMetricRegistryInfo.getMetricsContext());
             builder.tag(hadoopMetricsInfo, metricTag);
             this.snapshotAllMetrics(metricRegistry, builder);
         }
 
-        private void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsRecordBuilder builder) {
+        private void snapshotAllMetrics(MetricRegistry metricRegistry,
+                MetricsRecordBuilder builder) {
             Map<String, Metric> metrics = metricRegistry.getMetrics();
             Iterator iterator = metrics.entrySet().iterator();
 
-            while(iterator.hasNext()) {
-                Entry<String, Metric> e = (Entry)iterator.next();
+            while (iterator.hasNext()) {
+                Entry<String, Metric> e = (Entry) iterator.next();
                 String name = StringUtils.capitalize(e.getKey());
                 Metric metric = e.getValue();
                 if (metric instanceof Gauge) {
-                    this.addGauge(name, (Gauge)metric, builder);
+                    this.addGauge(name, (Gauge) metric, builder);
                 } else if (metric instanceof Counter) {
-                    this.addCounter(name, (Counter)metric, builder);
+                    this.addCounter(name, (Counter) metric, builder);
                 } else if (metric instanceof Histogram) {
-                    this.addHistogram(name, (Histogram)metric, builder);
+                    this.addHistogram(name, (Histogram) metric, builder);
                 } else if (metric instanceof Meter) {
-                    this.addMeter(name, (Meter)metric, builder);
+                    this.addMeter(name, (Meter) metric, builder);
                 } else if (metric instanceof Timer) {
-                    this.addTimer(name, (Timer)metric, builder);
+                    this.addTimer(name, (Timer) metric, builder);
                 } else {
                     LOGGER.info("Ignoring unknown Metric class " + metric.getClass().getName());
                 }
@@ -129,13 +133,13 @@ public class GlobalMetricRegistriesAdapter {
             MetricsInfo info = Interns.info(name, "");
             Object o = gauge.getValue();
             if (o instanceof Integer) {
-                builder.addGauge(info, (Integer)o);
+                builder.addGauge(info, (Integer) o);
             } else if (o instanceof Long) {
-                builder.addGauge(info, (Long)o);
+                builder.addGauge(info, (Long) o);
             } else if (o instanceof Float) {
-                builder.addGauge(info, (Float)o);
+                builder.addGauge(info, (Float) o);
             } else if (o instanceof Double) {
-                builder.addGauge(info, (Double)o);
+                builder.addGauge(info, (Double) o);
             } else {
                 LOGGER.warn("Ignoring Gauge (" + name + ") with unhandled type: " + o.getClass());
             }
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
new file mode 100644
index 0000000..c935cdf
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/hadoop/hbase/regionserver/CompatStoreFileScanner.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+
+// This has to be in org.apache.hadoop.hbase.regionserver due to the default
+// visibility of some methods
+public abstract class CompatStoreFileScanner extends StoreFileScanner {
+
+    public CompatStoreFileScanner(Reader reader, HFileScanner hfs, boolean useMVCC, boolean hasMVCC,
+            long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
+        super(reader, hfs, useMVCC, hasMVCC, readPt, scannerOrder, canOptimizeForNonNullColumn);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
new file mode 100644
index 0000000..7baffd8
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateHTable.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.client.Table;
+
+public abstract class CompatDelegateHTable implements Table {
+    protected final Table delegate;
+
+    public CompatDelegateHTable(Table delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public int getReadRpcTimeout() {
+        return delegate.getReadRpcTimeout();
+    }
+
+    @Override
+    public void setReadRpcTimeout(int readRpcTimeout) {
+        delegate.setReadRpcTimeout(readRpcTimeout);
+    }
+
+    @Override
+    public int getWriteRpcTimeout() {
+        return delegate.getWriteRpcTimeout();
+    }
+
+    @Override
+    public void setWriteRpcTimeout(int writeRpcTimeout) {
+        delegate.setWriteRpcTimeout(writeRpcTimeout);
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..c89601f
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegateRegionCoprocessorEnvironment.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+
+public abstract class CompatDelegateRegionCoprocessorEnvironment
+        implements RegionCoprocessorEnvironment {
+
+    protected RegionCoprocessorEnvironment delegate;
+
+    @Override
+    public MetricRegistry getMetricRegistryForRegionServer() {
+        return delegate.getMetricRegistryForRegionServer();
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
new file mode 100644
index 0000000..c6b06f7
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatDelegatingHBaseRpcController.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController;
+
+// We need to copy the HBase implementation, because we need to have CompatHBaseRpcController
+// as ancestor, so we cannot simply subclass the HBase Delegating* class
+public abstract class CompatDelegatingHBaseRpcController extends DelegatingHBaseRpcController
+        implements CompatHBaseRpcController {
+
+    public CompatDelegatingHBaseRpcController(CompatHBaseRpcController delegate) {
+        super(delegate);
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
new file mode 100644
index 0000000..a4827f2
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcController.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+
+public interface CompatHBaseRpcController extends HBaseRpcController {
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java
new file mode 100644
index 0000000..3fc88e3
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatHBaseRpcControllerImpl.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl;
+
+public class CompatHBaseRpcControllerImpl extends HBaseRpcControllerImpl
+        implements CompatHBaseRpcController {
+
+    public CompatHBaseRpcControllerImpl() {
+        super();
+    }
+
+    public CompatHBaseRpcControllerImpl(final CellScanner cellScanner) {
+        super(cellScanner);
+    }
+
+    public CompatHBaseRpcControllerImpl(final List<CellScannable> cellIterables) {
+        super(cellIterables);
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
new file mode 100644
index 0000000..7b71ef8
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRegionCoprocessorEnvironment.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+
+public abstract class CompatRegionCoprocessorEnvironment implements RegionCoprocessorEnvironment {
+
+    @Override
+    public MetricRegistry getMetricRegistryForRegionServer() {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
new file mode 100644
index 0000000..22bd0eb
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatRpcControllerFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+
+public abstract class CompatRpcControllerFactory extends RpcControllerFactory {
+
+    public CompatRpcControllerFactory(Configuration conf) {
+        super(conf);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController() {
+        return new CompatHBaseRpcControllerImpl();
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final CellScanner cellScanner) {
+        return new CompatHBaseRpcControllerImpl(cellScanner);
+    }
+
+    @Override
+    public CompatHBaseRpcController newController(final List<CellScannable> cellIterables) {
+        return new CompatHBaseRpcControllerImpl(cellIterables);
+    }
+
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
new file mode 100644
index 0000000..046e04e
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/compat/hbase/CompatUtil.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compat.hbase;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl;
+
+public abstract class CompatUtil {
+    public static Class[] getMrMetricsClasses() {
+        return new Class[] { Gauge.class, MetricRegistriesImpl.class };
+    }
+
+    public static Scan setStartRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        return scan.withStartRow(indexRowKey, inclusive);
+    }
+
+    public static Scan setStopRow(Scan scan, byte[] indexRowKey, boolean inclusive) {
+        return scan.withStopRow(indexRowKey, inclusive);
+    }
+}
diff --git a/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
new file mode 100644
index 0000000..d0eb389
--- /dev/null
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/CompatGlobalClientMetricsRegistry.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.monitoring;
+
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompatGlobalClientMetricsRegistry {
+
+    private static final Logger LOGGER =
+            LoggerFactory.getLogger(CompatGlobalClientMetricsRegistry.class);
+
+    protected static MetricRegistry metricRegistry;
+
+    protected static void createRegistry() {
+        LOGGER.info("Creating Metric Registry for Phoenix Global Metrics");
+        MetricRegistryInfo registryInfo =
+                new MetricRegistryInfo("PHOENIX", "Phoenix Client Metrics", "phoenix",
+                        "Phoenix,sub=CLIENT", true);
+        metricRegistry = MetricRegistries.global().create(registryInfo);
+    }
+
+    protected static void registerMetricToRegistry(final String name,
+            final ValueProvider valueProvider) {
+        metricRegistry.register(name, new Gauge<Long>() {
+            @Override
+            public Long getValue() {
+                return valueProvider.getValue();
+            }
+        });
+    }
+
+    protected static void registerMetricsAdapter(String metricTag) {
+        GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry,
+            metricTag);
+    }
+
+    // Glue interface to break dependency on org.apache.hadoop.hbase.metrics.Gauge
+    protected interface ValueProvider {
+        public Long getValue();
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
similarity index 81%
rename from phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
rename to phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
index 27aa9b9..66556f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
+++ b/phoenix-hbase-compat-1.5.0/src/main/java/org/apache/phoenix/monitoring/GlobalMetricRegistriesAdapter.java
@@ -37,13 +37,12 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.Interns;
 import org.apache.hadoop.metrics2.lib.MutableHistogram;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Contents mostly copied from GlobalMetricRegistriesAdapter class from hbase-hadoop2-compat
- * The adapter attaches HBase's MetricRegistry to Hadoop's DefaultMetricsSystem
+ * Contents mostly copied from GlobalMetricRegistriesAdapter class from hbase-hadoop2-compat The
+ * adapter attaches HBase's MetricRegistry to Hadoop's DefaultMetricsSystem 
  * Note: This DOES NOT handle dynamic attach/detach of registries
  */
 public class GlobalMetricRegistriesAdapter {
@@ -61,13 +60,14 @@ public class GlobalMetricRegistriesAdapter {
         return INSTANCE;
     }
 
-    public void registerMetricRegistry(MetricRegistry registry) {
+    public void registerMetricRegistry(MetricRegistry registry, String metricTag) {
         if (registry == null) {
             LOGGER.warn("Registry cannot be registered with Hadoop Metrics 2 since it is null.");
             return;
         }
 
-        HBaseMetrics2HadoopMetricsAdapter adapter = new HBaseMetrics2HadoopMetricsAdapter(registry);
+        HBaseMetrics2HadoopMetricsAdapter adapter =
+                new HBaseMetrics2HadoopMetricsAdapter(registry, metricTag);
         adapter.registerToDefaultMetricsSystem();
     }
 
@@ -80,45 +80,49 @@ public class GlobalMetricRegistriesAdapter {
         private final MetricRegistry registry;
         private final String metricTag;
 
-        private HBaseMetrics2HadoopMetricsAdapter(MetricRegistry registry) {
+        private HBaseMetrics2HadoopMetricsAdapter(MetricRegistry registry, String tag) {
             this.registry = registry;
-            metricTag = QueryServicesOptions.withDefaults().getClientMetricTag();
+            this.metricTag = tag;
         }
 
         private void registerToDefaultMetricsSystem() {
             MetricRegistryInfo info = registry.getMetricRegistryInfo();
-            LOGGER.info("Registering " + info.getMetricsJmxContext() +
-                    " " + info.getMetricsDescription() + " into DefaultMetricsSystem");
-            DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(), info.getMetricsDescription(), this);
+            LOGGER.info("Registering " + info.getMetricsJmxContext() + " "
+                    + info.getMetricsDescription() + " into DefaultMetricsSystem");
+            DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(),
+                info.getMetricsDescription(), this);
         }
 
         private void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsCollector collector) {
             MetricRegistryInfo hbaseMetricRegistryInfo = metricRegistry.getMetricRegistryInfo();
-            MetricsInfo hadoopMetricsInfo = Interns.info(hbaseMetricRegistryInfo.getMetricsName(), hbaseMetricRegistryInfo.getMetricsDescription());
+            MetricsInfo hadoopMetricsInfo =
+                    Interns.info(hbaseMetricRegistryInfo.getMetricsName(),
+                        hbaseMetricRegistryInfo.getMetricsDescription());
             MetricsRecordBuilder builder = collector.addRecord(hadoopMetricsInfo);
             builder.setContext(hbaseMetricRegistryInfo.getMetricsContext());
             builder.tag(hadoopMetricsInfo, metricTag);
             this.snapshotAllMetrics(metricRegistry, builder);
         }
 
-        private void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsRecordBuilder builder) {
+        private void snapshotAllMetrics(MetricRegistry metricRegistry,
+                MetricsRecordBuilder builder) {
             Map<String, Metric> metrics = metricRegistry.getMetrics();
             Iterator iterator = metrics.entrySet().iterator();
 
-            while(iterator.hasNext()) {
-                Entry<String, Metric> e = (Entry)iterator.next();
+            while (iterator.hasNext()) {
+                Entry<String, Metric> e = (Entry) iterator.next();
                 String name = StringUtils.capitalize(e.getKey());
                 Metric metric = e.getValue();
                 if (metric instanceof Gauge) {
-                    this.addGauge(name, (Gauge)metric, builder);
+                    this.addGauge(name, (Gauge) metric, builder);
                 } else if (metric instanceof Counter) {
-                    this.addCounter(name, (Counter)metric, builder);
+                    this.addCounter(name, (Counter) metric, builder);
                 } else if (metric instanceof Histogram) {
-                    this.addHistogram(name, (Histogram)metric, builder);
+                    this.addHistogram(name, (Histogram) metric, builder);
                 } else if (metric instanceof Meter) {
-                    this.addMeter(name, (Meter)metric, builder);
+                    this.addMeter(name, (Meter) metric, builder);
                 } else if (metric instanceof Timer) {
-                    this.addTimer(name, (Timer)metric, builder);
+                    this.addTimer(name, (Timer) metric, builder);
                 } else {
                     LOGGER.info("Ignoring unknown Metric class " + metric.getClass().getName());
                 }
@@ -129,13 +133,13 @@ public class GlobalMetricRegistriesAdapter {
             MetricsInfo info = Interns.info(name, "");
             Object o = gauge.getValue();
             if (o instanceof Integer) {
-                builder.addGauge(info, (Integer)o);
+                builder.addGauge(info, (Integer) o);
             } else if (o instanceof Long) {
-                builder.addGauge(info, (Long)o);
+                builder.addGauge(info, (Long) o);
             } else if (o instanceof Float) {
-                builder.addGauge(info, (Float)o);
+                builder.addGauge(info, (Float) o);
             } else if (o instanceof Double) {
-                builder.addGauge(info, (Double)o);
+                builder.addGauge(info, (Double) o);
             } else {
                 LOGGER.warn("Ignoring Gauge (" + name + ") with unhandled type: " + o.getClass());
             }
diff --git a/pom.xml b/pom.xml
index 1112ca9..45d3f80 100644
--- a/pom.xml
+++ b/pom.xml
@@ -42,6 +42,7 @@
   <modules>
     <module>phoenix-hbase-compat-1.5.0</module>
     <module>phoenix-hbase-compat-1.4.0</module>
+    <module>phoenix-hbase-compat-1.3.0</module>
     <module>phoenix-core</module>
     <module>phoenix-pherf</module>
     <module>phoenix-client</module>
@@ -591,6 +592,11 @@
         <artifactId>phoenix-hbase-compat-1.4.0</artifactId>
         <version>${project.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix-hbase-compat-1.3.0</artifactId>
+        <version>${project.version}</version>
+      </dependency>
       <!-- TODO Remove PQS and connector dependency rules -->
       <dependency>
         <groupId>org.apache.phoenix</groupId>
@@ -1205,6 +1211,26 @@
         <hbase.version>1.4.10</hbase.version>
       </properties>
     </profile>
+    <profile>
+      <id>phoenix-hbase-compat-1.3.0</id>
+      <activation>
+        <property>
+          <name>hbase.profile</name>
+          <value>1.3</value>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.phoenix</groupId>
+          <artifactId>phoenix-hbase-compat-1.3.0</artifactId>
+        </dependency>
+      </dependencies>
+      <properties>
+        <hbase.profile>1.3</hbase.profile>
+        <hbase.compat.version>1.3.0</hbase.compat.version>
+        <hbase.version>1.3.5</hbase.version>
+      </properties>
+    </profile>
   </profiles>
 
   <reporting>