You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/03/24 06:31:05 UTC

phoenix git commit: PHOENIX-1676 Set priority of Index Updates correctly

Repository: phoenix
Updated Branches:
  refs/heads/4.3 8005c038c -> 236f40785


PHOENIX-1676 Set priority of Index Updates correctly


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

Branch: refs/heads/4.3
Commit: 236f407859a15bb4225c14cc638460110397022a
Parents: 8005c03
Author: Thomas <td...@salesforce.com>
Authored: Mon Mar 23 22:17:16 2015 -0700
Committer: Thomas <td...@salesforce.com>
Committed: Mon Mar 23 22:30:41 2015 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/index/IndexQosIT.java       | 240 +++++++++++++++++++
 .../hbase/ipc/PhoenixIndexRpcScheduler.java     |   3 +
 .../phoenix/hbase/index/IndexQosCompat.java     |  98 --------
 .../index/IndexQosRpcControllerFactory.java     |  12 +-
 .../index/table/CoprocessorHTableFactory.java   |  20 --
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   7 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   7 +
 7 files changed, 260 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexQosIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexQosIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexQosIT.java
new file mode 100644
index 0000000..7338b40
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexQosIT.java
@@ -0,0 +1,240 @@
+/*
+ * 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.end2end.index;
+
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.ipc.BalancedQueueRpcExecutor;
+import org.apache.hadoop.hbase.ipc.CallRunner;
+import org.apache.hadoop.hbase.ipc.PhoenixIndexRpcScheduler;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.ipc.RpcExecutor;
+import org.apache.hadoop.hbase.ipc.RpcScheduler;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
+import org.apache.phoenix.hbase.index.IndexQosRpcControllerFactory;
+import org.apache.phoenix.hbase.index.ipc.PhoenixIndexRpcSchedulerFactory;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(NeedsOwnMiniClusterTest.class)
+public class IndexQosIT extends BaseTest {
+
+    private static final String SCHEMA_NAME = "S";
+    private static final String INDEX_TABLE_NAME = "I";
+    private static final String DATA_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "T");
+    private static final String INDEX_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "I");
+    private static final int NUM_SLAVES = 2;
+
+    private static String url;
+    private static PhoenixTestDriver driver;
+    private HBaseTestingUtility util;
+    private HBaseAdmin admin;
+    private Configuration conf;
+    private static RpcExecutor spyRpcExecutor = Mockito.spy(new BalancedQueueRpcExecutor("test-queue", 30, 1, 300));
+
+    /**
+     * Factory that uses a spyed RpcExecutor
+     */
+    public static class TestPhoenixIndexRpcSchedulerFactory extends PhoenixIndexRpcSchedulerFactory {
+        @Override
+        public RpcScheduler create(Configuration conf, RegionServerServices services) {
+            PhoenixIndexRpcScheduler phoenixIndexRpcScheduler = (PhoenixIndexRpcScheduler)super.create(conf, services);
+            phoenixIndexRpcScheduler.setExecutorForTesting(spyRpcExecutor);
+            return phoenixIndexRpcScheduler;
+        }
+    }
+
+    @Before
+    public void doSetup() throws Exception {
+        conf = HBaseConfiguration.create();
+        setUpConfigForMiniCluster(conf);
+        conf.set(HRegionServer.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
+                TestPhoenixIndexRpcSchedulerFactory.class.getName());
+        conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY, IndexQosRpcControllerFactory.class.getName());
+        util = new HBaseTestingUtility(conf);
+        // start cluster with 2 region servers
+        util.startMiniCluster(NUM_SLAVES);
+        admin = util.getHBaseAdmin();
+        String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
+        url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
+                + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        driver = initAndRegisterDriver(url, ReadOnlyProps.EMPTY_PROPS);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        try {
+            destroyDriver(driver);
+            if (admin!=null) {
+            	admin.close();
+            }
+        } finally {
+            util.shutdownMiniCluster();
+        }
+    }
+    
+    @Test
+    public void testIndexWriteQos() throws Exception { 
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = driver.connect(url, props);
+
+        // create the table 
+        conn.createStatement().execute(
+                "CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+
+        // create the index 
+        conn.createStatement().execute(
+                "CREATE INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v1) INCLUDE (v2)");
+
+        byte[] dataTableName = Bytes.toBytes(DATA_TABLE_FULL_NAME);
+        byte[] indexTableName = Bytes.toBytes(INDEX_TABLE_FULL_NAME);
+        MiniHBaseCluster cluster = util.getHBaseCluster();
+        HMaster master = cluster.getMaster();
+        AssignmentManager am = master.getAssignmentManager();
+
+        // verify there is only a single region for data table
+        List<HRegionInfo> tableRegions = admin.getTableRegions(dataTableName);
+        assertEquals("Expected single region for " + dataTableName, tableRegions.size(), 1);
+        HRegionInfo dataHri = tableRegions.get(0);
+
+        // verify there is only a single region for index table
+        tableRegions = admin.getTableRegions(indexTableName);
+        HRegionInfo indexHri = tableRegions.get(0);
+        assertEquals("Expected single region for " + indexTableName, tableRegions.size(), 1);
+
+        ServerName dataServerName = am.getRegionStates().getRegionServerOfRegion(dataHri);
+        ServerName indexServerName = am.getRegionStates().getRegionServerOfRegion(indexHri);
+
+        // if data table and index table are on same region server, move the index table to the other region server
+        if (dataServerName.equals(indexServerName)) {
+            HRegionServer server1 = util.getHBaseCluster().getRegionServer(0);
+            HRegionServer server2 = util.getHBaseCluster().getRegionServer(1);
+            HRegionServer dstServer = null;
+            HRegionServer srcServer = null;
+            if (server1.getServerName().equals(indexServerName)) {
+                dstServer = server2;
+                srcServer = server1;
+            } else {
+                dstServer = server1;
+                srcServer = server2;
+            }
+            byte[] encodedRegionNameInBytes = indexHri.getEncodedNameAsBytes();
+            admin.move(encodedRegionNameInBytes, Bytes.toBytes(dstServer.getServerName().getServerName()));
+            while (dstServer.getOnlineRegion(indexHri.getRegionName()) == null
+                    || dstServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameInBytes)
+                    || srcServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameInBytes)
+                    || master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
+                // wait for the move to be finished
+                Thread.sleep(1);
+            }
+        }
+
+        dataHri = admin.getTableRegions(dataTableName).get(0);
+        dataServerName = am.getRegionStates().getRegionServerOfRegion(dataHri);
+        indexHri = admin.getTableRegions(indexTableName).get(0);
+        indexServerName = am.getRegionStates().getRegionServerOfRegion(indexHri);
+
+        // verify index and data tables are on different servers
+        assertNotEquals("Index and Data table should be on different region servers dataServer " + dataServerName
+                + " indexServer " + indexServerName, dataServerName, indexServerName);
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?,?)");
+        stmt.setString(1, "k1");
+        stmt.setString(2, "v1");
+        stmt.setString(3, "v2");
+        stmt.execute();
+        conn.commit();
+
+        // run select query that should use the index
+        String selectSql = "SELECT k, v2 from " + DATA_TABLE_FULL_NAME + " WHERE v1=?";
+        stmt = conn.prepareStatement(selectSql);
+        stmt.setString(1, "v1");
+
+        // verify that the query does a range scan on the index table
+        ResultSet rs = stmt.executeQuery("EXPLAIN " + selectSql);
+        assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER S.I ['v1']", QueryUtil.getExplainPlan(rs));
+
+        // verify that the correct results are returned
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("k1", rs.getString(1));
+        assertEquals("v2", rs.getString(2));
+        assertFalse(rs.next());
+        
+        // drop index table 
+        conn.createStatement().execute(
+                "DROP INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME );
+        // create a data table with the same name as the index table 
+        conn.createStatement().execute(
+                "CREATE TABLE " + INDEX_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+        
+        // upsert one row to the table (which has the same table name as the previous index table)
+        stmt = conn.prepareStatement("UPSERT INTO " + INDEX_TABLE_FULL_NAME + " VALUES(?,?,?)");
+        stmt.setString(1, "k1");
+        stmt.setString(2, "v1");
+        stmt.setString(3, "v2");
+        stmt.execute();
+        conn.commit();
+        
+        // run select query on the new table
+        selectSql = "SELECT k, v2 from " + INDEX_TABLE_FULL_NAME + " WHERE v1=?";
+        stmt = conn.prepareStatement(selectSql);
+        stmt.setString(1, "v1");
+
+        // verify that the correct results are returned
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        assertEquals("k1", rs.getString(1));
+        assertEquals("v2", rs.getString(2));
+        assertFalse(rs.next());
+        
+        // verify that that index queue is used only once (for the first upsert)
+        Mockito.verify(spyRpcExecutor).dispatch(Mockito.any(CallRunner.class));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcScheduler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcScheduler.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcScheduler.java
index 98002a7..4709304 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcScheduler.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcScheduler.java
@@ -43,6 +43,7 @@ public class PhoenixIndexRpcScheduler extends RpcScheduler {
     private int minPriority;
     private int maxPriority;
     private RpcExecutor callExecutor;
+    private int port;
 
     public PhoenixIndexRpcScheduler(int indexHandlerCount, Configuration conf,
             RpcScheduler delegate, int minPriority, int maxPriority) {
@@ -67,11 +68,13 @@ public class PhoenixIndexRpcScheduler extends RpcScheduler {
     @Override
     public void init(Context context) {
         delegate.init(context);
+        this.port = context.getListenerAddress().getPort();
     }
 
     @Override
     public void start() {
         delegate.start();
+        callExecutor.start(port);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosCompat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosCompat.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosCompat.java
deleted file mode 100644
index 5681d71..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosCompat.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * 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.hbase.index;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.phoenix.hbase.index.ipc.PhoenixIndexRpcSchedulerFactory;
-
-/**
- * Helper class to avoid loading HBase 0.98.3+ classes in older HBase installations
- */
-public class IndexQosCompat {
-
-    private static final Log LOG = LogFactory.getLog(IndexQosCompat.class);
-
-    /**
-     * Full class name of the RpcControllerFactory. This is copied here so we don't need the static reference, so we can work with older versions of HBase 0.98, which don't have this class
-     */
-    private static final String HBASE_RPC_CONTROLLER_CLASS_NAME =
-            "org.apache.hadoop.hbase.ipc.RpcControllerFactory";
-    private static volatile boolean checked = false;
-    private static boolean rpcControllerExists = false;
-
-    private IndexQosCompat() {
-        // private ctor for util class
-    }
-
-    /**
-     * @param tableName name of the index table
-     * @return configuration key for if a table should have Index QOS writes (its a target index
-     *         table)
-     */
-    public static String getTableIndexQosConfKey(String tableName) {
-        return "phoenix.index.table.qos._" + tableName;
-    }
-
-    /**
-     * Set the index rpc controller, if the rpc controller exists. No-op if there the RpcController
-     * is not on the classpath.
-     * @param conf to update
-     */
-    public static void setPhoenixIndexRpcController(Configuration conf) {
-        if (rpcControllerExists()) {
-            // then we can load the class just fine
-            conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY,
-                PhoenixIndexRpcSchedulerFactory.class.getName());
-        }
-    }
-
-    private static boolean rpcControllerExists() {
-        if (checked) {
-            synchronized (IndexQosCompat.class) {
-                if (!checked) {
-                    // try loading the class
-                    try {
-                        Class.forName(HBASE_RPC_CONTROLLER_CLASS_NAME);
-                        rpcControllerExists = true;
-                    } catch (ClassNotFoundException e) {
-                        LOG.warn("RpcControllerFactory doesn't exist, not setting custom index handler properties.");
-                        rpcControllerExists = false;
-                    }
-
-                    checked = true;
-                }
-            }
-        }
-        return rpcControllerExists;
-    }
-
-    /**
-     * Ensure that the given table is enabled for index QOS handling
-     * @param conf configuration to read/update
-     * @param tableName name of the table to configure for index handlers
-     */
-    public static void enableIndexQosForTable(Configuration conf, String tableName) {
-        String confKey = IndexQosCompat.getTableIndexQosConfKey(tableName);
-        if (conf.get(confKey) == null) {
-            conf.setBoolean(confKey, true);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosRpcControllerFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosRpcControllerFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosRpcControllerFactory.java
index aa8b8d1..a192feb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosRpcControllerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/IndexQosRpcControllerFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ipc.DelegatingPayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.phoenix.hbase.index.ipc.PhoenixIndexRpcSchedulerFactory;
+import org.apache.phoenix.util.SchemaUtil;
 
 /**
  * {@link RpcControllerFactory} that overrides the standard {@link PayloadCarryingRpcController} to
@@ -60,27 +61,22 @@ public class IndexQosRpcControllerFactory extends RpcControllerFactory {
 
     private class IndexQosRpcController extends DelegatingPayloadCarryingRpcController {
 
-        private Configuration conf;
         private int priority;
 
         public IndexQosRpcController(PayloadCarryingRpcController delegate, Configuration conf) {
             super(delegate);
-            this.conf = conf;
             this.priority = PhoenixIndexRpcSchedulerFactory.getMinPriority(conf);
         }
-
         @Override
         public void setPriority(final TableName tn) {
             // if its an index table, then we override to the index priority
-            if (isIndexTable(tn)) {
+            if (!tn.isSystemTable() &&  !SchemaUtil.isSystemDataTable(tn.getNameAsString())) {
                 setPriority(this.priority);
-            } else {
+            } 
+            else {
                 super.setPriority(tn);
             }
         }
 
-        private boolean isIndexTable(TableName tn) {
-            return conf.get(IndexQosCompat.getTableIndexQosConfKey(tn.getNameAsString())) == null;
-        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
index 72a28be..ded618d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/table/CoprocessorHTableFactory.java
@@ -19,21 +19,13 @@ package org.apache.phoenix.hbase.index.table;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.hbase.index.IndexQosCompat;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
 
 public class CoprocessorHTableFactory implements HTableFactory {
 
-    private static final Log LOG = LogFactory.getLog(CoprocessorHTableFactory.class);
     private CoprocessorEnvironment e;
 
     public CoprocessorHTableFactory(CoprocessorEnvironment e) {
@@ -42,18 +34,6 @@ public class CoprocessorHTableFactory implements HTableFactory {
 
     @Override
     public HTableInterface getTable(ImmutableBytesPtr tablename) throws IOException {
-        Configuration conf = e.getConfiguration();
-
-        // make sure we use the index priority writer for our rpcs
-        IndexQosCompat.setPhoenixIndexRpcController(conf);
-
-        // make sure we include the index table in the tables we need to track
-        String tableName = Bytes.toString(tablename.copyBytesIfNecessary());
-        IndexQosCompat.enableIndexQosForTable(conf, tableName);
-
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Creating new HTable: " + tableName);
-        }
         return this.e.getTable(TableName.valueOf(tablename.copyBytesIfNecessary()));
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 154fef7..15bcfd0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -67,6 +67,7 @@ import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 
@@ -277,7 +278,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final int MUTABLE_SI_VERSION_THRESHOLD = VersionUtil.encodeVersion("0", "94", "10");
     /** Version below which we fall back on the generic KeyValueBuilder */
     public static final int CLIENT_KEY_VALUE_BUILDER_THRESHOLD = VersionUtil.encodeVersion("0", "94", "14");
-
+    
+    // list of system tables
+    public static final List<String> SYSTEM_TABLE_NAMES = new ImmutableList.Builder<String>().add(SYSTEM_CATALOG_NAME)
+            .add(SYSTEM_STATS_NAME).add(SEQUENCE_FULLNAME).build();
+    
     PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException {
         this.emptyResultSet = new PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, new PhoenixStatement(connection));
         this.connection = connection;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/236f4078/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index c9574e3..3bd5057 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -403,6 +403,13 @@ public class SchemaUtil {
         if (QueryConstants.SYSTEM_SCHEMA_NAME.equals(schemaName)) return true;
         return false;
     }
+    
+    /**
+     * Returns true if the given table is a system table (does not include future system indexes)
+     */
+    public static boolean isSystemDataTable(String fullTableName) {
+    	return PhoenixDatabaseMetaData.SYSTEM_TABLE_NAMES.contains(fullTableName);
+    }
 
     // Given the splits and the rowKeySchema, find out the keys that 
     public static byte[][] processSplits(byte[][] splits, LinkedHashSet<PColumn> pkColumns, Integer saltBucketNum, boolean defaultRowKeyOrder) throws SQLException {