You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/10 07:55:01 UTC

phoenix git commit: PHOENIX-3461 Addendum: Tweak utility method to use configuration object. Add tests.

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 b3d439e04 -> f53b57e13


PHOENIX-3461 Addendum: Tweak utility method to use configuration object. Add tests.


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

Branch: refs/heads/4.x-HBase-1.1
Commit: f53b57e13e9774f8a5664f4e4bd2ff198df4216e
Parents: b3d439e
Author: Samarth <sa...@salesforce.com>
Authored: Wed Nov 9 23:51:04 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Wed Nov 9 23:51:04 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/BaseOwnClusterIT.java       |  16 ++-
 .../end2end/BaseUniqueNamesOwnClusterIT.java    |  38 +++++++
 .../end2end/CountDistinctCompressionIT.java     |   7 +-
 .../apache/phoenix/end2end/QueryTimeoutIT.java  |  18 ++--
 .../phoenix/end2end/QueryWithLimitIT.java       |  30 ++++--
 .../apache/phoenix/end2end/RenewLeaseIT.java    |   6 +-
 .../phoenix/end2end/StatsCollectorIT.java       | 103 ++++++++++++-------
 ...SysTableNamespaceMappedStatsCollectorIT.java |  49 +++++++++
 .../phoenix/end2end/index/BaseLocalIndexIT.java |   4 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |   4 +-
 .../end2end/index/txn/TxWriteFailureIT.java     |  42 +++++---
 .../RoundRobinResultIteratorWithStatsIT.java    |  21 ++--
 .../phoenix/monitoring/PhoenixMetricsIT.java    |  60 ++++++-----
 .../apache/phoenix/rpc/PhoenixClientRpcIT.java  |  33 ++++--
 .../apache/phoenix/rpc/PhoenixServerRpcIT.java  |  42 +++++---
 .../org/apache/phoenix/util/SchemaUtil.java     |  10 +-
 16 files changed, 329 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterIT.java
index 44bd3a1..3b3d611 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterIT.java
@@ -18,18 +18,14 @@
 package org.apache.phoenix.end2end;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.phoenix.query.BaseTest;
 import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.experimental.categories.Category;
-
-@Category(NeedsOwnMiniClusterTest.class)
-public class BaseOwnClusterIT extends BaseTest {
-    @AfterClass
-    public static void doTeardown() throws Exception {
-        tearDownMiniCluster();
-    }
 
+/**
+ * Base class for tests that need their own mini cluster. Classes that extend this base class don't necessarily have to
+ * use unique names for tables and sequences. Any new integration tests that need their own mini cluster should be
+ * extending {@link BaseUniqueNamesOwnClusterIT} class directly.
+ */
+public class BaseOwnClusterIT extends BaseUniqueNamesOwnClusterIT {
     @After
     public void cleanUpAfterTest() throws Exception {
         deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseUniqueNamesOwnClusterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseUniqueNamesOwnClusterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseUniqueNamesOwnClusterIT.java
new file mode 100644
index 0000000..8c48997
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseUniqueNamesOwnClusterIT.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+import org.apache.phoenix.query.BaseTest;
+import org.junit.AfterClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Base class for tests that need to be executed on their mini cluster. You must create unique names using
+ * {@link #generateUniqueName()} for each table and sequence used to prevent collisions.
+ * <p>
+ * TODO: Convert all tests extending {@link BaseOwnClusterIT} to use unique names for tables and sequences. Once that is
+ * done either rename this class or get rid of the {@link BaseOwnClusterIT} base class.
+ * </p>
+ */
+@Category(NeedsOwnMiniClusterTest.class)
+public class BaseUniqueNamesOwnClusterIT extends BaseTest {
+    @AfterClass
+    public static void doTeardown() throws Exception {
+        tearDownMiniCluster();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
index aa7a89c..1539ad7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
@@ -18,7 +18,6 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -40,7 +39,7 @@ import org.junit.Test;
 import com.google.common.collect.Maps;
 
 
-public class CountDistinctCompressionIT extends BaseOwnClusterIT {
+public class CountDistinctCompressionIT extends BaseUniqueNamesOwnClusterIT {
     @BeforeClass
     public static void doSetup() throws Exception {
         Map<String, String> props = Maps.newHashMapWithExpectedSize(3);
@@ -52,9 +51,9 @@ public class CountDistinctCompressionIT extends BaseOwnClusterIT {
     @Test
     public void testDistinctCountOnColumn() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), (Date)null, null, getUrl());
+        String tableName = initATableValues(null, tenantId, getDefaultSplits(tenantId), (Date)null, null, getUrl());
 
-        String query = "SELECT count(DISTINCT A_STRING) FROM aTable";
+        String query = "SELECT count(DISTINCT A_STRING) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryTimeoutIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryTimeoutIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryTimeoutIT.java
index 2d58615..03af042 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryTimeoutIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryTimeoutIT.java
@@ -42,14 +42,20 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
 
-public class QueryTimeoutIT extends BaseOwnClusterIT {
-    private static final String TEST_TABLE_NAME = "T";
+public class QueryTimeoutIT extends BaseUniqueNamesOwnClusterIT {
+    private String tableName;
+    
+    @Before
+    public void generateTableName() throws SQLException {
+        tableName = generateUniqueName();
+    }
     
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -105,8 +111,8 @@ public class QueryTimeoutIT extends BaseOwnClusterIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + TEST_TABLE_NAME + "(k BIGINT PRIMARY KEY, v VARCHAR)");
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TEST_TABLE_NAME + " VALUES(?, 'AAAAAAAAAAAAAAAAAAAA')");
+                "CREATE TABLE " + tableName + "(k BIGINT PRIMARY KEY, v VARCHAR)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?, 'AAAAAAAAAAAAAAAAAAAA')");
         for (int i = 1; i <= nRows; i++) {
             stmt.setLong(1, i);
             stmt.executeUpdate();
@@ -115,13 +121,13 @@ public class QueryTimeoutIT extends BaseOwnClusterIT {
             }
         }
         conn.commit();
-        conn.createStatement().execute("UPDATE STATISTICS " + TEST_TABLE_NAME);
+        conn.createStatement().execute("UPDATE STATISTICS " + tableName);
         
         PhoenixStatement pstmt = conn.createStatement().unwrap(PhoenixStatement.class);
         pstmt.setQueryTimeout(1);
         long startTime = System.currentTimeMillis();
         try {
-            ResultSet rs = pstmt.executeQuery("SELECT count(*) FROM " + TEST_TABLE_NAME);
+            ResultSet rs = pstmt.executeQuery("SELECT count(*) FROM " + tableName);
             // Force lots of chunks so query is cancelled
             assertTrue(pstmt.getQueryPlan().getSplits().size() > 1000);
             rs.next();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithLimitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithLimitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithLimitIT.java
index 17a5577..1160f4e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithLimitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithLimitIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.KEYONLY_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -37,13 +36,22 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
 
-public class QueryWithLimitIT extends BaseOwnClusterIT {
+public class QueryWithLimitIT extends BaseUniqueNamesOwnClusterIT {
+    
+    private String tableName;
+    
+    @Before
+    public void generateTableName() {
+        tableName = generateUniqueName();
+    }
+    
 
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -62,19 +70,19 @@ public class QueryWithLimitIT extends BaseOwnClusterIT {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
-            conn.createStatement().execute("create table " + KEYONLY_NAME + "\n" +
+            conn.createStatement().execute("create table " + tableName + "\n" +
                 "   (i1 integer not null, i2 integer not null\n" +
                 "    CONSTRAINT pk PRIMARY KEY (i1,i2))");
             initTableValues(conn, 100);
             
-            String query = "SELECT i1 FROM KEYONLY LIMIT 1";
+            String query = "SELECT i1 FROM " + tableName +" LIMIT 1";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(0, rs.getInt(1));
             assertFalse(rs.next());
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-            assertEquals("CLIENT SERIAL 1-WAY FULL SCAN OVER KEYONLY\n" + 
+            assertEquals("CLIENT SERIAL 1-WAY FULL SCAN OVER " + tableName + "\n" + 
                     "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                     "    SERVER 1 ROW LIMIT\n" + 
                     "CLIENT 1 ROW LIMIT", QueryUtil.getExplainPlan(rs));
@@ -88,13 +96,13 @@ public class QueryWithLimitIT extends BaseOwnClusterIT {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
 
-        conn.createStatement().execute("create table " + KEYONLY_NAME + "\n" +
+        conn.createStatement().execute("create table " + tableName + "\n" +
                 "   (i1 integer not null, i2 integer not null\n" +
                 "    CONSTRAINT pk PRIMARY KEY (i1,i2))");
         initTableValues(conn, 100);
-        conn.createStatement().execute("UPDATE STATISTICS " + KEYONLY_NAME);
+        conn.createStatement().execute("UPDATE STATISTICS " + tableName);
         
-        String query = "SELECT i1 FROM KEYONLY";
+        String query = "SELECT i1 FROM " + tableName;
         try {
             ResultSet rs = conn.createStatement().executeQuery(query);
             rs.next();
@@ -105,10 +113,10 @@ public class QueryWithLimitIT extends BaseOwnClusterIT {
         conn.close();
     }
     
-    protected static void initTableValues(Connection conn, int nRows) throws Exception {
+    protected void initTableValues(Connection conn, int nRows) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
-            "upsert into " +
-            "KEYONLY VALUES (?, ?)");
+            "upsert into " + tableName + 
+            " VALUES (?, ?)");
         for (int i = 0; i < nRows; i++) {
             stmt.setInt(1, i);
             stmt.setInt(2, i+1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java
index aba4ddb..78c27f3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java
@@ -42,10 +42,10 @@ import org.junit.Test;
 import com.google.common.collect.Maps;
 
 
-public class RenewLeaseIT extends BaseOwnClusterIT {
+public class RenewLeaseIT extends BaseUniqueNamesOwnClusterIT {
     private static final long RPC_TIMEOUT = 2000;
     private static volatile boolean SLEEP_NOW = false;
-    private static final String TABLE_NAME = "FOO_BAR";
+    private final String TABLE_NAME = generateUniqueName();
     
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -72,7 +72,7 @@ public class RenewLeaseIT extends BaseOwnClusterIT {
         }
     }
     
-    public static class SleepingRegionObserver extends SimpleRegionObserver {
+    public class SleepingRegionObserver extends SimpleRegionObserver {
         @Override
         public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
                 final InternalScanner s, final List<Result> results,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 6193cad..85fc9d7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -34,6 +34,7 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
@@ -48,63 +49,91 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
 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.apache.phoenix.util.TestUtil;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import com.google.common.collect.Maps;
+
 @RunWith(Parameterized.class)
-public class StatsCollectorIT extends ParallelStatsEnabledIT {
+public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
     private final String tableDDLOptions;
     private String tableName;
     private String schemaName;
     private String fullTableName;
-        
-    public StatsCollectorIT( boolean transactional) {
+    private String physicalTableName;
+    private final boolean userTableNamespaceMapped;
+    
+    public StatsCollectorIT(boolean transactional, boolean userTableNamespaceMapped) {
         this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
+        this.userTableNamespaceMapped = userTableNamespaceMapped;
     }
     
-    private static Connection getConnection() throws SQLException {
-        return getConnection(Integer.MAX_VALUE);
+    @Parameters(name="transactional = {0}, isUserTableNamespaceMapped = {1}")
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] {{false,true}, {false, false}, {true, false}, {true, true}});
     }
-
-    private static Connection getConnection(Integer statsUpdateFreq) throws SQLException {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.setProperty(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(statsUpdateFreq));
-        return DriverManager.getConnection(getUrl(), props);
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        // enable name space mapping at global level on both client and server side
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        clientProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
     
     @Before
-    public void generateTableNames() {
-        schemaName = TestUtil.DEFAULT_SCHEMA_NAME;
+    public void generateTableNames() throws SQLException {
+        schemaName = generateUniqueName();
+        if (userTableNamespaceMapped) {
+            try (Connection conn = getConnection()) {
+                conn.createStatement().execute("CREATE SCHEMA " + schemaName);
+            }
+        }
         tableName = "T_" + generateUniqueName();
         fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        physicalTableName = SchemaUtil.getPhysicalHBaseTableName(fullTableName, userTableNamespaceMapped, PTableType.TABLE).getString();
     }
 
-    @Parameters(name="transactional = {0}")
-    public static Collection<Boolean> data() {
-        return Arrays.asList(false,true);
+    private Connection getConnection() throws SQLException {
+        return getConnection(Integer.MAX_VALUE);
     }
 
+    private Connection getConnection(Integer statsUpdateFreq) throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(statsUpdateFreq));
+        // enable/disable namespace mapping at connection level
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(userTableNamespaceMapped));
+        return DriverManager.getConnection(getUrl(), props);
+    }
+    
     @Test
     public void testUpdateEmptyStats() throws Exception {
         Connection conn = getConnection();
         conn.setAutoCommit(true);
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName +" ( k CHAR(1) PRIMARY KEY )"  + tableDDLOptions);
-        conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
         ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
         String explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 1-CHUNK 0 ROWS 0 BYTES PARALLEL 1-WAY FULL SCAN OVER " + fullTableName + "\n" + 
+                "CLIENT 1-CHUNK 0 ROWS 0 BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName + "\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY",
                 explainPlan);
         conn.close();
@@ -123,20 +152,20 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM " + fullTableName + " WHERE v2='foo'");
         explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 3-CHUNK 0 ROWS 0 BYTES PARALLEL 3-WAY FULL SCAN OVER " + fullTableName + "\n" +
+                "CLIENT 3-CHUNK 0 ROWS 0 BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
                 "    SERVER FILTER BY B.V2 = 'foo'\n" + 
                 "CLIENT MERGE SORT",
                 explainPlan);
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
         explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 4-CHUNK 1 ROWS 34 BYTES PARALLEL 3-WAY FULL SCAN OVER " + fullTableName + "\n" +
+                "CLIENT 4-CHUNK 1 ROWS 34 BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
                 "CLIENT MERGE SORT",
                 explainPlan);
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName + " WHERE k = 'a'");
         explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 1-CHUNK 1 ROWS 202 BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + fullTableName + "\n" +
+                "CLIENT 1-CHUNK 1 ROWS 202 BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + physicalTableName + "\n" +
                 "CLIENT MERGE SORT",
                 explainPlan);
         
@@ -157,11 +186,11 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
                 		+ tableDDLOptions );
         String[] s;
         Array array;
-        conn = upsertValues(props, tableName);
+        conn = upsertValues(props, fullTableName);
         // CAll the update statistics query here. If already major compaction has run this will not get executed.
-        stmt = conn.prepareStatement("UPDATE STATISTICS " + tableName);
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
         stmt.execute();
-        stmt = upsertStmt(conn, tableName);
+        stmt = upsertStmt(conn, fullTableName);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -170,9 +199,9 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS " + tableName);
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName);
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
         assertTrue(rs.next());
         conn.close();
     }
@@ -370,8 +399,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
             stmt.executeUpdate();
         }
         conn.commit();
-        
-        compactTable(conn, tableName);
+        compactTable(conn, physicalTableName);
         if (statsUpdateFreq == null) {
             invalidateStats(conn, tableName);
         } else {
@@ -391,7 +419,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         conn.commit();
         assertEquals(5, nDeletedRows);
         
-        compactTable(conn, tableName);
+        compactTable(conn, physicalTableName);
         if (statsUpdateFreq == null) {
             invalidateStats(conn, tableName);
         }
@@ -407,7 +435,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         }
         assertEquals(nRows/2+1, keyRanges.size());
         ResultSet rs = conn.createStatement().executeQuery("SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM "
-                + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE PHYSICAL_NAME='" + tableName + "'");
+                + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE PHYSICAL_NAME='" + physicalTableName + "'");
         rs.next();
         assertEquals(nRows - nDeletedRows, rs.getLong(1));
     }
@@ -445,11 +473,11 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         List<KeyRange> keyRanges = getAllSplits(conn, fullTableName);
         assertEquals(26, keyRanges.size());
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 26-CHUNK 25 ROWS 12420 BYTES PARALLEL 1-WAY FULL SCAN OVER " + fullTableName,
+        assertEquals("CLIENT 26-CHUNK 25 ROWS 12420 BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
                 QueryUtil.getExplainPlan(rs));
 
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-        List<HRegionLocation> regions = services.getAllTableRegions(Bytes.toBytes(fullTableName));
+        List<HRegionLocation> regions = services.getAllTableRegions(Bytes.toBytes(physicalTableName));
         assertEquals(1, regions.size());
 
         TestUtil.analyzeTable(conn, fullTableName);
@@ -463,7 +491,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
                 .createStatement()
                 .executeQuery(
                         "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH),COUNT(*) from SYSTEM.STATS where PHYSICAL_NAME = '"
-                                + fullTableName + "' GROUP BY COLUMN_FAMILY ORDER BY COLUMN_FAMILY");
+                                + physicalTableName + "' GROUP BY COLUMN_FAMILY ORDER BY COLUMN_FAMILY");
 
         assertTrue(rs.next());
         assertEquals("A", rs.getString(1));
@@ -497,12 +525,12 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         TestUtil.analyzeTable(conn, fullTableName);
         // Assert that there are no more guideposts
         rs = conn.createStatement().executeQuery("SELECT count(1) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + 
-                " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + "='" + fullTableName + "' AND " + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NOT NULL");
+                " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + "='" + physicalTableName + "' AND " + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(0, rs.getLong(1));
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER " + fullTableName,
+        assertEquals("CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
                 QueryUtil.getExplainPlan(rs));
     }
 
@@ -532,9 +560,10 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
             int endIndex = r.nextInt(strings.length - startIndex) + startIndex;
             long rows = endIndex - startIndex;
             long c2Bytes = rows * 35;
+            String physicalTableName = SchemaUtil.getPhysicalHBaseTableName(fullTableName, userTableNamespaceMapped, PTableType.TABLE).getString();
             rs = conn.createStatement().executeQuery(
                     "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH) from SYSTEM.STATS where PHYSICAL_NAME = '"
-                            + fullTableName + "' AND GUIDE_POST_KEY>= cast('" + strings[startIndex]
+                            + physicalTableName + "' AND GUIDE_POST_KEY>= cast('" + strings[startIndex]
                             + "' as varbinary) AND  GUIDE_POST_KEY<cast('" + strings[endIndex]
                             + "' as varbinary) and COLUMN_FAMILY='C2' group by COLUMN_FAMILY");
             if (startIndex < endIndex) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
new file mode 100644
index 0000000..6b394c1
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.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.end2end;
+
+import java.util.Map;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+
+import com.google.common.collect.Maps;
+
+public class SysTableNamespaceMappedStatsCollectorIT extends StatsCollectorIT {
+    
+    public SysTableNamespaceMappedStatsCollectorIT(boolean transactional, boolean userTableNamespaceMapped) {
+        super(transactional, userTableNamespaceMapped);
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        // enable name space mapping at global level on both client and server side
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        serverProps.put(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, "true");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        clientProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        clientProps.put(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
index a30377b..547878c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
@@ -25,7 +25,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -40,7 +40,7 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public abstract class BaseLocalIndexIT extends BaseOwnClusterIT {
+public abstract class BaseLocalIndexIT extends BaseUniqueNamesOwnClusterIT {
     protected boolean isNamespaceMapped;
     protected String schemaName;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index e677878..3ee9721 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
@@ -67,7 +67,7 @@ import com.google.common.collect.Maps;
 
 
 @RunWith(Parameterized.class)
-public class ImmutableIndexIT extends BaseOwnClusterIT {
+public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
 
     private final boolean localIndex;
     private final String tableDDLOptions;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
index 03990c1..ec60151 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
@@ -41,13 +41,14 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.hbase.index.Indexer;
 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.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -57,13 +58,13 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class TxWriteFailureIT extends BaseOwnClusterIT {
+public class TxWriteFailureIT extends BaseUniqueNamesOwnClusterIT {
 	
-    private static final String SCHEMA_NAME = "S";
-    private static final String DATA_TABLE_NAME = "T";
-    private static final String INDEX_TABLE_NAME = "I";
-    private static final String DATA_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, DATA_TABLE_NAME);
-    private static final String INDEX_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, INDEX_TABLE_NAME);
+    private String schemaName;
+    private String dataTableName;
+    private String indexName;
+    private String dataTableFullName;
+    private String indexFullName;
     private static final String ROW_TO_FAIL = "fail";
     
     private final boolean localIndex;
@@ -92,6 +93,15 @@ public class TxWriteFailureIT extends BaseOwnClusterIT {
                  { false, false }, { false, true }, { true, false }, { true, true }
            });
     }
+    
+    @Before
+    public void generateTableNames() throws SQLException {
+        schemaName = generateUniqueName();
+        dataTableName = generateUniqueName();
+        indexName = generateUniqueName();
+        dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
+        indexFullName = SchemaUtil.getTableName(schemaName, indexName); 
+    }
 	
 	@Test
     public void testIndexTableWriteFailure() throws Exception {
@@ -110,11 +120,11 @@ public class TxWriteFailureIT extends BaseOwnClusterIT {
         Connection conn = driver.connect(url, props);
         conn.setAutoCommit(false);
         conn.createStatement().execute(
-                "CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR PRIMARY KEY, v1 VARCHAR)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
+                "CREATE TABLE " + dataTableFullName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
         conn.createStatement().execute(
-                "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v1)");
+                "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + dataTableFullName + " (v1)");
         
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableFullName + " VALUES(?,?)");
         // to create a data table write failure set k as the ROW_TO_FAIL, to create an index table write failure set v1 as the ROW_TO_FAIL, 
         // FailingRegionObserver will throw an exception if the put contains ROW_TO_FAIL
         stmt.setString(1, !indexTableWriteFailure ? ROW_TO_FAIL : "k1");
@@ -137,9 +147,9 @@ public class TxWriteFailureIT extends BaseOwnClusterIT {
         conn.commit();
         
         // verify that only k3,v3 exists in the data table
-        String dataSql = "SELECT k, v1 FROM " + DATA_TABLE_FULL_NAME + " order by k";
+        String dataSql = "SELECT k, v1 FROM " + dataTableFullName + " order by k";
         rs = conn.createStatement().executeQuery("EXPLAIN "+dataSql);
-        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER S.T",
+        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableFullName,
                 QueryUtil.getExplainPlan(rs));
         rs = conn.createStatement().executeQuery(dataSql);
         assertTrue(rs.next());
@@ -148,16 +158,16 @@ public class TxWriteFailureIT extends BaseOwnClusterIT {
         assertFalse(rs.next());
 
         // verify the only k3,v3  exists in the index table
-        String indexSql = "SELECT k, v1 FROM " + DATA_TABLE_FULL_NAME + " order by v1";
+        String indexSql = "SELECT k, v1 FROM " + dataTableFullName + " order by v1";
         rs = conn.createStatement().executeQuery("EXPLAIN "+indexSql);
         if(localIndex) {
             assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + DATA_TABLE_FULL_NAME + " [1]\n" + 
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableFullName + " [1]\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT",
                 QueryUtil.getExplainPlan(rs));
         } else {
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + INDEX_TABLE_FULL_NAME + "\n    SERVER FILTER BY FIRST KEY ONLY",
+	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexFullName + "\n    SERVER FILTER BY FIRST KEY ONLY",
 	                QueryUtil.getExplainPlan(rs));
         }
         rs = conn.createStatement().executeQuery(indexSql);
@@ -166,7 +176,7 @@ public class TxWriteFailureIT extends BaseOwnClusterIT {
         assertEquals("v3", rs.getString(2));
         assertFalse(rs.next());
         
-        conn.createStatement().execute("DROP TABLE " + DATA_TABLE_FULL_NAME);
+        conn.createStatement().execute("DROP TABLE " + dataTableFullName);
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorWithStatsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorWithStatsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorWithStatsIT.java
index c7c0b39..928e161 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorWithStatsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorWithStatsIT.java
@@ -29,19 +29,27 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.phoenix.compile.StatementContext;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class RoundRobinResultIteratorWithStatsIT extends BaseOwnClusterIT {
+public class RoundRobinResultIteratorWithStatsIT extends BaseUniqueNamesOwnClusterIT {
+    
+    private String tableName;
+    
+    @Before
+    public void generateTableName() {
+        tableName = generateUniqueName();
+    }
     
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -62,9 +70,8 @@ public class RoundRobinResultIteratorWithStatsIT extends BaseOwnClusterIT {
     public void testRoundRobinBehavior() throws Exception {
         int nRows = 30000;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String testTable = "testRoundRobinBehavior".toUpperCase();
-            conn.createStatement().execute("CREATE TABLE " + testTable + "(K VARCHAR PRIMARY KEY)");
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + testTable + " VALUES(?)");
+            conn.createStatement().execute("CREATE TABLE " + tableName + "(K VARCHAR PRIMARY KEY)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?)");
             for (int i = 1; i <= nRows; i++) {
                 stmt.setString(1, i + "");
                 stmt.executeUpdate();
@@ -73,11 +80,11 @@ public class RoundRobinResultIteratorWithStatsIT extends BaseOwnClusterIT {
                 }
             }
             conn.commit();
-            conn.createStatement().execute("UPDATE STATISTICS " + testTable);
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
             PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
             MockParallelIteratorFactory parallelIteratorFactory = new MockParallelIteratorFactory();
             phxConn.setIteratorFactory(parallelIteratorFactory);
-            ResultSet rs = stmt.executeQuery("SELECT * FROM " + testTable);
+            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
             StatementContext ctx = rs.unwrap(PhoenixResultSet.class).getContext();
             PTable table = ctx.getResolver().getTables().get(0).getTable();
             parallelIteratorFactory.setTable(table);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
index fd6f91a..3af8ce7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
@@ -46,7 +46,7 @@ import java.util.Properties;
 import java.util.Set;
 
 import org.apache.phoenix.compile.StatementContext;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.query.QueryServices;
@@ -61,7 +61,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-public class PhoenixMetricsIT extends BaseOwnClusterIT {
+public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
 
     private static final List<String> mutationMetricsToSkip = Lists
             .newArrayList(MetricType.MUTATION_COMMIT_TIME.name());
@@ -89,10 +89,11 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testGlobalPhoenixMetricsForQueries() throws Exception {
-        createTableAndInsertValues("T", true);
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
         resetGlobalMetrics(); // we want to count metrics related only to the below query
         Connection conn = DriverManager.getConnection(getUrl());
-        String query = "SELECT * FROM T";
+        String query = "SELECT * FROM " + tableName;
         ResultSet rs = conn.createStatement().executeQuery(query);
         while (rs.next()) {
             rs.getString(1);
@@ -116,7 +117,8 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testGlobalPhoenixMetricsForMutations() throws Exception {
-        createTableAndInsertValues("T", true);
+        String tableName = generateUniqueName();
+        createTableAndInsertValues(tableName, true);
         assertEquals(10, GLOBAL_MUTATION_BATCH_SIZE.getMetric().getTotalSum());
         assertEquals(10, GLOBAL_MUTATION_SQL_COUNTER.getMetric().getTotalSum());
         assertTrue(GLOBAL_MUTATION_BYTES.getMetric().getTotalSum() > 0);
@@ -131,13 +133,15 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testGlobalPhoenixMetricsForUpsertSelect() throws Exception {
-        createTableAndInsertValues("T", true);
+        String tableFrom = generateUniqueName();
+        String tableTo = generateUniqueName();
+        createTableAndInsertValues(tableFrom, true);
         resetGlobalMetrics();
-        String ddl = "CREATE TABLE T2 (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
+        String ddl = "CREATE TABLE " + tableTo + "  (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute(ddl);
         resetGlobalMetrics();
-        String dml = "UPSERT INTO T2 (K, V) SELECT K, V FROM T";
+        String dml = "UPSERT INTO " + tableTo + " (K, V) SELECT K, V FROM " + tableFrom;
         conn.createStatement().executeUpdate(dml);
         conn.commit();
         assertEquals(10, GLOBAL_MUTATION_BATCH_SIZE.getMetric().getTotalSum());
@@ -181,7 +185,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testOverallQueryMetricsForSelect() throws Exception {
-        String tableName = "SCANMETRICS";
+        String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = 6";
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute(ddl);
@@ -189,7 +193,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testReadMetricsForSelect() throws Exception {
-        String tableName = "READMETRICSFORSELECT";
+        String tableName = generateUniqueName();
         long numSaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + numSaltBuckets;
@@ -214,7 +218,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForUpsert() throws Exception {
-        String tableName = "UPSERTMETRICS";
+        String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = 6";
         Connection ddlConn = DriverManager.getConnection(getUrl());
         ddlConn.createStatement().execute(ddl);
@@ -256,7 +260,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForUpsertSelect() throws Exception {
-        String tableName1 = "UPSERTFROM";
+        String tableName1 = generateUniqueName();
         long table1SaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName1 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + table1SaltBuckets;
@@ -266,7 +270,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
         int numRows = 10;
         insertRowsInTable(tableName1, numRows);
 
-        String tableName2 = "UPSERTTO";
+        String tableName2 = generateUniqueName();
         ddl = "CREATE TABLE " + tableName2 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = 10";
         ddlConn = DriverManager.getConnection(getUrl());
         ddlConn.createStatement().execute(ddl);
@@ -286,7 +290,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForDelete() throws Exception {
-        String tableName = "DELETEMETRICS";
+        String tableName = generateUniqueName();
         long tableSaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + tableSaltBuckets;
@@ -309,7 +313,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testNoMetricsCollectedForConnection() throws Exception {
-        String tableName = "NOMETRICS";
+        String tableName = generateUniqueName();
         long tableSaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + tableSaltBuckets;
@@ -334,7 +338,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForUpsertWithAutoCommit() throws Exception {
-        String tableName = "VERIFYUPSERTAUTOCOMMIT";
+        String tableName = generateUniqueName();
         long tableSaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + tableSaltBuckets;
@@ -374,7 +378,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForDeleteWithAutoCommit() throws Exception {
-        String tableName = "VERIFYDELETEAUTOCOMMIT";
+        String tableName = generateUniqueName();
         long tableSaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + tableSaltBuckets;
@@ -420,7 +424,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForUpsertSelectWithAutoCommit() throws Exception {
-        String tableName1 = "UPSERTFROMAUTOCOMMIT";
+        String tableName1 = generateUniqueName();
         long table1SaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName1 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + table1SaltBuckets;
@@ -430,7 +434,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
         int numRows = 10;
         insertRowsInTable(tableName1, numRows);
 
-        String tableName2 = "UPSERTTOAUTCOMMIT";
+        String tableName2 = generateUniqueName();
         ddl = "CREATE TABLE " + tableName2 + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = 10";
         ddlConn = DriverManager.getConnection(getUrl());
         ddlConn.createStatement().execute(ddl);
@@ -508,11 +512,11 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
     @Test
     public void testMutationMetricsWhenUpsertingToMultipleTables() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String table1 = "TABLE1";
+            String table1 = generateUniqueName();
             createTableAndInsertValues(true, 10, conn, table1);
-            String table2 = "TABLE2";
+            String table2 = generateUniqueName();
             createTableAndInsertValues(true, 10, conn, table2);
-            String table3 = "TABLE3";
+            String table3 = generateUniqueName();
             createTableAndInsertValues(true, 10, conn, table3);
             Map<String, Map<String, Long>> mutationMetrics = PhoenixRuntime.getWriteMetricsForMutationsSinceLastReset(conn);
             assertTrue("Mutation metrics not present for " + table1, mutationMetrics.get(table1) != null);
@@ -528,7 +532,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
         Connection conn = null;
         try {
             conn = DriverManager.getConnection(getUrl());
-            createTableAndInsertValues(true, 10, conn, "clearmetrics");
+            createTableAndInsertValues(true, 10, conn, generateUniqueName());
             assertTrue("Mutation metrics not present", PhoenixRuntime.getWriteMetricsForMutationsSinceLastReset(conn).size() > 0);
         } finally {
             if (conn != null) {
@@ -541,15 +545,15 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
 
     @Test
     public void testMetricsForUpsertingIntoImmutableTableWithIndices() throws Exception {
-        String dataTable = "IMMTABLEWITHINDICES";
+        String dataTable = generateUniqueName();
         String tableDdl = "CREATE TABLE "
                 + dataTable
                 + " (K1 VARCHAR NOT NULL, K2 VARCHAR NOT NULL, V1 INTEGER, V2 INTEGER, V3 INTEGER CONSTRAINT NAME_PK PRIMARY KEY(K1, K2)) IMMUTABLE_ROWS = true";
-        String index1 = "I1";
+        String index1 = generateUniqueName() + "_IDX";
         String index1Ddl = "CREATE INDEX " + index1 + " ON " + dataTable + " (V1) include (V2)";
-        String index2 = "I2";
+        String index2 = generateUniqueName() + "_IDX";
         String index2Ddl = "CREATE INDEX " + index2 + " ON " + dataTable + " (V2) include (V3)";
-        String index3 = "I3";
+        String index3 = generateUniqueName() + "_IDX";
         String index3Ddl = "CREATE INDEX " + index3 + " ON " + dataTable + " (V3) include (V1)";
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute(tableDdl);
@@ -586,7 +590,7 @@ public class PhoenixMetricsIT extends BaseOwnClusterIT {
     
     @Test
     public void testMetricsForUpsertSelectSameTable() throws Exception {
-        String tableName = "UPSERTSAME";
+        String tableName = generateUniqueName();
         long table1SaltBuckets = 6;
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + " SALT_BUCKETS = "
                 + table1SaltBuckets;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
index c015f66..8b7d5f2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
@@ -17,27 +17,30 @@ import static org.junit.Assert.assertTrue;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.ipc.CallRunner;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 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.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-public class PhoenixClientRpcIT extends BaseOwnClusterIT {
+public class PhoenixClientRpcIT extends BaseUniqueNamesOwnClusterIT {
 
-    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 String schemaName;
+    private String indexName;
+    private String indexFullName;
+    private String dataTableFullName;
 
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -52,6 +55,14 @@ public class PhoenixClientRpcIT extends BaseOwnClusterIT {
         TestPhoenixIndexRpcSchedulerFactory.reset();
         tearDownMiniCluster();
     }
+    
+    @Before
+    public void generateTableNames() throws SQLException {
+        schemaName = generateUniqueName();
+        indexName = generateUniqueName();
+        indexFullName = SchemaUtil.getTableName(schemaName, indexName);
+        dataTableFullName = SchemaUtil.getTableName(schemaName, generateUniqueName());
+    }
 
     @Test
     public void testIndexQos() throws Exception {
@@ -60,14 +71,14 @@ public class PhoenixClientRpcIT extends BaseOwnClusterIT {
         try {
             // create the table
             conn.createStatement().execute(
-                    "CREATE TABLE " + DATA_TABLE_FULL_NAME
+                    "CREATE TABLE " + dataTableFullName
                             + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) IMMUTABLE_ROWS=true");
 
             // create the index
             conn.createStatement().execute(
-                    "CREATE INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v1) INCLUDE (v2)");
+                    "CREATE INDEX " + indexName + " ON " + dataTableFullName + " (v1) INCLUDE (v2)");
 
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableFullName + " VALUES(?,?,?)");
             stmt.setString(1, "k1");
             stmt.setString(2, "v1");
             stmt.setString(3, "v2");
@@ -75,13 +86,13 @@ public class PhoenixClientRpcIT extends BaseOwnClusterIT {
             conn.commit();
 
             // run select query that should use the index
-            String selectSql = "SELECT k, v2 from " + DATA_TABLE_FULL_NAME + " WHERE v1=?";
+            String selectSql = "SELECT k, v2 from " + dataTableFullName + " 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));
+            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexFullName + " ['v1']", QueryUtil.getExplainPlan(rs));
 
             // verify that the correct results are returned
             rs = stmt.executeQuery();
@@ -103,7 +114,7 @@ public class PhoenixClientRpcIT extends BaseOwnClusterIT {
         Connection conn = driver.connect(getUrl(), props);
         try {
             // create the table
-            conn.createStatement().execute("CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR)");
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR)");
             // verify that that metadata queue is used at least once
             Mockito.verify(TestPhoenixIndexRpcSchedulerFactory.getMetadataRpcExecutor(), Mockito.atLeastOnce()).dispatch(Mockito.any(CallRunner.class));
         } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
index 6605f16..92f7294 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixServerRpcIT.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.assertTrue;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -43,22 +44,23 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 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.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-public class PhoenixServerRpcIT extends BaseOwnClusterIT {
+public class PhoenixServerRpcIT extends BaseUniqueNamesOwnClusterIT {
 
-    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 String schemaName;
+    private String indexName;
+    private String dataTableFullName;
+    private String indexTableFullName;
     
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -76,6 +78,14 @@ public class PhoenixServerRpcIT extends BaseOwnClusterIT {
         TestPhoenixIndexRpcSchedulerFactory.reset();
     }
     
+    @Before
+    public void generateTableNames() throws SQLException {
+        schemaName = generateUniqueName();
+        indexName = generateUniqueName();
+        indexTableFullName = SchemaUtil.getTableName(schemaName, indexName);
+        dataTableFullName = SchemaUtil.getTableName(schemaName, generateUniqueName());
+    }
+    
     @Test
     public void testIndexQos() throws Exception { 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -83,15 +93,15 @@ public class PhoenixServerRpcIT extends BaseOwnClusterIT {
         try {
             // create the table 
             conn.createStatement().execute(
-                    "CREATE TABLE " + DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+                    "CREATE TABLE " + dataTableFullName + " (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)");
+                    "CREATE INDEX " + indexName + " ON " + dataTableFullName + " (v1) INCLUDE (v2)");
 
-            ensureTablesOnDifferentRegionServers(DATA_TABLE_FULL_NAME, INDEX_TABLE_FULL_NAME);
+            ensureTablesOnDifferentRegionServers(dataTableFullName, indexTableFullName);
     
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableFullName + " VALUES(?,?,?)");
             stmt.setString(1, "k1");
             stmt.setString(2, "v1");
             stmt.setString(3, "v2");
@@ -99,13 +109,13 @@ public class PhoenixServerRpcIT extends BaseOwnClusterIT {
             conn.commit();
     
             // run select query that should use the index
-            String selectSql = "SELECT k, v2 from " + DATA_TABLE_FULL_NAME + " WHERE v1=?";
+            String selectSql = "SELECT k, v2 from " + dataTableFullName + " 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));
+            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableFullName + " ['v1']", QueryUtil.getExplainPlan(rs));
     
             // verify that the correct results are returned
             rs = stmt.executeQuery();
@@ -116,13 +126,13 @@ public class PhoenixServerRpcIT extends BaseOwnClusterIT {
             
             // drop index table 
             conn.createStatement().execute(
-                    "DROP INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME );
+                    "DROP INDEX " + indexName + " ON " + dataTableFullName );
             // 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)");
+                    "CREATE TABLE " + indexTableFullName + " (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 = conn.prepareStatement("UPSERT INTO " + indexTableFullName + " VALUES(?,?,?)");
             stmt.setString(1, "k1");
             stmt.setString(2, "v1");
             stmt.setString(3, "v2");
@@ -130,7 +140,7 @@ public class PhoenixServerRpcIT extends BaseOwnClusterIT {
             conn.commit();
             
             // run select query on the new table
-            selectSql = "SELECT k, v2 from " + INDEX_TABLE_FULL_NAME + " WHERE v1=?";
+            selectSql = "SELECT k, v2 from " + indexTableFullName + " WHERE v1=?";
             stmt = conn.prepareStatement(selectSql);
             stmt.setString(1, "v1");
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f53b57e1/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 003fd73..9f22239 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
@@ -994,7 +994,7 @@ public class SchemaUtil {
 
     public static TableName getPhysicalTableName(byte[] fullTableName, Configuration conf) {
         return getPhysicalTableName(fullTableName, isNamespaceMappingEnabled(
-                isSystemTable(fullTableName) ? PTableType.SYSTEM : null, new ReadOnlyProps(conf.iterator())));
+                isSystemTable(fullTableName) ? PTableType.SYSTEM : null, conf));
     }
 
     public static TableName getPhysicalName(byte[] fullTableName, ReadOnlyProps readOnlyProps) {
@@ -1019,6 +1019,14 @@ public class SchemaUtil {
     public static boolean isSchemaCheckRequired(PTableType tableType, ReadOnlyProps props) {
         return PTableType.TABLE.equals(tableType) && isNamespaceMappingEnabled(tableType, props);
     }
+    
+    public static boolean isNamespaceMappingEnabled(PTableType type, Configuration conf) {
+        return conf.getBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED,
+                QueryServicesOptions.DEFAULT_IS_NAMESPACE_MAPPING_ENABLED)
+                && (type == null || !PTableType.SYSTEM.equals(type)
+                        || conf.getBoolean(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE,
+                                QueryServicesOptions.DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE));
+    }
 
     public static boolean isNamespaceMappingEnabled(PTableType type, ReadOnlyProps readOnlyProps) {
         return readOnlyProps.getBoolean(QueryServices.IS_NAMESPACE_MAPPING_ENABLED,