You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/09/14 21:42:59 UTC

[01/50] [abbrv] phoenix git commit: PHOENIX-3126 Tie a driver instance to a specific user (Prabhjyot Singh)

Repository: phoenix
Updated Branches:
  refs/heads/calcite 6861a5eb3 -> faff90afd


PHOENIX-3126 Tie a driver instance to a specific user (Prabhjyot Singh)

Prevent the case where a user's Kerberos credentials are
unintentionally used by a different user.


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

Branch: refs/heads/calcite
Commit: a9ea8a3baa32714a1640c1197609910863daca79
Parents: 545cc1c
Author: Josh Elser <el...@apache.org>
Authored: Tue Aug 2 16:56:34 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 2 18:23:37 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/jdbc/PhoenixEmbeddedDriver.java   | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9ea8a3b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index d2dd94f..375388a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.jdbc;
 
 import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Driver;
 import java.sql.DriverPropertyInfo;
@@ -35,6 +36,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -340,6 +342,7 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
         private final boolean isConnectionless;
         private final String principal;
         private final String keytab;
+        private final User user;
         
         public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode, String principal, String keytab) {
             this.zookeeperQuorum = zookeeperQuorum;
@@ -348,6 +351,14 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             this.isConnectionless = PhoenixRuntime.CONNECTIONLESS.equals(zookeeperQuorum);
             this.principal = principal;
             this.keytab = keytab;
+            try {
+                this.user = User.getCurrent();
+            } catch (IOException e) {
+                throw new RuntimeException("Couldn't get the current user!!");
+            }
+            if (null == this.user) {
+                throw new RuntimeException("Acquired null user which should never happen");
+            }
         }
         
         public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode) {
@@ -406,6 +417,8 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             result = prime * result + ((rootNode == null) ? 0 : rootNode.hashCode());
             result = prime * result + ((principal == null) ? 0 : principal.hashCode());
             result = prime * result + ((keytab == null) ? 0 : keytab.hashCode());
+            // `user` is guaranteed to be non-null
+            result = prime * result + user.hashCode();
             return result;
         }
 
@@ -415,6 +428,8 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             if (obj == null) return false;
             if (getClass() != obj.getClass()) return false;
             ConnectionInfo other = (ConnectionInfo) obj;
+            // `user` is guaranteed to be non-null
+            if (!other.user.equals(user)) return false;
             if (zookeeperQuorum == null) {
                 if (other.zookeeperQuorum != null) return false;
             } else if (!zookeeperQuorum.equals(other.zookeeperQuorum)) return false;


[26/50] [abbrv] phoenix git commit: PHOENIX-1751 Perform aggregations, sorting, etc, in the preScannerNext instead of postScannerOpen (Lars Hofhansl)

Posted by ma...@apache.org.
PHOENIX-1751 Perform aggregations, sorting, etc, in the preScannerNext instead of postScannerOpen (Lars Hofhansl)


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

Branch: refs/heads/calcite
Commit: 9cc77c8104245a138a62ba251e926e14a74d8c11
Parents: a0ae802
Author: James Taylor <ja...@apache.org>
Authored: Tue Aug 23 12:39:45 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Wed Aug 24 09:21:28 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/RenewLeaseIT.java    |  90 +++++++++++++
 .../coprocessor/BaseScannerRegionObserver.java  | 126 +++++++++++++------
 .../coprocessor/DelegateRegionScanner.java      |   2 +-
 3 files changed, 176 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9cc77c81/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
new file mode 100644
index 0000000..fa0bc8e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RenewLeaseIT.java
@@ -0,0 +1,90 @@
+/*
+ * 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 maynot 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 applicablelaw 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 static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+
+public class RenewLeaseIT extends BaseOwnClusterHBaseManagedTimeIT {
+    private static final long RPC_TIMEOUT = 2000;
+    private static volatile boolean SLEEP_NOW = false;
+    private static final String TABLE_NAME = "FOO_BAR";
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put("hbase.coprocessor.region.classes", SleepingRegionObserver.class.getName());
+        Map<String,String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        // Must update config before starting server
+        clientProps.put("hbase.rpc.timeout", Long.toString(RPC_TIMEOUT));
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testLeaseDoesNotTimeout() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        conn.createStatement().execute("create table " + TABLE_NAME + "(k VARCHAR PRIMARY KEY)");
+        SLEEP_NOW = true;
+        try {
+            ResultSet rs = conn.createStatement().executeQuery("select count(*) from " + TABLE_NAME);
+            assertTrue(rs.next());
+            assertEquals(0, rs.getLong(1));
+        } finally {
+            SLEEP_NOW = false;
+        }
+    }
+    
+    public static class SleepingRegionObserver extends SimpleRegionObserver {
+        @Override
+        public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
+                final InternalScanner s, final List<Result> results,
+                final int limit, final boolean hasMore) throws IOException {
+            try {
+                if (SLEEP_NOW && c.getEnvironment().getRegion().getRegionInfo().getTable().getNameAsString().equals(TABLE_NAME)) {
+                    Thread.sleep(RPC_TIMEOUT * 2);
+                }
+            } catch (InterruptedException e) {
+                throw new IOException(e);
+            }
+            return super.preScannerNext(c, s, results, limit, hasMore);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9cc77c81/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 39ac6fe..4fa1399 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
@@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
-import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -60,11 +58,10 @@ import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.tephra.Transaction;
 
 import com.google.common.collect.ImmutableList;
 
-import org.apache.tephra.Transaction;
-
 
 abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
 
@@ -189,7 +186,89 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
         return s;
     }
 
-    /**
+    private class RegionScannerHolder extends DelegateRegionScanner {
+            private final Scan scan;
+            private final ObserverContext<RegionCoprocessorEnvironment> c;
+            private boolean wasOverriden;
+            
+            public RegionScannerHolder(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan, final RegionScanner scanner) {
+                super(scanner);
+                this.c = c;
+                this.scan = scan;
+            }
+    
+            private void overrideDelegate() throws IOException {
+                if (wasOverriden) {
+                    return;
+                }
+                boolean success = false;
+                // Save the current span. When done with the child span, reset the span back to
+                // what it was. Otherwise, this causes the thread local storing the current span
+                // to not be reset back to null causing catastrophic infinite loops
+                // and region servers to crash. See https://issues.apache.org/jira/browse/PHOENIX-1596
+                // TraceScope can't be used here because closing the scope will end up calling
+                // currentSpan.stop() and that should happen only when we are closing the scanner.
+                final Span savedSpan = Trace.currentSpan();
+                final Span child = Trace.startSpan(SCANNER_OPENED_TRACE_INFO, savedSpan).getSpan();
+                try {
+                    RegionScanner scanner = doPostScannerOpen(c, scan, delegate);
+                    scanner = new DelegateRegionScanner(scanner) {
+                        // This isn't very obvious but close() could be called in a thread
+                        // that is different from the thread that created the scanner.
+                        @Override
+                        public void close() throws IOException {
+                            try {
+                                delegate.close();
+                            } finally {
+                                if (child != null) {
+                                    child.stop();
+                                }
+                            }
+                        }
+                    };
+                    this.delegate = scanner;
+                    wasOverriden = true;
+                    success = true;
+                } catch (Throwable t) {
+                    ServerUtil.throwIOException(c.getEnvironment().getRegionInfo().getRegionNameAsString(), t);
+                } finally {
+                    try {
+                        if (!success && child != null) {
+                            child.stop();
+                        }
+                    } finally {
+                        Trace.continueSpan(savedSpan);
+                    }
+                }
+            }
+            
+            @Override
+            public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+                overrideDelegate();
+                return super.next(result, scannerContext);
+            }
+
+            @Override
+            public boolean next(List<Cell> result) throws IOException {
+                overrideDelegate();
+                return super.next(result);
+            }
+
+            @Override
+            public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+                overrideDelegate();
+                return super.nextRaw(result, scannerContext);
+            }
+            
+            @Override
+            public boolean nextRaw(List<Cell> result) throws IOException {
+                overrideDelegate();
+                return super.nextRaw(result);
+            }
+        }
+        
+
+        /**
      * Wrapper for {@link #postScannerOpen(ObserverContext, Scan, RegionScanner)} that ensures no non IOException is thrown,
      * to prevent the coprocessor from becoming blacklisted.
      *
@@ -202,42 +281,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             if (!isRegionObserverFor(scan)) {
                 return s;
             }
-            boolean success = false;
-            // Save the current span. When done with the child span, reset the span back to
-            // what it was. Otherwise, this causes the thread local storing the current span
-            // to not be reset back to null causing catastrophic infinite loops
-            // and region servers to crash. See https://issues.apache.org/jira/browse/PHOENIX-1596
-            // TraceScope can't be used here because closing the scope will end up calling
-            // currentSpan.stop() and that should happen only when we are closing the scanner.
-            final Span savedSpan = Trace.currentSpan();
-            final Span child = Trace.startSpan(SCANNER_OPENED_TRACE_INFO, savedSpan).getSpan();
-            try {
-                RegionScanner scanner = doPostScannerOpen(c, scan, s);
-                scanner = new DelegateRegionScanner(scanner) {
-                    // This isn't very obvious but close() could be called in a thread
-                    // that is different from the thread that created the scanner.
-                    @Override
-                    public void close() throws IOException {
-                        try {
-                            delegate.close();
-                        } finally {
-                            if (child != null) {
-                                child.stop();
-                            }
-                        }
-                    }
-                };
-                success = true;
-                return scanner;
-            } finally {
-                try {
-                    if (!success && child != null) {
-                        child.stop();
-                    }
-                } finally {
-                    Trace.continueSpan(savedSpan);
-                }
-            }
+            return new RegionScannerHolder(c, scan, s);
         } catch (Throwable t) {
             // If the exception is NotServingRegionException then throw it as
             // StaleRegionBoundaryCacheException to handle it by phoenix client other wise hbase

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9cc77c81/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
index 43c35a8..0ddabed 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 public class DelegateRegionScanner implements RegionScanner {
 
-    protected final RegionScanner delegate;
+    protected RegionScanner delegate;
 
     public DelegateRegionScanner(RegionScanner scanner) {
         this.delegate = scanner;


[19/50] [abbrv] phoenix git commit: PHOENIX-2995 Write performance severely degrades with large number of views

Posted by ma...@apache.org.
PHOENIX-2995 Write performance severely degrades with large number of views


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

Branch: refs/heads/calcite
Commit: 3130fa9919ba937d98853f9be8068a0a8a8a96f5
Parents: 657917b
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Fri Jul 22 14:24:38 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Aug 17 11:16:52 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/UpsertSelectIT.java  |   2 +-
 .../phoenix/compile/CreateTableCompiler.java    |   7 +-
 .../apache/phoenix/execute/MutationState.java   | 328 ++++++++++++-------
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  57 ++--
 .../query/ConnectionQueryServicesImpl.java      |  93 +++---
 .../query/ConnectionlessQueryServicesImpl.java  |  47 ++-
 .../query/DelegateConnectionQueryServices.java  |  38 +--
 .../apache/phoenix/query/MetaDataMutated.java   |  19 +-
 .../org/apache/phoenix/schema/PMetaData.java    |   4 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    | 107 +++---
 .../phoenix/schema/PSynchronizedMetaData.java   | 249 ++++++++++++++
 .../apache/phoenix/util/TransactionUtil.java    |   4 +-
 .../phoenix/schema/PMetaDataImplTest.java       |  68 ++--
 13 files changed, 652 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 30de4de..4d811a4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -1022,7 +1022,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
             // Upsert data with scn set on the connection. The timestamp of the put will be the value of the row_timestamp column.
             long rowTimestamp1 = 100;
             Date rowTimestampDate = new Date(rowTimestamp1);
-            try (Connection conn = getConnection(ts)) {
+            try (Connection conn = getConnection(ts+1)) {
                 PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + tableName + " (PK1, PK2, PK3, KV1) VALUES(?, ?, ?, ?)");
                 stmt.setInt(1, 1);
                 stmt.setDate(2, rowTimestampDate);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index b545156..3928f66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -54,7 +54,6 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PDatum;
-import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableType;
@@ -143,11 +142,11 @@ public class CreateTableCompiler {
                             // on our connection.
                             new DelegateConnectionQueryServices(connection.getQueryServices()) {
                                 @Override
-                                public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
-                                    return connection.addTable(table, resolvedTime);
+                                public void addTable(PTable table, long resolvedTime) throws SQLException {
+                                    connection.addTable(table, resolvedTime);
                                 }
                             },
-                            connection, tableRef.getTimeStamp());
+                            connection, tableRef.getTimeStamp()+1);
                     viewColumnConstantsToBe = new byte[nColumns][];
                     ViewWhereExpressionVisitor visitor = new ViewWhereExpressionVisitor(parentToBe, viewColumnConstantsToBe);
                     where.accept(visitor);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index fb514b0..7a9282c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -30,6 +30,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -70,6 +71,7 @@ import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PMetaData;
+import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
@@ -358,7 +360,7 @@ public class MutationState implements SQLCloseable {
         HTableInterface htable = this.getConnection().getQueryServices().getTable(table.getPhysicalName().getBytes());
         Transaction currentTx;
         if (table.isTransactional() && (currentTx=getTransaction()) != null) {
-            TransactionAwareHTable txAware = TransactionUtil.getTransactionAwareHTable(htable, table);
+            TransactionAwareHTable txAware = TransactionUtil.getTransactionAwareHTable(htable, table.isImmutableRows());
             // Using cloned mutationState as we may have started a new transaction already
             // if auto commit is true and we need to use the original one here.
             txAware.startTx(currentTx);
@@ -552,7 +554,7 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-    private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
+    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
             final long timestamp, boolean includeAllIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
@@ -564,7 +566,7 @@ public class MutationState implements SQLCloseable {
         final List<Mutation> mutationList = Lists.newArrayListWithExpectedSize(values.size());
         final List<Mutation> mutationsPertainingToIndex = indexes.hasNext() ? Lists.<Mutation>newArrayListWithExpectedSize(values.size()) : null;
         generateMutations(tableRef, timestamp, values, mutationList, mutationsPertainingToIndex);
-        return new Iterator<Pair<byte[],List<Mutation>>>() {
+        return new Iterator<Pair<PName,List<Mutation>>>() {
             boolean isFirst = true;
 
             @Override
@@ -573,10 +575,10 @@ public class MutationState implements SQLCloseable {
             }
 
             @Override
-            public Pair<byte[], List<Mutation>> next() {
+            public Pair<PName, List<Mutation>> next() {
                 if (isFirst) {
                     isFirst = false;
-                    return new Pair<byte[],List<Mutation>>(table.getPhysicalName().getBytes(), mutationList);
+                    return new Pair<PName,List<Mutation>>(table.getPhysicalName(), mutationList);
                 }
                 PTable index = indexes.next();
                 List<Mutation> indexMutations;
@@ -597,7 +599,7 @@ public class MutationState implements SQLCloseable {
                 } catch (SQLException e) {
                     throw new IllegalDataException(e);
                 }
-                return new Pair<byte[],List<Mutation>>(index.getPhysicalName().getBytes(),indexMutations);
+                return new Pair<PName,List<Mutation>>(index.getPhysicalName(),indexMutations);
             }
 
             @Override
@@ -684,7 +686,24 @@ public class MutationState implements SQLCloseable {
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
                     
             private Iterator<Pair<byte[],List<Mutation>>> init() {
-                return addRowMutations(current.getKey(), current.getValue(), timestamp, includeMutableIndexes, true);
+                final Iterator<Pair<PName, List<Mutation>>> mutationIterator = addRowMutations(current.getKey(), current.getValue(), timestamp, includeMutableIndexes, true);
+                return new Iterator<Pair<byte[],List<Mutation>>>() {
+                    @Override
+                    public boolean hasNext() {
+                        return mutationIterator.hasNext();
+                    }
+
+                    @Override
+                    public Pair<byte[], List<Mutation>> next() {
+                        Pair<PName, List<Mutation>> pair = mutationIterator.next();
+                        return new Pair<byte[], List<Mutation>>(pair.getFirst().getBytes(), pair.getSecond());
+                    }
+                    
+                    @Override
+                    public void remove() {
+                        mutationIterator.remove();
+                    }
+                };
             }
             
             @Override
@@ -869,6 +888,55 @@ public class MutationState implements SQLCloseable {
         }
     }
     
+    private static class TableInfo {
+        
+        private final boolean isDataTable;
+        @Nonnull private final PName hTableName;
+        @Nonnull private final TableRef origTableRef;
+        
+        public TableInfo(boolean isDataTable, PName hTableName, TableRef origTableRef) {
+            super();
+            checkNotNull(hTableName);
+            checkNotNull(origTableRef);
+            this.isDataTable = isDataTable;
+            this.hTableName = hTableName;
+            this.origTableRef = origTableRef;
+        }
+
+        public boolean isDataTable() {
+            return isDataTable;
+        }
+
+        public PName getHTableName() {
+            return hTableName;
+        }
+
+        public TableRef getOrigTableRef() {
+            return origTableRef;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + hTableName.hashCode();
+            result = prime * result + (isDataTable ? 1231 : 1237);
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            TableInfo other = (TableInfo) obj;
+            if (!hTableName.equals(other.hTableName)) return false;
+            if (isDataTable != other.isDataTable) return false;
+            return true;
+        }
+
+    }
+    
     @SuppressWarnings("deprecation")
     private void send(Iterator<TableRef> tableRefIterator) throws SQLException {
         int i = 0;
@@ -882,6 +950,7 @@ public class MutationState implements SQLCloseable {
 
         Map<ImmutableBytesPtr, RowMutationState> valuesMap;
         List<TableRef> txTableRefs = Lists.newArrayListWithExpectedSize(mutations.size());
+        Map<TableInfo,List<Mutation>> physicalTableMutationMap = Maps.newLinkedHashMap(); 
         // add tracing for this operation
         try (TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables")) {
             Span span = trace.getSpan();
@@ -897,126 +966,24 @@ public class MutationState implements SQLCloseable {
                 // Validate as we go if transactional since we can undo if a problem occurs (which is unlikely)
                 long serverTimestamp = serverTimeStamps == null ? validate(tableRef, valuesMap) : serverTimeStamps[i++];
                 final PTable table = tableRef.getTable();
-                // Track tables to which we've sent uncommitted data
-                if (isTransactional = table.isTransactional()) {
-                    txTableRefs.add(tableRef);
-                    addDMLFence(table);
-                    uncommittedPhysicalNames.add(table.getPhysicalName().getString());
-                }
+                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, serverTimestamp, false, sendAll);
+                // build map from physical table to mutation list
                 boolean isDataTable = true;
-                table.getIndexMaintainers(indexMetaDataPtr, connection);
-                Iterator<Pair<byte[],List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, serverTimestamp, false, sendAll);
                 while (mutationsIterator.hasNext()) {
-                    Pair<byte[],List<Mutation>> pair = mutationsIterator.next();
-                    byte[] htableName = pair.getFirst();
+                    Pair<PName,List<Mutation>> pair = mutationsIterator.next();
+                    PName hTableName = pair.getFirst();
                     List<Mutation> mutationList = pair.getSecond();
-                    
-                    //create a span per target table
-                    //TODO maybe we can be smarter about the table name to string here?
-                    Span child = Tracing.child(span,"Writing mutation batch for table: "+Bytes.toString(htableName));
-    
-                    int retryCount = 0;
-                    boolean shouldRetry = false;
-                    do {
-                        final ServerCache cache = isDataTable ? setMetaDataOnMutations(tableRef, mutationList, indexMetaDataPtr) : null;
-                    
-                        // If we haven't retried yet, retry for this case only, as it's possible that
-                        // a split will occur after we send the index metadata cache to all known
-                        // region servers.
-                        shouldRetry = cache != null;
-                        SQLException sqlE = null;
-                        HTableInterface hTable = connection.getQueryServices().getTable(htableName);
-                        try {
-                            if (isTransactional) {
-                                // If we have indexes, wrap the HTable in a delegate HTable that
-                                // will attach the necessary index meta data in the event of a
-                                // rollback
-                                if (!table.getIndexes().isEmpty()) {
-                                    hTable = new MetaDataAwareHTable(hTable, tableRef);
-                                }
-                                TransactionAwareHTable txnAware = TransactionUtil.getTransactionAwareHTable(hTable, table);
-                                // Don't add immutable indexes (those are the only ones that would participate
-                                // during a commit), as we don't need conflict detection for these.
-                                if (isDataTable) {
-                                    // Even for immutable, we need to do this so that an abort has the state
-                                    // necessary to generate the rows to delete.
-                                    addTransactionParticipant(txnAware);
-                                } else {
-                                    txnAware.startTx(getTransaction());
-                                }
-                                hTable = txnAware;
-                            }
-                            long numMutations = mutationList.size();
-                            GLOBAL_MUTATION_BATCH_SIZE.update(numMutations);
-                            
-                            long startTime = System.currentTimeMillis();
-                            child.addTimelineAnnotation("Attempt " + retryCount);
-                            hTable.batch(mutationList);
-                            if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
-                            child.stop();
-                            child.stop();
-                            shouldRetry = false;
-                            long mutationCommitTime = System.currentTimeMillis() - startTime;
-                            GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
-                            
-                            long mutationSizeBytes = calculateMutationSize(mutationList);
-                            MutationMetric mutationsMetric = new MutationMetric(numMutations, mutationSizeBytes, mutationCommitTime);
-                            mutationMetricQueue.addMetricsForTable(Bytes.toString(htableName), mutationsMetric);
-                        } catch (Exception e) {
-                            SQLException inferredE = ServerUtil.parseServerExceptionOrNull(e);
-                            if (inferredE != null) {
-                                if (shouldRetry && retryCount == 0 && inferredE.getErrorCode() == SQLExceptionCode.INDEX_METADATA_NOT_FOUND.getErrorCode()) {
-                                    // Swallow this exception once, as it's possible that we split after sending the index metadata
-                                    // and one of the region servers doesn't have it. This will cause it to have it the next go around.
-                                    // If it fails again, we don't retry.
-                                    String msg = "Swallowing exception and retrying after clearing meta cache on connection. " + inferredE;
-                                    logger.warn(LogUtil.addCustomAnnotations(msg, connection));
-                                    connection.getQueryServices().clearTableRegionCache(htableName);
-    
-                                    // add a new child span as this one failed
-                                    child.addTimelineAnnotation(msg);
-                                    child.stop();
-                                    child = Tracing.child(span,"Failed batch, attempting retry");
-    
-                                    continue;
-                                }
-                                e = inferredE;
-                            }
-                            // Throw to client an exception that indicates the statements that
-                            // were not committed successfully.
-                            sqlE = new CommitException(e, getUncommittedStatementIndexes());
-                        } finally {
-                            try {
-                                if (cache != null) {
-                                    cache.close();
-                                }
-                            } finally {
-                                try {
-                                    hTable.close();
-                                } 
-                                catch (IOException e) {
-                                    if (sqlE != null) {
-                                        sqlE.setNextException(ServerUtil.parseServerException(e));
-                                    } else {
-                                        sqlE = ServerUtil.parseServerException(e);
-                                    }
-                                } 
-                                if (sqlE != null) {
-                                    throw sqlE;
-                                }
-                            }
-                        }
-                    } while (shouldRetry && retryCount++ < 1);
+                    TableInfo tableInfo = new TableInfo(isDataTable, hTableName, tableRef);
+                    List<Mutation> oldMutationList = physicalTableMutationMap.put(tableInfo, mutationList);
+                    if (oldMutationList!=null)
+                        mutationList.addAll(0, oldMutationList);
                     isDataTable = false;
                 }
-                if (tableRef.getTable().getType() != PTableType.INDEX) {
-                    numRows -= valuesMap.size();
-                }
                 // For transactions, track the statement indexes as we send data
                 // over because our CommitException should include all statements
                 // involved in the transaction since none of them would have been
                 // committed in the event of a failure.
-                if (isTransactional) {
+                if (table.isTransactional()) {
                     addUncommittedStatementIndexes(valuesMap.values());
                     if (txMutations.isEmpty()) {
                         txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -1028,15 +995,122 @@ public class MutationState implements SQLCloseable {
                     // indexes have changed.
                     joinMutationState(new TableRef(tableRef), valuesMap, txMutations);
                 }
-                // Remove batches as we process them
-                if (sendAll) {
-                    // Iterating through map key set in this case, so we cannot use
-                    // the remove method without getting a concurrent modification
-                    // exception.
-                    tableRefIterator.remove();
-                } else {
-                    mutations.remove(tableRef);
-                }
+            }
+            Iterator<Entry<TableInfo, List<Mutation>>> mutationsIterator = physicalTableMutationMap.entrySet().iterator();
+            while (mutationsIterator.hasNext()) {
+                Entry<TableInfo, List<Mutation>> pair = mutationsIterator.next();
+                TableInfo tableInfo = pair.getKey();
+                byte[] htableName = tableInfo.getHTableName().getBytes();
+                List<Mutation> mutationList = pair.getValue();
+                
+                //create a span per target table
+                //TODO maybe we can be smarter about the table name to string here?
+                Span child = Tracing.child(span,"Writing mutation batch for table: "+Bytes.toString(htableName));
+
+                int retryCount = 0;
+                boolean shouldRetry = false;
+                do {
+                    TableRef origTableRef = tableInfo.getOrigTableRef();
+                    PTable table = origTableRef.getTable();
+                    table.getIndexMaintainers(indexMetaDataPtr, connection);
+                    final ServerCache cache = tableInfo.isDataTable() ? setMetaDataOnMutations(origTableRef, mutationList, indexMetaDataPtr) : null;
+                    // If we haven't retried yet, retry for this case only, as it's possible that
+                    // a split will occur after we send the index metadata cache to all known
+                    // region servers.
+                    shouldRetry = cache!=null;
+                    SQLException sqlE = null;
+                    HTableInterface hTable = connection.getQueryServices().getTable(htableName);
+                    try {
+                        if (table.isTransactional()) {
+                            // Track tables to which we've sent uncommitted data
+                            txTableRefs.add(origTableRef);
+                            addDMLFence(table);
+                            uncommittedPhysicalNames.add(table.getPhysicalName().getString());
+                            
+                            // If we have indexes, wrap the HTable in a delegate HTable that
+                            // will attach the necessary index meta data in the event of a
+                            // rollback
+                            if (!table.getIndexes().isEmpty()) {
+                                hTable = new MetaDataAwareHTable(hTable, origTableRef);
+                            }
+                            TransactionAwareHTable txnAware = TransactionUtil.getTransactionAwareHTable(hTable, table.isImmutableRows());
+                            // Don't add immutable indexes (those are the only ones that would participate
+                            // during a commit), as we don't need conflict detection for these.
+                            if (tableInfo.isDataTable()) {
+                                // Even for immutable, we need to do this so that an abort has the state
+                                // necessary to generate the rows to delete.
+                                addTransactionParticipant(txnAware);
+                            } else {
+                                txnAware.startTx(getTransaction());
+                            }
+                            hTable = txnAware;
+                        }
+                        
+                        long numMutations = mutationList.size();
+                        GLOBAL_MUTATION_BATCH_SIZE.update(numMutations);
+                        
+                        long startTime = System.currentTimeMillis();
+                        child.addTimelineAnnotation("Attempt " + retryCount);
+                        hTable.batch(mutationList);
+                        if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
+                        child.stop();
+                        child.stop();
+                        shouldRetry = false;
+                        long mutationCommitTime = System.currentTimeMillis() - startTime;
+                        GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
+                        
+                        long mutationSizeBytes = calculateMutationSize(mutationList);
+                        MutationMetric mutationsMetric = new MutationMetric(numMutations, mutationSizeBytes, mutationCommitTime);
+                        mutationMetricQueue.addMetricsForTable(Bytes.toString(htableName), mutationsMetric);
+                        if (tableInfo.isDataTable()) {
+                            numRows -= numMutations;
+                        }
+                        // Remove batches as we process them
+                        mutations.remove(origTableRef);
+                    } catch (Exception e) {
+                        SQLException inferredE = ServerUtil.parseServerExceptionOrNull(e);
+                        if (inferredE != null) {
+                            if (shouldRetry && retryCount == 0 && inferredE.getErrorCode() == SQLExceptionCode.INDEX_METADATA_NOT_FOUND.getErrorCode()) {
+                                // Swallow this exception once, as it's possible that we split after sending the index metadata
+                                // and one of the region servers doesn't have it. This will cause it to have it the next go around.
+                                // If it fails again, we don't retry.
+                                String msg = "Swallowing exception and retrying after clearing meta cache on connection. " + inferredE;
+                                logger.warn(LogUtil.addCustomAnnotations(msg, connection));
+                                connection.getQueryServices().clearTableRegionCache(htableName);
+
+                                // add a new child span as this one failed
+                                child.addTimelineAnnotation(msg);
+                                child.stop();
+                                child = Tracing.child(span,"Failed batch, attempting retry");
+
+                                continue;
+                            }
+                            e = inferredE;
+                        }
+                        // Throw to client an exception that indicates the statements that
+                        // were not committed successfully.
+                        sqlE = new CommitException(e, getUncommittedStatementIndexes());
+                    } finally {
+                        try {
+                            if (cache!=null) 
+                                cache.close();
+                        } finally {
+                            try {
+                                hTable.close();
+                            } 
+                            catch (IOException e) {
+                                if (sqlE != null) {
+                                    sqlE.setNextException(ServerUtil.parseServerException(e));
+                                } else {
+                                    sqlE = ServerUtil.parseServerException(e);
+                                }
+                            } 
+                            if (sqlE != null) {
+                                throw sqlE;
+                            }
+                        }
+                    }
+                } while (shouldRetry && retryCount++ < 1);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index fd6aa12..0ebe67f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -112,8 +112,9 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.SchemaUtil;
-
 import org.apache.tephra.TransactionContext;
+import org.cloudera.htrace.Sampler;
+import org.cloudera.htrace.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
@@ -279,20 +280,21 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
                 long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
                 return (table.getType() != PTableType.SYSTEM && 
                         (  table.getTimeStamp() >= maxTimestamp || 
-                         ! Objects.equal(tenantId, table.getTenantId())) );
+                         (table.getTenantId()!=null && ! Objects.equal(tenantId, table.getTenantId()))));
             }
             
             @Override
             public boolean prune(PFunction function) {
                 long maxTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
                 return ( function.getTimeStamp() >= maxTimestamp ||
-                         ! Objects.equal(tenantId, function.getTenantId()));
+                        (function.getTenantId()!=null && ! Objects.equal(tenantId, function.getTenantId())));
             }
         };
         this.isRequestLevelMetricsEnabled = JDBCUtil.isCollectingRequestLevelMetricsEnabled(url, info, this.services.getProps());
         this.mutationState = mutationState == null ? newMutationState(maxSize) : new MutationState(mutationState);
-        this.metaData = metaData.pruneTables(pruner);
-        this.metaData = metaData.pruneFunctions(pruner);
+        this.metaData = metaData;
+        this.metaData.pruneTables(pruner);
+        this.metaData.pruneFunctions(pruner);
         this.services.addConnection(this);
 
         // setup tracing, if its enabled
@@ -899,79 +901,71 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     }
     
     @Override
-    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
-        metaData = metaData.addTable(table, resolvedTime);
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
+        metaData.addTable(table, resolvedTime);
         //Cascade through to connectionQueryServices too
         getQueryServices().addTable(table, resolvedTime);
-        return metaData;
     }
     
     @Override
-    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
-    	metaData = metaData.updateResolvedTimestamp(table, resolvedTime);
+    public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
+    	metaData.updateResolvedTimestamp(table, resolvedTime);
     	//Cascade through to connectionQueryServices too
         getQueryServices().updateResolvedTimestamp(table, resolvedTime);
-        return metaData;
     }
     
     @Override
-    public PMetaData addFunction(PFunction function) throws SQLException {
+    public void addFunction(PFunction function) throws SQLException {
         // TODO: since a connection is only used by one thread at a time,
         // we could modify this metadata in place since it's not shared.
         if (scn == null || scn > function.getTimeStamp()) {
-            metaData = metaData.addFunction(function);
+            metaData.addFunction(function);
         }
         //Cascade through to connectionQueryServices too
         getQueryServices().addFunction(function);
-        return metaData;
     }
 
     @Override
-    public PMetaData addSchema(PSchema schema) throws SQLException {
-        metaData = metaData.addSchema(schema);
+    public void addSchema(PSchema schema) throws SQLException {
+        metaData.addSchema(schema);
         // Cascade through to connectionQueryServices too
         getQueryServices().addSchema(schema);
-        return metaData;
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
+    public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
             long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls,
             boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime)
                     throws SQLException {
-        metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
+        metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
                 isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped,
                 resolvedTime);
         // Cascade through to connectionQueryServices too
         getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
                 isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped,
                 resolvedTime);
-        return metaData;
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
-        metaData = metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
+    public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
+        metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
         //Cascade through to connectionQueryServices too
         getQueryServices().removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
-        return metaData;
     }
 
     @Override
-    public PMetaData removeFunction(PName tenantId, String functionName, long tableTimeStamp) throws SQLException {
-        metaData = metaData.removeFunction(tenantId, functionName, tableTimeStamp);
+    public void removeFunction(PName tenantId, String functionName, long tableTimeStamp) throws SQLException {
+        metaData.removeFunction(tenantId, functionName, tableTimeStamp);
         //Cascade through to connectionQueryServices too
         getQueryServices().removeFunction(tenantId, functionName, tableTimeStamp);
-        return metaData;
     }
 
     @Override
-    public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
+    public void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
             long tableSeqNum, long resolvedTime) throws SQLException {
-        metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
+        metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
         //Cascade through to connectionQueryServices too
         getQueryServices().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
-        return metaData;
     }
 
     protected boolean removeStatement(PhoenixStatement statement) throws SQLException {
@@ -1071,11 +1065,10 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     }
 
     @Override
-    public PMetaData removeSchema(PSchema schema, long schemaTimeStamp) {
-        metaData = metaData.removeSchema(schema, schemaTimeStamp);
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
+        metaData.removeSchema(schema, schemaTimeStamp);
         // Cascade through to connectionQueryServices too
         getQueryServices().removeSchema(schema, schemaTimeStamp);
-        return metaData;
 
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index d757001..a5dcca8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -31,8 +31,6 @@ import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
 import java.lang.ref.WeakReference;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -84,7 +82,6 @@ import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
-import org.apache.hadoop.hbase.regionserver.LocalIndexSplitter;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -159,8 +156,8 @@ import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PMetaDataImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PSynchronizedMetaData;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
@@ -181,7 +178,6 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.ConfigUtil;
-import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.JDBCUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixContextExecutor;
@@ -287,7 +283,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private PMetaData newEmptyMetaData() {
         long maxSizeBytes = props.getLong(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB,
                 QueryServicesOptions.DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE);
-        return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, maxSizeBytes);
+        return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, maxSizeBytes));
     }
     
     /**
@@ -553,7 +549,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
             try {
                 throwConnectionClosedIfNullMetaData();
@@ -561,26 +557,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If a client opens a connection at an earlier timestamp, this can happen
                 PTable existingTable = latestMetaData.getTableRef(new PTableKey(table.getTenantId(), table.getName().getString())).getTable();
                 if (existingTable.getTimeStamp() >= table.getTimeStamp()) {
-                    return latestMetaData;
+                    return;
                 }
             } catch (TableNotFoundException e) {}
-            latestMetaData = latestMetaData.addTable(table, resolvedTime);
+            latestMetaData.addTable(table, resolvedTime);
             latestMetaDataLock.notifyAll();
-            return latestMetaData;
         }
     }
     
-    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
-    	synchronized (latestMetaDataLock) {
+    @Override
+    public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
+        synchronized (latestMetaDataLock) {
             throwConnectionClosedIfNullMetaData();
-            latestMetaData = latestMetaData.updateResolvedTimestamp(table, resolvedTime);
+            latestMetaData.updateResolvedTimestamp(table, resolvedTime);
             latestMetaDataLock.notifyAll();
-            return latestMetaData;
         }
     }
 
     private static interface Mutator {
-        PMetaData mutate(PMetaData metaData) throws SQLException;
+        void mutate(PMetaData metaData) throws SQLException;
     }
 
     /**
@@ -603,7 +598,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                          */
                         if (table.getSequenceNumber() + 1 == tableSeqNum) {
                             // TODO: assert that timeStamp is bigger that table timeStamp?
-                            metaData = mutator.mutate(metaData);
+                            mutator.mutate(metaData);
                             break;
                         } else if (table.getSequenceNumber() >= tableSeqNum) {
                             logger.warn("Attempt to cache older version of " + tableName + ": current= " + table.getSequenceNumber() + ", new=" + tableSeqNum);
@@ -618,7 +613,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         logger.warn("Unable to update meta data repo within " + (DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS/1000) + " seconds for " + tableName);
                         // There will never be a parentTableName here, as that would only
                         // be non null for an index an we never add/remove columns from an index.
-                        metaData = metaData.removeTable(tenantId, tableName, null, HConstants.LATEST_TIMESTAMP);
+                        metaData.removeTable(tenantId, tableName, null, HConstants.LATEST_TIMESTAMP);
                         break;
                     }
                     latestMetaDataLock.wait(waitTime);
@@ -636,46 +631,43 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      }
 
 	@Override
-    public PMetaData addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
+    public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    return metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
-            public PMetaData mutate(PMetaData metaData) throws SQLException {
+            public void mutate(PMetaData metaData) throws SQLException {
                 try {
-                    return metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum,
+                    metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum,
                             isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional,
                             updateCacheFrequency, isNamespaceMapped, resolvedTime);
                 } catch (TableNotFoundException e) {
                     // The DROP TABLE may have been processed first, so just ignore.
-                    return metaData;
                 }
             }
         });
      }
 
     @Override
-    public PMetaData removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
+    public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
         synchronized (latestMetaDataLock) {
             throwConnectionClosedIfNullMetaData();
-            latestMetaData = latestMetaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
+            latestMetaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
             latestMetaDataLock.notifyAll();
-            return latestMetaData;
         }
     }
 
     @Override
-    public PMetaData removeColumn(final PName tenantId, final String tableName, final List<PColumn> columnsToRemove, final long tableTimeStamp, final long tableSeqNum, final long resolvedTime) throws SQLException {
-        return metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+    public void removeColumn(final PName tenantId, final String tableName, final List<PColumn> columnsToRemove, final long tableTimeStamp, final long tableSeqNum, final long resolvedTime) throws SQLException {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
-            public PMetaData mutate(PMetaData metaData) throws SQLException {
+            public void mutate(PMetaData metaData) throws SQLException {
                 try {
-                    return metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
+                    metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
                 } catch (TableNotFoundException e) {
                     // The DROP TABLE may have been processed first, so just ignore.
-                    return metaData;
                 }
             }
         });
@@ -686,10 +678,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public PhoenixConnection connect(String url, Properties info) throws SQLException {
         checkClosed();
         PMetaData metadata = latestMetaData;
-        if (metadata == null) {
-            throwConnectionClosedException();
-        }
-
+        throwConnectionClosedIfNullMetaData();
+        metadata = metadata.clone();
         return new PhoenixConnection(this, url, info, metadata);
     }
 
@@ -1647,9 +1637,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table;
         try {
             PMetaData metadata = latestMetaData;
-            if (metadata == null) {
-                throwConnectionClosedException();
-            }
+            throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
                                                      // the case
@@ -2168,14 +2156,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             .build().buildException();
         }
     }
-    
+
     private HashSet<String> existingColumnFamiliesForBaseTable(PName baseTableName) throws TableNotFoundException {
-        synchronized (latestMetaDataLock) {
-            throwConnectionClosedIfNullMetaData();
-            PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
-            latestMetaDataLock.notifyAll();
-            return existingColumnFamilies(table);
-        }
+        throwConnectionClosedIfNullMetaData();
+        PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
+        return existingColumnFamilies(table);
     }
     
     private HashSet<String> existingColumnFamilies(PTable table) {
@@ -3364,7 +3349,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public PMetaData addFunction(PFunction function) throws SQLException {
+    public void addFunction(PFunction function) throws SQLException {
         synchronized (latestMetaDataLock) {
             try {
                 throwConnectionClosedIfNullMetaData();
@@ -3372,23 +3357,21 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If a client opens a connection at an earlier timestamp, this can happen
                 PFunction existingFunction = latestMetaData.getFunction(new PTableKey(function.getTenantId(), function.getFunctionName()));
                 if (existingFunction.getTimeStamp() >= function.getTimeStamp()) {
-                    return latestMetaData;
+                    return;
                 }
             } catch (FunctionNotFoundException e) {}
-            latestMetaData = latestMetaData.addFunction(function);
+            latestMetaData.addFunction(function);
             latestMetaDataLock.notifyAll();
-            return latestMetaData;
         }
     }
 
     @Override
-    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+    public void removeFunction(PName tenantId, String function, long functionTimeStamp)
             throws SQLException {
         synchronized (latestMetaDataLock) {
             throwConnectionClosedIfNullMetaData();
-            latestMetaData = latestMetaData.removeFunction(tenantId, function, functionTimeStamp);
+            latestMetaData.removeFunction(tenantId, function, functionTimeStamp);
             latestMetaDataLock.notifyAll();
-            return latestMetaData;
         }
     }
 
@@ -3644,13 +3627,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public PMetaData addSchema(PSchema schema) throws SQLException {
-        return latestMetaData = latestMetaData.addSchema(schema);
+    public void addSchema(PSchema schema) throws SQLException {
+        latestMetaData.addSchema(schema);
     }
 
     @Override
-    public PMetaData removeSchema(PSchema schema, long schemaTimeStamp) {
-        return latestMetaData = latestMetaData.removeSchema(schema, schemaTimeStamp);
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
+        latestMetaData.removeSchema(schema, schemaTimeStamp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index f373de2..25aca74 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -24,8 +24,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Map.Entry;
+import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
 
@@ -86,14 +86,13 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SequenceUtil;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
 import org.apache.tephra.TransactionManager;
 import org.apache.tephra.TransactionSystemClient;
 import org.apache.tephra.inmemory.InMemoryTxSystemClient;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 
 /**
  *
@@ -172,41 +171,41 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
-        return metaData = metaData.addTable(table, resolvedTime);
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
+        metaData.addTable(table, resolvedTime);
     }
     
     @Override
-    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
-        return metaData = metaData.updateResolvedTimestamp(table, resolvedTimestamp);
+    public void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+        metaData.updateResolvedTimestamp(table, resolvedTimestamp);
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
+    public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
             long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls,
             boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime)
                     throws SQLException {
-        return metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
+        metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
                 isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped,
                 resolvedTime);
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
+    public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
             throws SQLException {
-        return metaData = metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
+        metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
     }
 
     @Override
-    public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
+    public void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
             long tableSeqNum, long resolvedTime) throws SQLException {
-        return metaData = metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
+        metaData.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
     }
 
     
     @Override
     public PhoenixConnection connect(String url, Properties info) throws SQLException {
-        return new PhoenixConnection(this, url, info, metaData);
+        return new PhoenixConnection(this, url, info, metaData.clone());
     }
 
     @Override
@@ -549,14 +548,14 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addFunction(PFunction function) throws SQLException {
-        return metaData = this.metaData.addFunction(function);
+    public void addFunction(PFunction function) throws SQLException {
+        this.metaData.addFunction(function);
     }
 
     @Override
-    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+    public void removeFunction(PName tenantId, String function, long functionTimeStamp)
             throws SQLException {
-        return metaData = this.metaData.removeFunction(tenantId, function, functionTimeStamp);
+        this.metaData.removeFunction(tenantId, function, functionTimeStamp);
     }
 
     @Override
@@ -615,8 +614,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addSchema(PSchema schema) throws SQLException {
-        return metaData = this.metaData.addSchema(schema);
+    public void addSchema(PSchema schema) throws SQLException {
+        this.metaData.addSchema(schema);
     }
 
     @Override
@@ -629,8 +628,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData removeSchema(PSchema schema, long schemaTimeStamp) {
-        return metaData = metaData.removeSchema(schema, schemaTimeStamp);
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
+        metaData.removeSchema(schema, schemaTimeStamp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 953c73d..99ad59c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -39,7 +39,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
@@ -47,7 +46,6 @@ import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.stats.PTableStats;
-
 import org.apache.tephra.TransactionSystemClient;
 
 
@@ -78,35 +76,35 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
-        return getDelegate().addTable(table, resolvedTime);
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
+        getDelegate().addTable(table, resolvedTime);
     }
     
     @Override
-    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
-        return getDelegate().updateResolvedTimestamp(table, resolvedTimestamp);
+    public void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+        getDelegate().updateResolvedTimestamp(table, resolvedTimestamp);
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
+    public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
             long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls,
             boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime)
                     throws SQLException {
-        return getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
+        getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows,
                 isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped,
                 resolvedTime);
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
+    public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp)
             throws SQLException {
-        return getDelegate().removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
+        getDelegate().removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
     }
 
     @Override
-    public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
+    public void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp,
             long tableSeqNum, long resolvedTime) throws SQLException {
-        return getDelegate().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
+        getDelegate().removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum, resolvedTime);
     }
 
     @Override
@@ -279,14 +277,14 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addFunction(PFunction function) throws SQLException {
-        return getDelegate().addFunction(function);
+    public void addFunction(PFunction function) throws SQLException {
+        getDelegate().addFunction(function);
     }
 
     @Override
-    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+    public void removeFunction(PName tenantId, String function, long functionTimeStamp)
             throws SQLException {
-        return getDelegate().removeFunction(tenantId, function, functionTimeStamp);
+        getDelegate().removeFunction(tenantId, function, functionTimeStamp);
     }
 
     @Override
@@ -319,8 +317,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData addSchema(PSchema schema) throws SQLException {
-        return getDelegate().addSchema(schema);
+    public void addSchema(PSchema schema) throws SQLException {
+        getDelegate().addSchema(schema);
     }
 
     @Override
@@ -334,8 +332,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public PMetaData removeSchema(PSchema schema, long schemaTimeStamp) {
-        return getDelegate().removeSchema(schema, schemaTimeStamp);
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
+        getDelegate().removeSchema(schema, schemaTimeStamp);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
index f532dc8..0b6a644 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -23,7 +23,6 @@ import java.util.List;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 
@@ -36,13 +35,13 @@ import org.apache.phoenix.schema.PTable;
  * @since 0.1
  */
 public interface MetaDataMutated {
-    PMetaData addTable(PTable table, long resolvedTime) throws SQLException;
-    PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException;
-    PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
-    PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) throws SQLException;
-    PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException;
-    PMetaData addFunction(PFunction function) throws SQLException;
-    PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp) throws SQLException;
-    PMetaData addSchema(PSchema schema) throws SQLException;
-    PMetaData removeSchema(PSchema schema, long schemaTimeStamp);
+    void addTable(PTable table, long resolvedTime) throws SQLException;
+    void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException;
+    void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
+    void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) throws SQLException;
+    void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException;
+    void addFunction(PFunction function) throws SQLException;
+    void removeFunction(PName tenantId, String function, long functionTimeStamp) throws SQLException;
+    void addSchema(PSchema schema) throws SQLException;
+    void removeSchema(PSchema schema, long schemaTimeStamp);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
index 6a710eb..cfeb13f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
@@ -30,9 +30,9 @@ public interface PMetaData extends MetaDataMutated, Iterable<PTable>, Cloneable
     public int size();
     public PMetaData clone();
     public PTableRef getTableRef(PTableKey key) throws TableNotFoundException;
-    public PMetaData pruneTables(Pruner pruner);
+    public void pruneTables(Pruner pruner);
     public PFunction getFunction(PTableKey key) throws FunctionNotFoundException;
-    public PMetaData pruneFunctions(Pruner pruner);
+    public void pruneFunctions(Pruner pruner);
     public long getAge(PTableRef ref);
     public PSchema getSchema(PTableKey key) throws SchemaNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 67a2714..5ffacca 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -28,7 +28,10 @@ import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TimeKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.MinMaxPriorityQueue;
@@ -43,7 +46,8 @@ import com.google.common.primitives.Longs;
  *
  */
 public class PMetaDataImpl implements PMetaData {
-        private static class PMetaDataCache implements Cloneable {
+    private static final Logger logger = LoggerFactory.getLogger(PMetaDataImpl.class);
+        static class PMetaDataCache implements Cloneable {
             private static final int MIN_REMOVAL_SIZE = 3;
             private static final Comparator<PTableRef> COMPARATOR = new Comparator<PTableRef>() {
                 @Override
@@ -239,7 +243,12 @@ public class PMetaDataImpl implements PMetaData {
             }
         }
             
-    private final PMetaDataCache metaData;
+    private PMetaDataCache metaData;
+    
+    @VisibleForTesting
+    public PMetaDataCache getMetaData() {
+        return metaData;
+    }
     
     public PMetaDataImpl(int initialCapacity, long maxByteSize) {
         this.metaData = new PMetaDataCache(initialCapacity, maxByteSize, TimeKeeper.SYSTEM);
@@ -250,12 +259,12 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     private PMetaDataImpl(PMetaDataCache metaData) {
-        this.metaData = metaData.clone();
+        this.metaData = metaData;
     }
     
     @Override
     public PMetaDataImpl clone() {
-        return new PMetaDataImpl(this.metaData);
+        return new PMetaDataImpl(new PMetaDataCache(this.metaData));
     }
     
     @Override
@@ -282,14 +291,12 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
-    	PMetaDataCache clone = metaData.clone();
-    	clone.putDuplicate(table.getKey(), table, resolvedTimestamp);
-    	return new PMetaDataImpl(clone);
+    public void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+    	metaData.putDuplicate(table.getKey(), table, resolvedTimestamp);
     }
 
     @Override
-    public PMetaData addTable(PTable table, long resolvedTime) throws SQLException {
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
         int netGain = 0;
         PTableKey key = table.getKey();
         PTableRef oldTableRef = metaData.get(key);
@@ -323,28 +330,27 @@ public class PMetaDataImpl implements PMetaData {
             netGain += table.getEstimatedSize();
         }
         long overage = metaData.getCurrentSize() + netGain - metaData.getMaxSize();
-        PMetaDataCache newMetaData = overage <= 0 ? metaData.clone() : metaData.cloneMinusOverage(overage);
+        metaData = overage <= 0 ? metaData : metaData.cloneMinusOverage(overage);
         
         if (newParentTable != null) { // Upsert new index table into parent data table list
-            newMetaData.put(newParentTable.getKey(), newParentTable, parentResolvedTimestamp);
-            newMetaData.putDuplicate(table.getKey(), table, resolvedTime);
+            metaData.put(newParentTable.getKey(), newParentTable, parentResolvedTimestamp);
+            metaData.putDuplicate(table.getKey(), table, resolvedTime);
         } else {
-            newMetaData.put(table.getKey(), table, resolvedTime);
+            metaData.put(table.getKey(), table, resolvedTime);
         }
         for (PTable index : table.getIndexes()) {
-            newMetaData.putDuplicate(index.getKey(), index, resolvedTime);
+            metaData.putDuplicate(index.getKey(), index, resolvedTime);
         }
-        return new PMetaDataImpl(newMetaData);
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columnsToAdd, long tableTimeStamp,
+    public void addColumn(PName tenantId, String tableName, List<PColumn> columnsToAdd, long tableTimeStamp,
             long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls,
             boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime)
                     throws SQLException {
         PTableRef oldTableRef = metaData.get(new PTableKey(tenantId, tableName));
         if (oldTableRef == null) {
-            return this;
+            return;
         }
         List<PColumn> oldColumns = PTableImpl.getColumnsToClone(oldTableRef.getTable());
         List<PColumn> newColumns;
@@ -358,12 +364,11 @@ public class PMetaDataImpl implements PMetaData {
         PTable newTable = PTableImpl.makePTable(oldTableRef.getTable(), tableTimeStamp, tableSeqNum, newColumns,
                 isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency,
                 isNamespaceMapped);
-        return addTable(newTable, resolvedTime);
+        addTable(newTable, resolvedTime);
     }
 
     @Override
-    public PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
-        PMetaDataCache tables = null;
+    public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
         PTableRef parentTableRef = null;
         PTableKey key = new PTableKey(tenantId, tableName);
         if (metaData.get(key) == null) {
@@ -371,16 +376,15 @@ public class PMetaDataImpl implements PMetaData {
                 parentTableRef = metaData.get(new PTableKey(tenantId, parentTableName));
             }
             if (parentTableRef == null) {
-                return this;
+                return;
             }
         } else {
-            tables = metaData.clone();
-            PTable table = tables.remove(key);
+            PTable table = metaData.remove(key);
             for (PTable index : table.getIndexes()) {
-                tables.remove(index.getKey());
+                metaData.remove(index.getKey());
             }
             if (table.getParentName() != null) {
-                parentTableRef = tables.get(new PTableKey(tenantId, table.getParentName().getString()));
+                parentTableRef = metaData.get(new PTableKey(tenantId, table.getParentName().getString()));
             }
         }
         // also remove its reference from parent table
@@ -397,26 +401,22 @@ public class PMetaDataImpl implements PMetaData {
                                 parentTableRef.getTable(),
                                 tableTimeStamp == HConstants.LATEST_TIMESTAMP ? parentTableRef.getTable().getTimeStamp() : tableTimeStamp,
                                 newIndexes);
-                        if (tables == null) { 
-                            tables = metaData.clone();
-                        }
-                        tables.put(parentTable.getKey(), parentTable, parentTableRef.getResolvedTimeStamp());
+                        metaData.put(parentTable.getKey(), parentTable, parentTableRef.getResolvedTimeStamp());
                         break;
                     }
                 }
             }
         }
-        return tables == null ? this : new PMetaDataImpl(tables);
     }
     
     @Override
-    public PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException {
+    public void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException {
         PTableRef tableRef = metaData.get(new PTableKey(tenantId, tableName));
         if (tableRef == null) {
-            return this;
+            return;
         }
         PTable table = tableRef.getTable();
-        PMetaDataCache tables = metaData.clone();
+        PMetaDataCache tables = metaData;
         for (PColumn columnToRemove : columnsToRemove) {
             PColumn column;
             String familyName = columnToRemove.getFamilyName().getString();
@@ -445,25 +445,21 @@ public class PMetaDataImpl implements PMetaData {
             table = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, columns);
         }
         tables.put(table.getKey(), table, resolvedTime);
-        return new PMetaDataImpl(tables);
     }
 
     @Override
-    public PMetaData pruneTables(Pruner pruner) {
+    public void pruneTables(Pruner pruner) {
         List<PTableKey> keysToPrune = Lists.newArrayListWithExpectedSize(this.size());
         for (PTable table : this) {
             if (pruner.prune(table)) {
                 keysToPrune.add(table.getKey());
             }
         }
-        if (keysToPrune.isEmpty()) {
-            return this;
-        }
-        PMetaDataCache tables = metaData.clone();
-        for (PTableKey key : keysToPrune) {
-            tables.remove(key);
+        if (!keysToPrune.isEmpty()) {
+            for (PTableKey key : keysToPrune) {
+                metaData.remove(key);
+            }
         }
-        return new PMetaDataImpl(tables);
     }
 
     @Override
@@ -472,35 +468,29 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData addFunction(PFunction function) throws SQLException {
+    public void addFunction(PFunction function) throws SQLException {
         this.metaData.functions.put(function.getKey(), function);
-        return this;
     }
 
     @Override
-    public PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp)
+    public void removeFunction(PName tenantId, String function, long functionTimeStamp)
             throws SQLException {
         this.metaData.functions.remove(new PTableKey(tenantId, function));
-        return this;
     }
 
     @Override
-    public PMetaData pruneFunctions(Pruner pruner) {
+    public void pruneFunctions(Pruner pruner) {
         List<PTableKey> keysToPrune = Lists.newArrayListWithExpectedSize(this.size());
         for (PFunction function : this.metaData.functions.values()) {
             if (pruner.prune(function)) {
                 keysToPrune.add(function.getKey());
             }
         }
-        if (keysToPrune.isEmpty()) {
-            return this;
-        }
-        PMetaDataCache clone = metaData.clone();
-        for (PTableKey key : keysToPrune) {
-            clone.functions.remove(key);
+        if (!keysToPrune.isEmpty()) {
+            for (PTableKey key : keysToPrune) {
+                metaData.functions.remove(key);
+            }
         }
-        return new PMetaDataImpl(clone);
-    
     }
 
     @Override
@@ -509,9 +499,8 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData addSchema(PSchema schema) throws SQLException {
+    public void addSchema(PSchema schema) throws SQLException {
         this.metaData.schemas.put(schema.getSchemaKey(), schema);
-        return this;
     }
 
     @Override
@@ -522,8 +511,8 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData removeSchema(PSchema schema, long schemaTimeStamp) {
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
         this.metaData.schemas.remove(SchemaUtil.getSchemaKey(schema.getSchemaName()));
-        return this;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java
new file mode 100644
index 0000000..af4bc60
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java
@@ -0,0 +1,249 @@
+/*
+ * 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.schema;
+
+import java.sql.SQLException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import javax.annotation.concurrent.GuardedBy;
+
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PSchema;
+
+public class PSynchronizedMetaData implements PMetaData {
+
+    @GuardedBy("readWriteLock")
+    private PMetaData delegate;
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    public PSynchronizedMetaData(PMetaData metadata) {
+        this.delegate = metadata;
+    }
+    
+    @Override
+    public Iterator<PTable> iterator() {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.iterator();
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public int size() {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.size();
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public PMetaData clone() {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.clone();
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void addTable(PTable table, long resolvedTime) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.addTable(table, resolvedTime);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public PTableRef getTableRef(PTableKey key) throws TableNotFoundException {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.getTableRef(key);
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.updateResolvedTimestamp(table, resolvedTimestamp);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void pruneTables(Pruner pruner) {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.pruneTables(pruner);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public PFunction getFunction(PTableKey key) throws FunctionNotFoundException {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.getFunction(key);
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void removeTable(PName tenantId, String tableName, String parentTableName,
+            long tableTimeStamp) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.removeTable(tenantId, tableName, parentTableName, tableTimeStamp);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void pruneFunctions(Pruner pruner) {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.pruneFunctions(pruner);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public long getAge(PTableRef ref) {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.getAge(ref);
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void addColumn(PName tenantId, String tableName, List<PColumn> columns,
+            long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled,
+            boolean isMultitenant, boolean storeNulls, boolean isTransactional,
+            long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime)
+            throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum,
+                isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional,
+                updateCacheFrequency, isNamespaceMapped, resolvedTime);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public PSchema getSchema(PTableKey key) throws SchemaNotFoundException {
+        readWriteLock.readLock().lock();
+        try {
+            return delegate.getSchema(key);
+        }
+        finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove,
+            long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.removeColumn(tenantId, tableName, columnsToRemove, tableTimeStamp, tableSeqNum,
+                resolvedTime);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void addFunction(PFunction function) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.addFunction(function);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void removeFunction(PName tenantId, String function, long functionTimeStamp)
+            throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.removeFunction(tenantId, function, functionTimeStamp);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void addSchema(PSchema schema) throws SQLException {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.addSchema(schema);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    @Override
+    public void removeSchema(PSchema schema, long schemaTimeStamp) {
+        readWriteLock.writeLock().lock();
+        try {
+            delegate.removeSchema(schema, schemaTimeStamp);
+        }
+        finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+}


[16/50] [abbrv] phoenix git commit: PHOENIX-3164 Cache UGI instances for remote users in PQS

Posted by ma...@apache.org.
PHOENIX-3164 Cache UGI instances for remote users in PQS

equals(Object) and hashCode() on UGI are implemented
via reference checks (rather than the values themselves).
This creates a situation where new PhoenixConnections are
opened for the same user via PQS.


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

Branch: refs/heads/calcite
Commit: 44cf0827f1e5577510ee95f149df8d552a56da3c
Parents: 2afb16d
Author: Josh Elser <el...@apache.org>
Authored: Tue Aug 9 14:52:20 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Aug 15 18:31:48 2016 -0400

----------------------------------------------------------------------
 .../org/apache/phoenix/query/QueryServices.java |  3 +
 .../phoenix/query/QueryServicesOptions.java     |  4 +
 .../apache/phoenix/queryserver/server/Main.java | 61 ++++++++++++--
 .../server/PhoenixDoAsCallbackTest.java         | 89 ++++++++++++++++++++
 4 files changed, 149 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/44cf0827/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 2ad2d43..22fa45a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -206,6 +206,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String QUERY_SERVER_DNS_NAMESERVER_ATTRIB = "phoenix.queryserver.dns.nameserver";
     public static final String QUERY_SERVER_DNS_INTERFACE_ATTRIB = "phoenix.queryserver.dns.interface";
     public static final String QUERY_SERVER_HBASE_SECURITY_CONF_ATTRIB = "hbase.security.authentication";
+    public static final String QUERY_SERVER_UGI_CACHE_MAX_SIZE = "phoenix.queryserver.ugi.cache.max.size";
+    public static final String QUERY_SERVER_UGI_CACHE_INITIAL_SIZE = "phoenix.queryserver.ugi.cache.initial.size";
+    public static final String QUERY_SERVER_UGI_CACHE_CONCURRENCY = "phoenix.queryserver.ugi.cache.concurrency";
     
     public static final String RENEW_LEASE_ENABLED = "phoenix.scanner.lease.renew.enabled";
     public static final String RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS = "phoenix.scanner.lease.renew.interval";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/44cf0827/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index b7ea82e..83347c8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -241,6 +241,10 @@ public class QueryServicesOptions {
     // doesn't depend on phoenix-core.
     public static final String DEFAULT_QUERY_SERVER_SERIALIZATION = "PROTOBUF";
     public static final int DEFAULT_QUERY_SERVER_HTTP_PORT = 8765;
+    public static final long DEFAULT_QUERY_SERVER_UGI_CACHE_MAX_SIZE = 1000L;
+    public static final int DEFAULT_QUERY_SERVER_UGI_CACHE_INITIAL_SIZE = 100;
+    public static final int DEFAULT_QUERY_SERVER_UGI_CACHE_CONCURRENCY = 10;
+
     public static final boolean DEFAULT_RENEW_LEASE_ENABLED = true;
     public static final int DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS =
             DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD / 2;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/44cf0827/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/Main.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/Main.java b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/Main.java
index fc2ee34..4b3ca7e 100644
--- a/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/Main.java
+++ b/phoenix-queryserver/src/main/java/org/apache/phoenix/queryserver/server/Main.java
@@ -18,14 +18,15 @@
 package org.apache.phoenix.queryserver.server;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.remote.Driver;
 import org.apache.calcite.avatica.remote.LocalService;
 import org.apache.calcite.avatica.remote.Service;
-import org.apache.calcite.avatica.server.AvaticaHandler;
-import org.apache.calcite.avatica.server.AvaticaServerConfiguration;
 import org.apache.calcite.avatica.server.DoAsRemoteUserCallback;
-import org.apache.calcite.avatica.server.HandlerFactory;
 import org.apache.calcite.avatica.server.HttpServer;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,7 +44,6 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 
 import java.io.File;
-import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.RuntimeMXBean;
 import java.security.PrivilegedExceptionAction;
@@ -54,6 +54,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -210,7 +211,7 @@ public final class Main extends Configured implements Tool, Runnable {
         // Enable SPNEGO and impersonation (through standard Hadoop configuration means)
         builder.withSpnego(ugi.getUserName())
             .withAutomaticLogin(keytab)
-            .withImpersonation(new PhoenixDoAsCallback(ugi));
+            .withImpersonation(new PhoenixDoAsCallback(ugi, getConf()));
       }
 
       // Build and start the HttpServer
@@ -261,15 +262,29 @@ public final class Main extends Configured implements Tool, Runnable {
    */
   static class PhoenixDoAsCallback implements DoAsRemoteUserCallback {
     private final UserGroupInformation serverUgi;
+    private final LoadingCache<String,UserGroupInformation> ugiCache;
 
-    public PhoenixDoAsCallback(UserGroupInformation serverUgi) {
+    public PhoenixDoAsCallback(UserGroupInformation serverUgi, Configuration conf) {
       this.serverUgi = Objects.requireNonNull(serverUgi);
+      this.ugiCache = CacheBuilder.newBuilder()
+          .initialCapacity(conf.getInt(QueryServices.QUERY_SERVER_UGI_CACHE_INITIAL_SIZE,
+                  QueryServicesOptions.DEFAULT_QUERY_SERVER_UGI_CACHE_INITIAL_SIZE))
+          .concurrencyLevel(conf.getInt(QueryServices.QUERY_SERVER_UGI_CACHE_CONCURRENCY,
+                  QueryServicesOptions.DEFAULT_QUERY_SERVER_UGI_CACHE_CONCURRENCY))
+          .maximumSize(conf.getLong(QueryServices.QUERY_SERVER_UGI_CACHE_MAX_SIZE,
+                  QueryServicesOptions.DEFAULT_QUERY_SERVER_UGI_CACHE_MAX_SIZE))
+          .build(new UgiCacheLoader(this.serverUgi));
     }
 
     @Override
     public <T> T doAsRemoteUser(String remoteUserName, String remoteAddress, final Callable<T> action) throws Exception {
-      // Proxy this user on top of the server's user (the real user)
-      UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(remoteUserName, serverUgi);
+      // We are guaranteed by Avatica that the `remoteUserName` is properly authenticated by the
+      // time this method is called. We don't have to verify the wire credentials, we can assume the
+      // user provided valid credentials for who it claimed it was.
+
+      // Proxy this user on top of the server's user (the real user). Get a cached instance, the
+      // LoadingCache will create a new instance for us if one isn't cached.
+      UserGroupInformation proxyUser = createProxyUser(remoteUserName);
 
       // Check if this user is allowed to be impersonated.
       // Will throw AuthorizationException if the impersonation as this user is not allowed
@@ -283,6 +298,36 @@ public final class Main extends Configured implements Tool, Runnable {
         }
       });
     }
+
+      @VisibleForTesting
+      UserGroupInformation createProxyUser(String remoteUserName) throws ExecutionException {
+          // PHOENIX-3164 UGI's hashCode and equals methods rely on reference checks, not
+          // value-based checks. We need to make sure we return the same UGI instance for a remote
+          // user, otherwise downstream code in Phoenix and HBase may not treat two of the same
+          // calls from one user as equivalent.
+          return ugiCache.get(remoteUserName);
+      }
+
+      @VisibleForTesting
+      LoadingCache<String,UserGroupInformation> getCache() {
+          return ugiCache;
+      }
+  }
+
+  /**
+   * CacheLoader implementation which creates a "proxy" UGI instance for the given user name.
+   */
+  static class UgiCacheLoader extends CacheLoader<String,UserGroupInformation> {
+      private final UserGroupInformation serverUgi;
+
+      public UgiCacheLoader(UserGroupInformation serverUgi) {
+          this.serverUgi = Objects.requireNonNull(serverUgi);
+      }
+
+      @Override
+      public UserGroupInformation load(String remoteUserName) throws Exception {
+          return UserGroupInformation.createProxyUser(remoteUserName, serverUgi);
+      }
   }
 
   public static void main(String[] argv) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/44cf0827/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/PhoenixDoAsCallbackTest.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/PhoenixDoAsCallbackTest.java b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/PhoenixDoAsCallbackTest.java
new file mode 100644
index 0000000..000baec
--- /dev/null
+++ b/phoenix-queryserver/src/test/java/org/apache/phoenix/queryserver/server/PhoenixDoAsCallbackTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.queryserver.server;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.phoenix.queryserver.server.Main.PhoenixDoAsCallback;
+import org.junit.Test;
+
+/**
+ * Tests for the authorization callback hook Avatica provides for Phoenix to implement.
+ */
+public class PhoenixDoAsCallbackTest {
+
+    @Test
+    public void ugiInstancesAreCached() throws Exception {
+        Configuration conf = new Configuration(false);
+        UserGroupInformation serverUgi = UserGroupInformation.createUserForTesting("server", new String[0]);
+        PhoenixDoAsCallback callback = new PhoenixDoAsCallback(serverUgi, conf);
+
+        UserGroupInformation ugi1 = callback.createProxyUser("user1");
+        assertEquals(1, callback.getCache().size());
+        assertTrue(ugi1.getRealUser() == serverUgi);
+        UserGroupInformation ugi2 = callback.createProxyUser("user2");
+        assertEquals(2, callback.getCache().size());
+        assertTrue(ugi2.getRealUser() == serverUgi);
+
+        UserGroupInformation ugi1Reference = callback.createProxyUser("user1");
+        assertTrue(ugi1 == ugi1Reference);
+        assertEquals(2, callback.getCache().size());
+    }
+
+    @Test
+    public void proxyingUsersAreCached() throws Exception {
+      Configuration conf = new Configuration(false);
+      // The user "server" can impersonate anyone
+      conf.set("hadoop.proxyuser.server.groups", "*");
+      conf.set("hadoop.proxyuser.server.hosts", "*");
+      // Trigger ProxyUsers to refresh itself with the above configuration
+      ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+      UserGroupInformation serverUgi = UserGroupInformation.createUserForTesting("server", new String[0]);
+      PhoenixDoAsCallback callback = new PhoenixDoAsCallback(serverUgi, conf);
+
+      UserGroupInformation user1 = callback.doAsRemoteUser("user1", "localhost:1234", new Callable<UserGroupInformation>() {
+          public UserGroupInformation call() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+      });
+
+      UserGroupInformation user2 = callback.doAsRemoteUser("user2", "localhost:1235", new Callable<UserGroupInformation>() {
+          public UserGroupInformation call() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+      });
+
+      UserGroupInformation user1Reference = callback.doAsRemoteUser("user1", "localhost:1234", new Callable<UserGroupInformation>() {
+          public UserGroupInformation call() throws Exception {
+            return UserGroupInformation.getCurrentUser();
+          }
+      });
+
+      // The UserGroupInformation.getCurrentUser() actually returns a new UGI instance, but the internal
+      // subject is the same. We can verify things will work as expected that way.
+      assertNotEquals(user1.hashCode(), user2.hashCode());
+      assertEquals("These should be the same (cached) instance", user1.hashCode(), user1Reference.hashCode());
+      assertEquals("These should be the same (cached) instance", user1, user1Reference);
+    }
+}


[31/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index b7537a6..8283b28 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -43,7 +43,6 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
@@ -51,7 +50,8 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
+import org.apache.phoenix.end2end.BaseOwnClusterIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -78,16 +78,13 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class IndexIT extends BaseHBaseManagedTimeIT {
+public class IndexIT extends BaseOwnClusterIT {
 	
 	private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
 	private final String tableDDLOptions;
-	private final String tableName;
-    private final String indexName;
-    private final String fullTableName;
-    private final String fullIndexName;
+
 	
 	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
 		this.localIndex = localIndex;
@@ -102,14 +99,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 			optionBuilder.append(" TRANSACTIONAL=true ");
 		}
 		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
-        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
-        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
 	@BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
@@ -127,6 +120,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	@Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
@@ -183,6 +180,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
@@ -237,6 +238,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
         if (transactional) {
             testCreateIndexAfterUpsertStarted(true, fullTableName + "2", fullIndexName + "2");
@@ -334,6 +339,11 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -384,6 +394,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testGroupByCount() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
@@ -402,6 +415,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
@@ -425,6 +441,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
@@ -447,6 +466,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -485,6 +508,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -541,6 +568,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -616,6 +647,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectCF() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -673,6 +708,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -733,7 +772,12 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	String testTable = fullTableName+"_MULTIPLE_UPDATES";
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+
+        String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -741,8 +785,8 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        // make sure that the tables are empty, but reachable
 	        conn.createStatement().execute(
 	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
-	        		  + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
+	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
+	              + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
 	        query = "SELECT * FROM " + testTable;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertFalse(rs.next());
@@ -821,6 +865,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -932,6 +980,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -950,6 +1001,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testIndexWithDecimalCol() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index a48cc4b..8cce14a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -17,10 +17,7 @@
  */
 package org.apache.phoenix.end2end.index;
 
-import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
-import static org.apache.phoenix.util.TestUtil.INDEX_DATA_TABLE;
-import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -36,8 +33,9 @@ import java.sql.SQLException;
 import java.sql.Types;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -56,7 +54,7 @@ import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
 
-public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
+public class IndexMetadataIT extends BaseHBaseManagedTimeTableReuseIT {
 
 	private enum Order {ASC, DESC};
 	
@@ -122,167 +120,169 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String indexDataTable = generateRandomString();
+        String indexName = generateRandomString();
         try {
-            ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
-            String ddl = "CREATE INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE
+            ensureTableCreated(getUrl(), indexDataTable, MUTABLE_INDEX_DATA_TABLE);
+            String ddl = "CREATE INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
                     + " (varchar_col1 ASC, varchar_col2 ASC, int_pk DESC)"
                     + " INCLUDE (int_col1, int_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
             // Verify the metadata for index is correct.
-            ResultSet rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, false, false);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 1, "A:VARCHAR_COL1", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 2, "B:VARCHAR_COL2", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 3, ":INT_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 4, ":VARCHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 5, ":CHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 6, ":LONG_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 7, ":DECIMAL_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 8, ":DATE_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 9, "A:INT_COL1", null);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX", 10, "B:INT_COL2", null);
+            ResultSet rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 1, "A:VARCHAR_COL1", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 2, "B:VARCHAR_COL2", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 3, ":INT_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 4, ":VARCHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 5, ":CHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 6, ":LONG_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 7, ":DECIMAL_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 8, ":DATE_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 9, "A:INT_COL1", null);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 10, "B:INT_COL2", null);
             assertFalse(rs.next());
             
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), StringUtil.escapeLike("IDX"), new String[] {PTableType.INDEX.getValue().getString() });
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), StringUtil.escapeLike(indexName ), new String[] {PTableType.INDEX.getValue().getString() });
             assertTrue(rs.next());
             assertEquals(PIndexState.ACTIVE.toString(), rs.getString("INDEX_STATE"));
 
             // Verify that there is a row inserted into the data table for the index table.
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName );
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(1));
+            assertEquals(indexName , rs.getString(1));
             assertFalse(rs.next());
             
-            assertActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
             
-            ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " UNUSABLE";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " UNUSABLE";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "IDX", new String[] {PTableType.INDEX.toString()});
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(3));
+            assertEquals(indexName , rs.getString(3));
             assertEquals(PIndexState.INACTIVE.toString(), rs.getString("INDEX_STATE"));
             assertFalse(rs.next());
             
-            assertActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
 
-            ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " USABLE";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " USABLE";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "IDX", new String[] {PTableType.INDEX.toString()});
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(3));
+            assertEquals(indexName , rs.getString(3));
             assertEquals(PIndexState.ACTIVE.toString(), rs.getString("INDEX_STATE"));
             assertFalse(rs.next());
             
-            assertActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
 
-            ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " DISABLE";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " DISABLE";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "IDX", new String[] {PTableType.INDEX.toString()});
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(3));
+            assertEquals(indexName , rs.getString(3));
             assertEquals(PIndexState.DISABLE.toString(), rs.getString("INDEX_STATE"));
             assertFalse(rs.next());
             
-            assertNoActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertNoActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
 
             try {
-                ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " USABLE";
+                ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " USABLE";
                 conn.createStatement().execute(ddl);
                 fail();
             } catch (SQLException e) {
                 assertEquals(SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION.getErrorCode(), e.getErrorCode());
             }
             try {
-                ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " UNUSABLE";
+                ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " UNUSABLE";
                 conn.createStatement().execute(ddl);
                 fail();
             } catch (SQLException e) {
                 assertEquals(SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION.getErrorCode(), e.getErrorCode());
             }
             
-            ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + " REBUILD";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " REBUILD";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "IDX", new String[] {PTableType.INDEX.toString()});
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(3));
+            assertEquals(indexName , rs.getString(3));
             assertEquals(PIndexState.ACTIVE.toString(), rs.getString("INDEX_STATE"));
             assertFalse(rs.next());
             
-            assertActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
 
-            ddl = "DROP INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+            ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
-            assertNoActiveIndex(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE);
+            assertNoActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
 
            // Assert the rows for index table is completely removed.
-            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, false, false);
+            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
             assertFalse(rs.next());
             
             // Assert the row in the original data table is removed.
             // Verify that there is a row inserted into the data table for the index table.
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName );
             assertFalse(rs.next());
             
             // Create another two indexes, and drops the table, verifies the indexes are dropped as well.
-            ddl = "CREATE INDEX IDX1 ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE
+            ddl = "CREATE INDEX " + indexName + "1 ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
                     + " (varchar_col1 ASC, varchar_col2 ASC, int_pk DESC)"
                     + " INCLUDE (int_col1, int_col2)";
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
-            ddl = "CREATE INDEX IDX2 ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE
+            ddl = "CREATE INDEX " + indexName + "2 ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
                     + " (varchar_col1 ASC, varchar_col2 ASC, int_pk DESC)"
                     + " INCLUDE (long_pk, int_col2)";
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, false, false);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 1, "A:VARCHAR_COL1", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 2, "B:VARCHAR_COL2", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 3, ":INT_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 4, ":VARCHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 5, ":CHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 6, ":LONG_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 7, ":DECIMAL_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 8, ":DATE_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 9, "A:INT_COL1", null);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1", 10, "B:INT_COL2", null);
+            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 1, "A:VARCHAR_COL1", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 2, "B:VARCHAR_COL2", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 3, ":INT_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 4, ":VARCHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 5, ":CHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 6, ":LONG_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 7, ":DECIMAL_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 8, ":DATE_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 9, "A:INT_COL1", null);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1", 10, "B:INT_COL2", null);
 
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 1, "A:VARCHAR_COL1", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 2, "B:VARCHAR_COL2", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 3, ":INT_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 4, ":VARCHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 5, ":CHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 6, ":LONG_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 7, ":DECIMAL_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 8, ":DATE_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2", 9, "B:INT_COL2", null);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 1, "A:VARCHAR_COL1", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 2, "B:VARCHAR_COL2", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 3, ":INT_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 4, ":VARCHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 5, ":CHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 6, ":LONG_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 7, ":DECIMAL_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 8, ":DATE_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2", 9, "B:INT_COL2", null);
             assertFalse(rs.next());
             
             // Create another table in the same schema
-            String diffTableNameInSameSchema = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + "2";
+            String diffTableNameInSameSchema = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + "2";
             conn.createStatement().execute("CREATE TABLE " + diffTableNameInSameSchema + "(k INTEGER PRIMARY KEY)");
             try {
-                conn.createStatement().execute("DROP INDEX IDX1 ON " + diffTableNameInSameSchema);
-                fail("Should have realized index IDX1 is not on the table");
+                conn.createStatement().execute("DROP INDEX " + indexName + "1 ON " + diffTableNameInSameSchema);
+                fail("Should have realized index " + indexName + "1 is not on the table");
             } catch (TableNotFoundException ignore) {
                 
             }
-            ddl = "DROP TABLE " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+            ddl = "DROP TABLE " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
-            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, false, false);
+            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
             assertFalse(rs.next());
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX1");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName + "1");
             assertFalse(rs.next());
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, MUTABLE_INDEX_DATA_TABLE, "IDX2");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName + "2");
             assertFalse(rs.next());
         } finally {
             conn.close();
@@ -295,54 +295,56 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
     	// column. The definition is defined in IndexUtil.getIndexColumnDataType.
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        String indexDataTable = generateRandomString();
+        String indexName = generateRandomString();
         conn.setAutoCommit(false);
         try {
-            ensureTableCreated(getUrl(), INDEX_DATA_TABLE);
-            String ddl = "CREATE INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE
+            ensureTableCreated(getUrl(), indexDataTable, INDEX_DATA_TABLE);
+            String ddl = "CREATE INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
                     + " (char_col1 ASC, int_col2 ASC, long_col2 DESC)"
                     + " INCLUDE (int_col1)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
             // Verify the CHAR, INT and LONG are converted to right type.
-            ResultSet rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, false, false);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 1, "A:CHAR_COL1", Order.ASC, Types.VARCHAR);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 2, "B:INT_COL2", Order.ASC, Types.DECIMAL);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 3, "B:LONG_COL2", Order.DESC, Types.DECIMAL);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 4, ":VARCHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 5, ":CHAR_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 6, ":INT_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 7, ":LONG_PK", Order.DESC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 8, ":DECIMAL_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 9, ":DATE_PK", Order.ASC);
-            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX", 10, "A:INT_COL1", null);
+            ResultSet rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 1, "A:CHAR_COL1", Order.ASC, Types.VARCHAR);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 2, "B:INT_COL2", Order.ASC, Types.DECIMAL);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 3, "B:LONG_COL2", Order.DESC, Types.DECIMAL);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 4, ":VARCHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 5, ":CHAR_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 6, ":INT_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 7, ":LONG_PK", Order.DESC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 8, ":DECIMAL_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 9, ":DATE_PK", Order.ASC);
+            assertIndexInfoMetadata(rs, INDEX_DATA_SCHEMA, indexDataTable, indexName , 10, "A:INT_COL1", null);
             assertFalse(rs.next());
             
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName );
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(1));
+            assertEquals(indexName , rs.getString(1));
             assertFalse(rs.next());
             
-            ddl = "ALTER INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE + " UNUSABLE";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " UNUSABLE";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
-            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "IDX", new String[] {PTableType.INDEX.toString()});
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
-            assertEquals("IDX", rs.getString(3));
+            assertEquals(indexName , rs.getString(3));
             assertEquals(PIndexState.INACTIVE.toString(), rs.getString("INDEX_STATE"));
             assertFalse(rs.next());
             
-            ddl = "DROP INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE;
+            ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
             // Assert the rows for index table is completely removed.
-            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, false, false);
+            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
             assertFalse(rs.next());
             
             // Assert the row in the original data table is removed.
             // Verify that there is a row inserted into the data table for the index table.
-            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, "IDX");
+            rs = IndexTestUtil.readDataTableIndexRow(conn, INDEX_DATA_SCHEMA, indexDataTable, indexName );
             assertFalse(rs.next());
         } finally {
             conn.close();
@@ -355,27 +357,28 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         String indexName = "\"lowerCaseIndex\"";
+        String indexDataTable = generateRandomString();
         try {
-            ensureTableCreated(getUrl(), INDEX_DATA_TABLE);
-            String ddl = "CREATE INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE
+            ensureTableCreated(getUrl(), indexDataTable, INDEX_DATA_TABLE);
+            String ddl = "CREATE INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
                     + " (char_col1 ASC, int_col2 ASC, long_col2 DESC)"
                     + " INCLUDE (int_col1)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
 
-            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE + " UNUSABLE";
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " UNUSABLE";
             conn.createStatement().execute(ddl);
             // Verify the metadata for index is correct.
             ResultSet rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), "lowerCaseIndex", new String[] {PTableType.INDEX.toString()});
             assertTrue(rs.next());
             assertEquals("lowerCaseIndex", rs.getString(3));
             
-            ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE;
+            ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
             // Assert the rows for index table is completely removed.
-            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, INDEX_DATA_TABLE, false, false);
+            rs = conn.getMetaData().getIndexInfo(null, INDEX_DATA_SCHEMA, indexDataTable, false, false);
             assertFalse(rs.next());
         } finally {
             conn.close();
@@ -388,9 +391,11 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String indexDataTable = generateRandomString();
+        String indexName = generateRandomString();
         try {
-            ensureTableCreated(getUrl(), TestUtil.INDEX_DATA_TABLE);
-            String ddl = "CREATE INDEX IDX ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE
+            ensureTableCreated(getUrl(), indexDataTable, TestUtil.INDEX_DATA_TABLE);
+            String ddl = "CREATE INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable
             		+ " (a.int_col1, a.long_col1, b.int_col2, b.long_col2)"
             		+ " INCLUDE(int_col1, int_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
@@ -447,18 +452,20 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
-        String ddl = "create table test_table (char_pk varchar not null,"
+        String testTable = generateRandomString();
+        String indexName = generateRandomString();
+        String ddl = "create table " + testTable  + " (char_pk varchar not null,"
         		+ " a.int_col integer, a.long_col integer,"
         		+ " b.int_col integer, b.long_col integer"
         		+ " constraint pk primary key (char_pk))";
         PreparedStatement stmt = conn.prepareStatement(ddl);
         stmt.execute();
         
-        ddl = "CREATE INDEX IDX1 ON test_table (a.int_col, b.int_col)";
+        ddl = "CREATE INDEX " + indexName + "1 ON " + testTable  + " (a.int_col, b.int_col)";
         stmt = conn.prepareStatement(ddl);
         stmt.execute();
         try {
-            ddl = "CREATE INDEX IDX2 ON test_table (int_col)";
+            ddl = "CREATE INDEX " + indexName + "2 ON " + testTable  + " (int_col)";
             stmt = conn.prepareStatement(ddl);
             stmt.execute();
             fail("Should have caught exception");
@@ -472,9 +479,11 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testBinaryNonnullableIndex() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String testTable = generateRandomString();
+        String indexName = generateRandomString();
         try {
             String ddl =
-                    "CREATE TABLE test_table ( "
+                    "CREATE TABLE " + testTable  + " ( "
                     + "v1 BINARY(64) NOT NULL, "
                     + "v2 VARCHAR, "
                     + "v3 BINARY(64), "
@@ -484,68 +493,72 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
             conn.commit();
 
             try {
-                conn.createStatement().execute("CREATE INDEX idx ON test_table (v3) INCLUDE (v4)");
+                conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + testTable  + " (v3) INCLUDE (v4)");
                 fail("Should have seen SQLExceptionCode.VARBINARY_IN_ROW_KEY");
             } catch (SQLException e) {
                 assertEquals(SQLExceptionCode.VARBINARY_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
             }
 
             try {
-                conn.createStatement().execute("CREATE INDEX idx3 ON test_table (v2, v3) INCLUDE (v4)");
+                conn.createStatement().execute("CREATE INDEX " + indexName + "3 ON " + testTable  + " (v2, v3) INCLUDE (v4)");
                 fail("Should have seen SQLExceptionCode.VARBINARY_IN_ROW_KEY");
             } catch (SQLException e) {
                 assertEquals(SQLExceptionCode.VARBINARY_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
             }
-            conn.createStatement().execute("CREATE INDEX idx4 ON test_table (v4) INCLUDE (v2)");
+            conn.createStatement().execute("CREATE INDEX " + indexName + "4 ON " + testTable  + " (v4) INCLUDE (v2)");
             conn.commit();
 
-            conn.createStatement().execute("CREATE INDEX varbinLastInRow ON test_table (v1, v3)");
+            conn.createStatement().execute("CREATE INDEX varbinLastInRow ON " + testTable  + " (v1, v3)");
             conn.commit();
 
-            conn.createStatement().execute( "CREATE INDEX idx5 ON test_table (v2) INCLUDE (v4, v3, v1)");
+            conn.createStatement().execute( "CREATE INDEX " + indexName + "5 ON " + testTable  + " (v2) INCLUDE (v4, v3, v1)");
             conn.commit();
 
             conn.createStatement().executeQuery(
-                "select v1,v2,v3,v4 FROM test_table where v2 = 'abc' and v3 != 'a'");
+                "select v1,v2,v3,v4 FROM " + testTable  + " where v2 = 'abc' and v3 != 'a'");
 
 
         } finally {
             conn.close();
         }
     }
-    
+
     @Test
     public void testAsyncCreatedDate() throws Exception {
         Date d0 = new Date(System.currentTimeMillis());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
-        String ddl = "create table test_table (k varchar primary key, v1 varchar, v2 varchar, v3 varchar)";
+        String testTable = generateRandomString();
+
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar, v2 varchar, v3 varchar)";
         PreparedStatement stmt = conn.prepareStatement(ddl);
         stmt.execute();
+        String indexName = "ASYNCIND_" + generateRandomString();
         
-        ddl = "CREATE INDEX IDX1 ON test_table (v1) ASYNC";
+        ddl = "CREATE INDEX " + indexName + "1 ON " + testTable  + " (v1) ASYNC";
         stmt = conn.prepareStatement(ddl);
         stmt.execute();
-        ddl = "CREATE INDEX IDX2 ON test_table (v2) ASYNC";
+        ddl = "CREATE INDEX " + indexName + "2 ON " + testTable  + " (v2) ASYNC";
         stmt = conn.prepareStatement(ddl);
         stmt.execute();
-        ddl = "CREATE INDEX IDX3 ON test_table (v3)";
+        ddl = "CREATE INDEX " + indexName + "3 ON " + testTable  + " (v3)";
         stmt = conn.prepareStatement(ddl);
         stmt.execute();
         
         ResultSet rs = conn.createStatement().executeQuery(
             "select table_name, " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " +
             "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") " +
-            "where " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " is not null " +
+            "where " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " is not null and table_name like 'ASYNCIND_%' " +
             "order by " + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE
         );
         assertTrue(rs.next());
-        assertEquals("IDX1", rs.getString(1));
+        assertEquals(indexName + "1", rs.getString(1));
         Date d1 = rs.getDate(2);
         assertTrue(d1.after(d0));
         assertTrue(rs.next());
-        assertEquals("IDX2", rs.getString(1));
+        assertEquals(indexName + "2", rs.getString(1));
         Date d2 = rs.getDate(2);
         assertTrue(d2.after(d1));
         assertFalse(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 416c452..f254c49 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.end2end.index;
 
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -49,7 +51,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -57,10 +59,8 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.*;
 import org.apache.phoenix.schema.PTable.IndexType;
-import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -75,24 +75,16 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class LocalIndexIT extends BaseHBaseManagedTimeIT {
-
-    private String schemaName="TEST";
+public class LocalIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     private boolean isNamespaceMapped;
-    private String tableName = schemaName + ".T";
-    private String indexTableName = schemaName + ".I";
-    private String indexName = "I";
-    private String indexPhysicalTableName;
-    private TableName physicalTableName;
+    String schemaName="TEST";
 
     public LocalIndexIT(boolean isNamespaceMapped) {
         this.isNamespaceMapped = isNamespaceMapped;
-        this.physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
-        this.indexPhysicalTableName = this.physicalTableName.getNameAsString();
     }
     
     @BeforeClass 
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Drop the HBase table metadata for this test
@@ -125,6 +117,10 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexRoundTrip() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+
         createBaseTable(tableName, null, null);
         Connection conn1 = DriverManager.getConnection(getUrl());
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
@@ -145,6 +141,9 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexCreationWithSplitsShouldFail() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, null, null);
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
@@ -161,6 +160,9 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexCreationWithSaltingShouldFail() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, null, null);
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
@@ -177,6 +179,11 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexTableRegionSplitPolicyAndSplitKeys() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         createBaseTable(tableName, null,"('e','i','o')");
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
@@ -202,23 +209,34 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         return DriverManager.getConnection(getUrl(),props);
     }
 
+
     @Test
     public void testDropLocalIndexTable() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
         createBaseTable(tableName, null, null);
+
+        String sequenceName = getViewIndexSequenceName(PNameFactory.newName(tableName), null, isNamespaceMapped);
+        String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceMapped);
+
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
+        verifySequence(null, sequenceName, sequenceSchemaName, true);
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
         conn1.createStatement().execute("DROP TABLE "+ tableName);
-        ResultSet rs = conn2.createStatement().executeQuery("SELECT "
-                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
-                + PhoenixDatabaseMetaData.SEQUENCE_NAME
-                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
-        assertFalse("View index sequences should be deleted.", rs.next());
+
+        verifySequence(null, sequenceName, sequenceSchemaName, false);
     }
     
     @Test
     public void testPutsToLocalIndexTable() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = getConnection();
         conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
@@ -253,6 +271,12 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testBuildIndexWhenUserTableAlreadyHasData() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         conn1.createStatement().execute("UPSERT INTO "+tableName+" values('b',1,2,4,'z')");
@@ -287,6 +311,12 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScan() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
@@ -419,6 +449,12 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScanJoinColumnsFromDataTable() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = getConnection();
         try{
@@ -553,6 +589,10 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testIndexPlanSelectionIfBothGlobalAndLocalIndexesHasSameColumnsAndOrder() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = getConnection();
         conn1.createStatement().execute("UPSERT INTO "+tableName+" values('b',1,2,4,'z')");
@@ -570,8 +610,12 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         conn1.close();
     }
 
+
     @Test
     public void testDropLocalIndexShouldDeleteDataFromLocalIndexTable() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         try {
@@ -583,7 +627,7 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
             conn1.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + tableName + "(v1)");
             conn1.createStatement().execute("DROP INDEX " + indexName + " ON " + tableName);
             HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
-            HTable table = new HTable(admin.getConfiguration() ,TableName.valueOf(TestUtil.DEFAULT_DATA_TABLE_NAME));
+            HTable table = new HTable(admin.getConfiguration() ,TableName.valueOf(tableName));
             Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
             byte[][] startKeys = startEndKeys.getFirst();
             byte[][] endKeys = startEndKeys.getSecond();
@@ -614,6 +658,10 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexRowsShouldBeDeletedWhenUserTableRowsDeleted() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         try {
@@ -636,6 +684,9 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testScanWhenATableHasMultipleLocalIndexes() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         try {
@@ -658,6 +709,9 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexesOnTableWithImmutableRows() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = getConnection();
         try {
@@ -701,6 +755,10 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScanWithInList() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String indexTableName = schemaName + "." + indexName;
+
         createBaseTable(tableName, null, "('e','i','o')");
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
@@ -731,13 +789,15 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
             Statement statement = conn1.createStatement();
-            statement.execute("create table example (id integer not null,fn varchar,"
+            String tableName = generateRandomString();
+            String indexName = generateRandomString();
+            statement.execute("create table " + tableName + " (id integer not null,fn varchar,"
                     + "ln varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
-            statement.execute("upsert into example values(1,'fn','ln')");
+            statement.execute("upsert into " + tableName + "  values(1,'fn','ln')");
             statement
-                    .execute("create local index my_idx on example (fn)");
-            statement.execute("upsert into example values(2,'fn1','ln1')");
-            ResultSet rs = statement.executeQuery("SELECT COUNT(*) FROM my_idx");
+                    .execute("create local index " + indexName + " on " + tableName + "  (fn)");
+            statement.execute("upsert into " + tableName + "  values(2,'fn1','ln1')");
+            ResultSet rs = statement.executeQuery("SELECT COUNT(*) FROM " + indexName );
             assertTrue(rs.next());
        } finally {
             conn1.close();
@@ -746,6 +806,11 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScanAfterRegionSplit() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         if (isNamespaceMapped) { return; }
         createBaseTable(tableName, null, "('e','j','o')");
         Connection conn1 = getConnection();
@@ -838,6 +903,9 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScanWithSmallChunks() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+
         createBaseTable(tableName, 3, null);
         Properties props = new Properties();
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, "2");
@@ -881,6 +949,11 @@ public class LocalIndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testLocalIndexScanAfterRegionsMerge() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String indexPhysicalTableName = physicalTableName.getNameAsString();
+
         if (isNamespaceMapped) { return; }
         createBaseTable(tableName, null, "('e','j','o')");
         Connection conn1 = getConnection();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 6a49076..0fde1c6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
@@ -66,30 +66,24 @@ import com.google.common.collect.Maps;
 import com.google.common.primitives.Doubles;
 
 @RunWith(Parameterized.class)
-public class MutableIndexIT extends BaseHBaseManagedTimeIT {
+public class MutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     
     protected final boolean localIndex;
     private final String tableDDLOptions;
-    private final String tableName;
-    private final String indexName;
-    private final String fullTableName;
-    private final String fullIndexName;
+	private  final boolean transactional;
 	
     public MutableIndexIT(boolean localIndex, boolean transactional) {
 		this.localIndex = localIndex;
+		this.transactional = transactional;
 		StringBuilder optionBuilder = new StringBuilder();
 		if (transactional) {
 			optionBuilder.append("TRANSACTIONAL=true");
 		}
 		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
-        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
-        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
     
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
@@ -110,7 +104,12 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
-            createMultiCFTestTable(fullTableName, tableDDLOptions);
+			String tableName = "TBL_" + generateRandomString();
+			String indexName = "IDX_" + generateRandomString();
+			String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+			String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+
+			createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName);
             PreparedStatement stmt = conn.prepareStatement("CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName 
             		+ " (char_col1 ASC, int_col1 ASC) INCLUDE (long_col1, long_col2)");
@@ -203,7 +202,12 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testCoveredColumns() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
+		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+
 	        conn.setAutoCommit(false);
 	        String query;
 	        ResultSet rs;
@@ -308,6 +312,10 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testCompoundIndexKey() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
+		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -423,6 +431,10 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMultipleUpdatesToSingleRow() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
+		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
 	        String query;
@@ -503,6 +515,9 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testUpsertingNullForIndexedColumns() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
+		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         String testTableName = tableName + "_" + System.currentTimeMillis();
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -586,8 +601,10 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     public void testAlterTableWithImmutability() throws Exception {
         String query;
         ResultSet rs;
-        String tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + "_" + System.currentTimeMillis();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+		String tableName = "TBL_" + generateRandomString();
+		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -623,16 +640,19 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(2));
         props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
         Connection conn1 = DriverManager.getConnection(getUrl());
-        HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+		String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+		HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
         dropTable(admin, conn1);
         try{
             String[] strings = {"a","b","c","d","e","f","g","h","i","j","k","l","m","n","o","p","q","r","s","t","u","v","w","x","y","z"};
-            createTableAndLoadData(conn1, strings, isReverse);
+            createTableAndLoadData(conn1, tableName, indexName, strings, isReverse);
 
             ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
-            splitDuringScan(conn1, strings, admin, isReverse);
+            splitDuringScan(conn1, tableName, indexName, strings, admin, isReverse);
             dropTable(admin, conn1);
+
        } finally {
            dropTable(admin, conn1);
            if(conn1 != null) conn1.close();
@@ -641,23 +661,22 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     }
 
     private void dropTable(HBaseAdmin admin, Connection conn) throws SQLException, IOException {
-        try {
-            conn.createStatement().execute("DROP TABLE IF EXISTS "+ tableName);
-        } finally {
-            if(admin.tableExists(tableName)) {
-                admin.disableTable(TableName.valueOf(tableName));
-                admin.deleteTable(TableName.valueOf(tableName));
-            }
-            if(!localIndex) {
-                if(admin.tableExists(indexName)) {
-                    admin.disableTable(TableName.valueOf(indexName));
-                    admin.deleteTable(TableName.valueOf(indexName));
-                }
-            }
+
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
+        conn.createStatement().execute("DROP TABLE IF EXISTS "+ tableName);
+        if(admin.tableExists(tableName)) {
+            admin.disableTable(TableName.valueOf(tableName));
+            admin.deleteTable(TableName.valueOf(tableName));
+        } 
+        if(!localIndex && admin.tableExists(indexName)) {
+            admin.disableTable(indexName);
+            admin.deleteTable(indexName);
+
         }
     }
 
-    private void createTableAndLoadData(Connection conn1, String[] strings, boolean isReverse) throws SQLException {
+    private void createTableAndLoadData(Connection conn1, String tableName, String indexName, String[] strings, boolean isReverse) throws SQLException {
         createBaseTable(conn1, tableName, null);
         for (int i = 0; i < 26; i++) {
             conn1.createStatement().execute(
@@ -673,6 +692,8 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     public void testIndexHalfStoreFileReader() throws Exception {
         Connection conn1 = DriverManager.getConnection(getUrl());
         HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
+		String tableName = "TBL_" + generateRandomString();
+		String indexName = "IDX_" + generateRandomString();
         try {
             dropTable(admin, conn1);
             createBaseTable(conn1, tableName, "('e')");
@@ -738,9 +759,11 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private List<HRegionInfo> splitDuringScan(Connection conn1, String[] strings, HBaseAdmin admin, boolean isReverse)
+
+    private List<HRegionInfo> splitDuringScan(Connection conn1, String tableName, String indexName, String[] strings, HBaseAdmin admin, boolean isReverse)
             throws SQLException, IOException, InterruptedException {
         ResultSet rs;
+
         String query = "SELECT t_id,k1,v1 FROM " + tableName;
         rs = conn1.createStatement().executeQuery(query);
         String[] tIdColumnValues = new String[26]; 
@@ -815,9 +838,10 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
     
   @Test
   public void testTenantSpecificConnection() throws Exception {
-      String tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + "_" + System.currentTimeMillis();
-      String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-      Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+	  String tableName = "TBL_" + generateRandomString();
+	  String indexName = "IDX_" + generateRandomString();
+	  String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+	  Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
       try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
           conn.setAutoCommit(false);
           // create data table


[41/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index ea376de..ae3b977 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -56,7 +56,7 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
+public class AlterTableWithViewsIT extends BaseHBaseManagedTimeTableReuseIT {
     
     private final boolean isMultiTenant;
     
@@ -73,7 +73,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
     }
 	
 	@BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
@@ -89,49 +89,53 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
     public void testAddNewColumnsToBaseTableWithViews() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {       
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+            String tableName = generateRandomString();
+            String viewOfTable = tableName + "_VIEW";
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL,"
                             + " COL2 bigint NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // adding a new pk column and a new regular column
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
         } 
     }
     
     @Test
     public void testDropColumnsFromBaseTableWithView() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
-                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) { 
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW (" + " %s ID char(1) NOT NULL,"
+                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
+            String tableName = generateRandomString();
+            String viewOfTable = tableName + "_VIEW";
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " (" + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL," + " COL2 bigint NOT NULL,"
                             + " COL3 varchar(10)," + " COL4 varchar(10)," + " COL5 varchar(10),"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)" + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 6,
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 6,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4",
                 "COL5");
 
             viewConn.createStatement()
                     .execute(
-                        "CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 8, 6,
+                        "CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 8, 6,
                 "ID", "COL1", "COL2", "COL3", "COL4", "COL5", "VIEW_COL1", "VIEW_COL2");
 
             // drop two columns from the base table
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW DROP COLUMN COL3, COL5");
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 4,
+            conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN COL3, COL5");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 4,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL4");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 6, 4,
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 6, 4,
                 "ID", "COL1", "COL2", "COL4", "VIEW_COL1", "VIEW_COL2");
         }
     }
@@ -142,20 +146,23 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+            String tableName = generateRandomString();
+            String viewOfTable = tableName + "_VIEW";
+
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(10) NOT NULL,"
                             + " COL1 integer NOT NULL,"
                             + " COL2 bigint NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256), VIEW_COL3 VARCHAR, VIEW_COL4 DECIMAL, VIEW_COL5 DECIMAL(10,2), VIEW_COL6 VARCHAR, CONSTRAINT pk PRIMARY KEY (VIEW_COL5, VIEW_COL6) ) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256), VIEW_COL3 VARCHAR, VIEW_COL4 DECIMAL, VIEW_COL5 DECIMAL(10,2), VIEW_COL6 VARCHAR, CONSTRAINT pk PRIMARY KEY (VIEW_COL5, VIEW_COL6) ) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn,viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
             
             // upsert single row into view
-            String dml = "UPSERT INTO VIEWOFTABLE VALUES(?,?,?,?,?, ?, ?, ?, ?)";
+            String dml = "UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?, ?, ?, ?, ?)";
             PreparedStatement stmt = viewConn.prepareStatement(dml);
             stmt.setString(1, "view1");
             stmt.setInt(2, 12);
@@ -171,7 +178,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there is already a view column with same name of different type
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 char(10)");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 char(10)");
                 fail();
             }
             catch (SQLException e) {
@@ -180,7 +187,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there is already a view column with same name with different scale
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,1)");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,1)");
                 fail();
             }
             catch (SQLException e) {
@@ -189,7 +196,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there is already a view column with same name with different length
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(9,2)");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(9,2)");
                 fail();
             }
             catch (SQLException e) {
@@ -198,7 +205,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there is already a view column with different length
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR");
                 fail();
             }
             catch (SQLException e) {
@@ -206,16 +213,16 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             }
             
             // validate that there were no columns added to the table or view
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
             
             // should succeed 
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
             
             // query table
-            ResultSet rs = stmt.executeQuery("SELECT * FROM TABLEWITHVIEW");
+            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -225,7 +232,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // query view
-            rs = stmt.executeQuery("SELECT * FROM VIEWOFTABLE");
+            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -246,20 +253,23 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {      
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+            String tableName = generateRandomString();
+            String viewOfTable = tableName + "_VIEW";
+
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
                             + " %s ID char(10) NOT NULL,"
                             + " COL1 integer NOT NULL,"
                             + " COL2 integer NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // upsert single row into view
-            String dml = "UPSERT INTO VIEWOFTABLE VALUES(?,?,?,?,?)";
+            String dml = "UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?)";
             PreparedStatement stmt = viewConn.prepareStatement(dml);
             stmt.setString(1, "view1");
             stmt.setInt(2, 12);
@@ -271,7 +281,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2 to the pk
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -280,7 +290,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2  to the pk 
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -289,7 +299,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2 to the pk
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -298,7 +308,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2  to the pk 
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256)");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256)");
                 fail();
             }
             catch (SQLException e) {
@@ -307,7 +317,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2 to the pk in the right order
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY, VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + "  ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY, VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -316,7 +326,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there we have to add both VIEW_COL1 and VIEW_COL2 with the right sort order
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY DESC, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY DESC, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -324,12 +334,12 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             }
             
             // add the pk column of the view to the base table
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // query table
-            ResultSet rs = stmt.executeQuery("SELECT * FROM TABLEWITHVIEW");
+            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -339,7 +349,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // query view
-            rs = stmt.executeQuery("SELECT * FROM VIEWOFTABLE");
+            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -353,25 +363,28 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testAddExistingViewPkColumnToBaseTableWithMultipleViews() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
-                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {              
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
+            String tableName = generateRandomString();
+            String viewOfTable1 = tableName + "_VIEW1";
+            String viewOfTable2 = tableName + "_VIEW2";
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + "("
                             + " %s ID char(10) NOT NULL,"
                             + " COL1 integer NOT NULL,"
                             + " COL2 integer NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                             + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE1 ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE1", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE2 ( VIEW_COL3 VARCHAR(256), VIEW_COL4 DECIMAL(10,2) CONSTRAINT pk PRIMARY KEY (VIEW_COL3, VIEW_COL4)) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE2", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL3", "VIEW_COL4");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL3 VARCHAR(256), VIEW_COL4 DECIMAL(10,2) CONSTRAINT pk PRIMARY KEY (VIEW_COL3, VIEW_COL4)) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL3", "VIEW_COL4");
             
             try {
                 // should fail because there are two view with different pk columns
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -380,7 +393,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because there are two view with different pk columns
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -389,7 +402,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because slot positions of pks are different
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY, VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY, VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -398,7 +411,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because slot positions of pks are different
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY, VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL3 VARCHAR PRIMARY KEY, VIEW_COL4 DECIMAL PRIMARY KEY, VIEW_COL1 DECIMAL PRIMARY KEY, VIEW_COL2 VARCHAR PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -415,23 +428,26 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
             viewConn2.setAutoCommit(false);
-            String ddlFormat = "CREATE TABLE IF NOT EXISTS TABLEWITHVIEW ("
+            String tableName = generateRandomString();
+            String viewOfTable1 = tableName + "_VIEW1";
+            String viewOfTable2 = tableName + "_VIEW2";
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + "("
                     + " %s ID char(10) NOT NULL,"
                     + " COL1 integer NOT NULL,"
                     + " COL2 integer NOT NULL,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
                     + " ) %s";
             conn.createStatement().execute(generateDDL(ddlFormat));
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE1 ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE1", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
-            viewConn2.createStatement().execute("CREATE VIEW VIEWOFTABLE2 ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM TABLEWITHVIEW");
-            assertTableDefinition(conn, "VIEWOFTABLE2", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn2.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR(256) CONSTRAINT pk PRIMARY KEY (VIEW_COL1, VIEW_COL2)) AS SELECT * FROM " + tableName);
+            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // upsert single row into both view
-            String dml = "UPSERT INTO VIEWOFTABLE1 VALUES(?,?,?,?,?)";
+            String dml = "UPSERT INTO " + viewOfTable1 + " VALUES(?,?,?,?,?)";
             PreparedStatement stmt = viewConn.prepareStatement(dml);
             stmt.setString(1, "view1");
             stmt.setInt(2, 12);
@@ -440,7 +456,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             stmt.setString(5, "view5");
             stmt.execute();
             viewConn.commit();
-            dml = "UPSERT INTO VIEWOFTABLE2 VALUES(?,?,?,?,?)";
+            dml = "UPSERT INTO " + viewOfTable2 + " VALUES(?,?,?,?,?)";
             stmt = viewConn2.prepareStatement(dml);
             stmt.setString(1, "view1");
             stmt.setInt(2, 12);
@@ -452,7 +468,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because the view have two extra columns in their pk
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -461,7 +477,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because the view have two extra columns in their pk
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
@@ -470,20 +486,20 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             
             try {
                 // should fail because slot positions of pks are different
-                conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL2 DECIMAL(10,2) PRIMARY KEY, VIEW_COL1 VARCHAR(256) PRIMARY KEY");
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 DECIMAL(10,2) PRIMARY KEY, VIEW_COL1 VARCHAR(256) PRIMARY KEY");
                 fail();
             }
             catch (SQLException e) {
                 assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
             }
             
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
-            assertTableDefinition(conn, "TABLEWITHVIEW", PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, "VIEWOFTABLE1", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
-            assertTableDefinition(conn, "VIEWOFTABLE2", PTableType.VIEW, "TABLEWITHVIEW", 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,2) PRIMARY KEY, VIEW_COL2 VARCHAR(256) PRIMARY KEY");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(conn, viewOfTable1, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            assertTableDefinition(conn, viewOfTable2, PTableType.VIEW, tableName, 1, 5, 5, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             // query table
-            ResultSet rs = stmt.executeQuery("SELECT * FROM TABLEWITHVIEW");
+            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -493,7 +509,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // query both views
-            rs = viewConn.createStatement().executeQuery("SELECT * FROM VIEWOFTABLE1");
+            rs = viewConn.createStatement().executeQuery("SELECT * FROM " + viewOfTable1);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -501,7 +517,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             assertEquals(14, rs.getInt("VIEW_COL1"));
             assertEquals("view5", rs.getString("VIEW_COL2"));
             assertFalse(rs.next());
-            rs = viewConn2.createStatement().executeQuery("SELECT * FROM VIEWOFTABLE2");
+            rs = viewConn2.createStatement().executeQuery("SELECT * FROM " + viewOfTable2);
             assertTrue(rs.next());
             assertEquals("view1", rs.getString("ID"));
             assertEquals(12, rs.getInt("COL1"));
@@ -521,7 +537,7 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
         else {
             cols = columnNames;
         }
-        AlterMultiTenantTableWithViews.assertTableDefinition(conn, tableName, tableType, parentTableName, sequenceNumber, columnCount + delta,
+        AlterMultiTenantTableWithViewsIT.assertTableDefinition(conn, tableName, tableType, parentTableName, sequenceNumber, columnCount + delta,
             baseColumnCount==QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT ? baseColumnCount : baseColumnCount +delta, cols);
     }
     
@@ -602,9 +618,9 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDivergedViewsStayDiverged() throws Exception {
-        String baseTable = "testDivergedViewsStayDiverged";
-        String view1 = baseTable + "_view1";
-        String view2 = baseTable + "_view2";
+        String baseTable = generateRandomString();
+        String view1 = baseTable + "_VIEW1";
+        String view2 = baseTable + "_VIEW2";
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ;
                 Connection viewConn2 = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL2) : conn) {
@@ -645,7 +661,8 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
     public void testMakeBaseTableTransactional() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {  
-            String baseTableName = "NONTXNTBL_" + (isMultiTenant ? "0":"1");
+            String baseTableName = "NONTXNTBL_" + generateRandomString() + (isMultiTenant ? "0":"1");
+            String viewOfTable = baseTableName + "_VIEW";
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTableName + " ("
                             + " %s ID char(1) NOT NULL,"
                             + " COL1 integer NOT NULL,"
@@ -655,24 +672,24 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
             conn.createStatement().execute(generateDDL(ddlFormat));
             assertTableDefinition(conn, baseTableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             
-            viewConn.createStatement().execute("CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM "+baseTableName);
-            assertTableDefinition(conn, "VIEWOFTABLE", PTableType.VIEW, "TABLEWITHVIEW", 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM "+baseTableName);
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, baseTableName, 0, 5, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2");
             
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             HTableInterface htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
             assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
             assertFalse(phoenixConn.getTable(new PTableKey(null, baseTableName)).isTransactional());
-            assertFalse(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, "VIEWOFTABLE")).isTransactional());
+            assertFalse(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());
             
             // make the base table transactional
             conn.createStatement().execute("ALTER TABLE " + baseTableName + " SET TRANSACTIONAL=true");
             // query the view to force the table cache to be updated
-            viewConn.createStatement().execute("SELECT * FROM VIEWOFTABLE");
+            viewConn.createStatement().execute("SELECT * FROM " + viewOfTable);
             htable = phoenixConn.getQueryServices().getTable(Bytes.toBytes(baseTableName));
             assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
             assertTrue(phoenixConn.getTable(new PTableKey(null, baseTableName)).isTransactional());
-            assertTrue(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, "VIEWOFTABLE")).isTransactional());
+            assertTrue(viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)).isTransactional());
         } 
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 080ccad..017a477 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -60,9 +60,11 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
+public class AppendOnlySchemaIT extends BaseHBaseManagedTimeTableReuseIT {
+
     
     private void testTableWithSameSchema(boolean notExists, boolean sameClient) throws Exception {
+
         // use a spyed ConnectionQueryServices so we can verify calls to getTable
         ConnectionQueryServices connectionQueryServices =
                 Mockito.spy(driver.getConnectionQueryServices(getUrl(),
@@ -72,20 +74,24 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
 
         try (Connection conn1 = connectionQueryServices.connect(getUrl(), props);
                 Connection conn2 = sameClient ? conn1 : connectionQueryServices.connect(getUrl(), props)) {
+
+            String metricTableName = generateRandomString();
+            String viewName = generateRandomString();
+            String metricIdSeqTableName = generateRandomString();
             // create sequence for auto partition
-            conn1.createStatement().execute("CREATE SEQUENCE metric_id_seq CACHE 1");
+            conn1.createStatement().execute("CREATE SEQUENCE " + metricIdSeqTableName + " CACHE 1");
             // create base table
-            conn1.createStatement().execute("CREATE TABLE metric_table (metricId INTEGER NOT NULL, metricVal DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))" 
-                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=metric_id_seq");
+            conn1.createStatement().execute("CREATE TABLE "+ metricTableName + "(metricId INTEGER NOT NULL, metricVal DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))"
+                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=" + metricIdSeqTableName);
             // create view
             String ddl =
-                    "CREATE VIEW " + (notExists ? "IF NOT EXISTS" : "")
-                            + " view1( hostName varchar NOT NULL, tagName varChar"
+                    "CREATE VIEW " + (notExists ? "IF NOT EXISTS " : "")
+                            + viewName + " ( hostName varchar NOT NULL, tagName varChar"
                             + " CONSTRAINT HOSTNAME_PK PRIMARY KEY (hostName))"
-                            + " AS SELECT * FROM metric_table"
+                            + " AS SELECT * FROM " + metricTableName
                             + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=300000";
             conn1.createStatement().execute(ddl);
-            conn1.createStatement().execute("UPSERT INTO view1(hostName, metricVal) VALUES('host1', 1.0)");
+            conn1.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal) VALUES('host1', 1.0)");
             conn1.commit();
             reset(connectionQueryServices);
 
@@ -103,7 +109,7 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
             }
             
             // verify getTable rpcs
-            verify(connectionQueryServices, sameClient ? never() : times(1)).getTable((PName)isNull(), eq(new byte[0]), eq(Bytes.toBytes("VIEW1")), anyLong(), anyLong());
+            verify(connectionQueryServices, sameClient ? never() : times(1)).getTable((PName)isNull(), eq(new byte[0]), eq(Bytes.toBytes(viewName)), anyLong(), anyLong());
             
             // verify no create table rpcs
             verify(connectionQueryServices, never()).createTable(anyListOf(Mutation.class),
@@ -112,7 +118,7 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
             reset(connectionQueryServices);
             
             // execute alter table ddl that adds the same column
-            ddl = "ALTER VIEW view1 ADD " + (notExists ? "IF NOT EXISTS" : "") + " tagName varchar";
+            ddl = "ALTER VIEW " + viewName + " ADD " + (notExists ? "IF NOT EXISTS" : "") + " tagName varchar";
             try {
                 conn2.createStatement().execute(ddl);
                 if (!notExists) {
@@ -130,10 +136,10 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
             verify(connectionQueryServices, notExists ? times(1) : never() ).addColumn(eq(Collections.<Mutation>emptyList()), any(PTable.class), anyMap(), anySetOf(String.class));
 
             // upsert one row
-            conn2.createStatement().execute("UPSERT INTO view1(hostName, metricVal) VALUES('host2', 2.0)");
+            conn2.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal) VALUES('host2', 2.0)");
             conn2.commit();
             // verify data in base table
-            ResultSet rs = conn2.createStatement().executeQuery("SELECT * from metric_table");
+            ResultSet rs = conn2.createStatement().executeQuery("SELECT * from " + metricTableName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(1.0, rs.getDouble(2), 1e-6);
@@ -142,7 +148,7 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
             assertEquals(2.0, rs.getDouble(2), 1e-6);
             assertFalse(rs.next());
             // verify data in view
-            rs = conn2.createStatement().executeQuery("SELECT * from view1");
+            rs = conn2.createStatement().executeQuery("SELECT * from " + viewName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(1.0, rs.getDouble(2), 1e-6);
@@ -179,47 +185,52 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn1 = DriverManager.getConnection(getUrl(), props);
                 Connection conn2 = sameClient ? conn1 : DriverManager.getConnection(getUrl(), props)) {
+
+            String metricTableName = generateRandomString();
+            String viewName = generateRandomString();
+            String metricIdSeqTableName = generateRandomString();
+
             // create sequence for auto partition
-            conn1.createStatement().execute("CREATE SEQUENCE metric_id_seq CACHE 1");
+            conn1.createStatement().execute("CREATE SEQUENCE " + metricIdSeqTableName + " CACHE 1");
             // create base table
-            conn1.createStatement().execute("CREATE TABLE metric_table (metricId INTEGER NOT NULL, metricVal1 DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))" 
-                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=metric_id_seq");
+            conn1.createStatement().execute("CREATE TABLE " + metricTableName + " (metricId INTEGER NOT NULL, metricVal1 DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))"
+                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=" + metricIdSeqTableName);
             // create view
             String ddl =
-                    "CREATE VIEW IF NOT EXISTS"
-                            + " view1( hostName varchar NOT NULL,"
+                    "CREATE VIEW IF NOT EXISTS "
+                            + viewName + "( hostName varchar NOT NULL,"
                             + " CONSTRAINT HOSTNAME_PK PRIMARY KEY (hostName))"
-                            + " AS SELECT * FROM metric_table"
+                            + " AS SELECT * FROM " + metricTableName
                             + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=300000";
             conn1.createStatement().execute(ddl);
             
-            conn1.createStatement().execute("UPSERT INTO view1(hostName, metricVal1) VALUES('host1', 1.0)");
+            conn1.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal1) VALUES('host1', 1.0)");
             conn1.commit();
 
             // execute ddl that creates that same view with an additional pk column and regular column
             // and also changes the order of the pk columns (which is not respected since we only 
             // allow appending columns)
             ddl =
-                    "CREATE VIEW IF NOT EXISTS"
-                            + " view1( instanceName varchar, hostName varchar, metricVal2 double, metricVal1 double"
+                    "CREATE VIEW IF NOT EXISTS "
+                            + viewName + "( instanceName varchar, hostName varchar, metricVal2 double, metricVal1 double"
                             + " CONSTRAINT HOSTNAME_PK PRIMARY KEY (instancename, hostName))"
-                            + " AS SELECT * FROM metric_table"
+                            + " AS SELECT * FROM " + metricTableName
                             + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=300000";
             conn2.createStatement().execute(ddl);
 
             conn2.createStatement().execute(
-                "UPSERT INTO view1(hostName, instanceName, metricVal1, metricval2) VALUES('host2', 'instance2', 21.0, 22.0)");
+                "UPSERT INTO " + viewName + "(hostName, instanceName, metricVal1, metricval2) VALUES('host2', 'instance2', 21.0, 22.0)");
             conn2.commit();
             
-            conn1.createStatement().execute("UPSERT INTO view1(hostName, metricVal1) VALUES('host3', 3.0)");
+            conn1.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal1) VALUES('host3', 3.0)");
             conn1.commit();
             
             // verify data exists
-            ResultSet rs = conn2.createStatement().executeQuery("SELECT * from view1");
+            ResultSet rs = conn2.createStatement().executeQuery("SELECT * from " + viewName);
             
             // verify the two columns were added correctly
             PTable table =
-                    conn2.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "VIEW1"));
+                    conn2.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, viewName));
             List<PColumn> pkColumns = table.getPKColumns();
             assertEquals(3,table.getPKColumns().size());
             // even though the second create view statement changed the order of the pk, the original order is maintained
@@ -274,36 +285,15 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
         testAddColumns(false);
     }
 
-    public void testCreateTableDropColumns() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            String ddl =
-                    "create table IF NOT EXISTS TEST( id1 char(2) NOT NULL," + " col1 integer,"
-                            + " col2 integer," + " CONSTRAINT NAME_PK PRIMARY KEY (id1))"
-                            + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=300000";
-            conn.createStatement().execute(ddl);
-            conn.createStatement().execute("UPSERT INTO TEST VALUES('a', 11)");
-            conn.commit();
-
-            // execute ddl while dropping a column
-            ddl = "alter table TEST drop column col1";
-            try {
-                conn.createStatement().execute(ddl);
-                fail("Dropping a column from a table with APPEND_ONLY_SCHEMA=true should fail");
-            } catch (SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA.getErrorCode(),
-                    e.getErrorCode());
-            }
-        }
-    }
-
     @Test
     public void testValidateAttributes() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String tableName = generateRandomString();
+            String viewName = generateRandomString();
             try {
                 conn.createStatement().execute(
-                    "create table IF NOT EXISTS TEST1 ( id char(1) NOT NULL,"
+                    "create table IF NOT EXISTS " + tableName + " ( id char(1) NOT NULL,"
                             + " col1 integer NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1))"
                             + " APPEND_ONLY_SCHEMA = true");
@@ -314,13 +304,13 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
             }
             
             conn.createStatement().execute(
-                "create table IF NOT EXISTS TEST1 ( id char(1) NOT NULL,"
+                "create table IF NOT EXISTS " + tableName + " ( id char(1) NOT NULL,"
                         + " col1 integer NOT NULL"
                         + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1))"
                         + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1000");
             try {
                 conn.createStatement().execute(
-                    "create view IF NOT EXISTS MY_VIEW (val1 integer NOT NULL) AS SELECT * FROM TEST1"
+                    "create view IF NOT EXISTS " + viewName + " (val1 integer NOT NULL) AS SELECT * FROM " + tableName
                             + " UPDATE_CACHE_FREQUENCY=1000");
                 fail("APPEND_ONLY_SCHEMA must be true for a view if it is true for the base table ");
             }
@@ -336,25 +326,28 @@ public class AppendOnlySchemaIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn1 = DriverManager.getConnection(getUrl(), props);
                 Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
+            String metricTableName = generateRandomString();
+            String viewName = generateRandomString();
+            String metricIdSeqTableName = generateRandomString();
             // create sequence for auto partition
-            conn1.createStatement().execute("CREATE SEQUENCE metric_id_seq CACHE 1");
+            conn1.createStatement().execute("CREATE SEQUENCE " + metricIdSeqTableName + "  CACHE 1");
             // create base table
-            conn1.createStatement().execute("CREATE TABLE metric_table (metricId INTEGER NOT NULL, metricVal DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))" 
-                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=metric_id_seq");
+            conn1.createStatement().execute("CREATE TABLE " + metricTableName + " (metricId INTEGER NOT NULL, metricVal DOUBLE, CONSTRAINT PK PRIMARY KEY(metricId))"
+                    + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1, AUTO_PARTITION_SEQ=" + metricIdSeqTableName);
             // create view
             String ddl =
-                    "CREATE VIEW IF NOT EXISTS"
-                            + " view1( hostName varchar NOT NULL,"
+                    "CREATE VIEW IF NOT EXISTS "
+                            + viewName + "( hostName varchar NOT NULL,"
                             + " CONSTRAINT HOSTNAME_PK PRIMARY KEY (hostName))"
-                            + " AS SELECT * FROM metric_table"
+                            + " AS SELECT * FROM " + metricTableName
                             + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=300000";
             conn1.createStatement().execute(ddl);
             
             // drop the table using a different connection
-            conn2.createStatement().execute("DROP VIEW view1");
+            conn2.createStatement().execute("DROP VIEW " + viewName);
             
             // upsert one row
-            conn1.createStatement().execute("UPSERT INTO view1(hostName, metricVal) VALUES('host1', 1.0)");
+            conn1.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal) VALUES('host1', 1.0)");
             // upsert doesn't fail since base table still exists
             conn1.commit();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
index b7f5f88..3f02448 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
@@ -656,8 +656,9 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeTableReuseIT {
     }
     @Test
     public void testSumDouble() throws Exception {
-        initSumDoubleValues(null, getUrl());
-        String query = "SELECT SUM(d) FROM SumDoubleTest";
+        String tableName = "TBL_" + generateRandomString();
+        initSumDoubleValues(tableName, null, getUrl());
+        String query = "SELECT SUM(d) FROM " + tableName ;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -673,8 +674,9 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @Test
     public void testSumUnsignedDouble() throws Exception {
-        initSumDoubleValues(null, getUrl());
-        String query = "SELECT SUM(ud) FROM SumDoubleTest";
+        String tableName = "TBL_" + generateRandomString();
+        initSumDoubleValues(tableName, null, getUrl());
+        String query = "SELECT SUM(ud) FROM " + tableName ;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -690,8 +692,9 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @Test
     public void testSumFloat() throws Exception {
-        initSumDoubleValues(null, getUrl());
-        String query = "SELECT SUM(f) FROM SumDoubleTest";
+        String tableName = "TBL_" + generateRandomString();
+        initSumDoubleValues(tableName, null, getUrl());
+        String query = "SELECT SUM(f) FROM " + tableName ;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -707,8 +710,9 @@ public class ArithmeticQueryIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @Test
     public void testSumUnsignedFloat() throws Exception {
-        initSumDoubleValues(null, getUrl());
-        String query = "SELECT SUM(uf) FROM SumDoubleTest";
+        String tableName = "TBL_" + generateRandomString();
+        initSumDoubleValues(tableName, null, getUrl());
+        String query = "SELECT SUM(uf) FROM " + tableName;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {


[20/50] [abbrv] phoenix git commit: PHOENIX-2995 Write performance severely degrades with large number of views (addendum)

Posted by ma...@apache.org.
PHOENIX-2995 Write performance severely degrades with large number of views (addendum)


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

Branch: refs/heads/calcite
Commit: 386cbbbf7a5dd736888e8e0bfe16e513d54e215c
Parents: 3130fa9
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Wed Aug 17 11:55:12 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Aug 17 11:56:46 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java   | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/386cbbbf/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 0ebe67f..7bd4b9a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -113,8 +113,6 @@ import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.tephra.TransactionContext;
-import org.cloudera.htrace.Sampler;
-import org.cloudera.htrace.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;


[02/50] [abbrv] phoenix git commit: PHOENIX-3136 Do not relocate org.apache.calcite in queryserver/thin-client

Posted by ma...@apache.org.
PHOENIX-3136 Do not relocate org.apache.calcite in queryserver/thin-client

Relocation of these classes breaks compatibility with previous versions
of Phoenix.


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

Branch: refs/heads/calcite
Commit: 7febdfa881d6a0d41efc289968c8df30953ad39e
Parents: a9ea8a3
Author: Josh Elser <el...@apache.org>
Authored: Tue Aug 2 11:51:37 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 2 18:42:27 2016 -0400

----------------------------------------------------------------------
 phoenix-queryserver-client/pom.xml | 7 +++----
 phoenix-queryserver/pom.xml        | 7 +++----
 2 files changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7febdfa8/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index 8b2ecc8..ecc9edd 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -113,10 +113,9 @@
                   <shadedPattern>${shaded.package}.com.fasterxml</shadedPattern>
                 </relocation>
                 <!-- ORG relocation -->
-                <relocation>
-                  <pattern>org.apache.calcite</pattern>
-                  <shadedPattern>${shaded.package}.org.apache.calcite</shadedPattern>
-                </relocation>
+                <!-- Calcite/Avatica is not relocated because the wire API (as of <=1.8.0) expects
+                     consistent class names on client and server. Relocating these would break
+                     backwards compatibility. -->
                 <relocation>
                   <pattern>org.apache.commons</pattern>
                   <shadedPattern>${shaded.package}.org.apache.commons</shadedPattern>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7febdfa8/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index f09e1cb..1c186a1 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -105,10 +105,9 @@
                   <pattern>org.eclipse.jetty</pattern>
                   <shadedPattern>${shaded.package}.org.eclipse.jetty</shadedPattern>
                 </relocation>
-                <relocation>
-                  <pattern>org.apache.calcite</pattern>
-                  <shadedPattern>${shaded.package}.org.apache.calcite</shadedPattern>
-                </relocation>
+                <!-- Calcite/Avatica is not relocated because the wire API (as of <=1.8.0) expects
+                     consistent class names on client and server. Relocating these would break
+                     backwards compatibility. -->
               </relocations>
             </configuration>
           </execution>


[11/50] [abbrv] phoenix git commit: PHOENIX-3156 DistinctPrefixFilter optimization produces incorrect results with some non-pk WHERE conditions.

Posted by ma...@apache.org.
PHOENIX-3156 DistinctPrefixFilter optimization produces incorrect results with some non-pk WHERE conditions.


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

Branch: refs/heads/calcite
Commit: 60b95f962e75d23553d53bc1f00373299b019e1b
Parents: 137eb98
Author: Lars Hofhansl <la...@apache.org>
Authored: Fri Aug 5 21:39:56 2016 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Fri Aug 5 21:41:28 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DistinctPrefixFilterIT.java    | 7 ++++---
 .../java/org/apache/phoenix/iterate/BaseResultIterators.java  | 2 +-
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/60b95f96/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
index 9d31070..1a0e4e1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
@@ -171,8 +171,6 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         testPlan("SELECT COUNT(*) FROM (SELECT DISTINCT(prefix1) FROM "+testTable+")", true);
         testPlan("SELECT /*+ RANGE_SCAN */ DISTINCT prefix1 FROM "+testTable, false);
         testPlan("SELECT DISTINCT prefix1, prefix2 FROM "+testTable, true);
-        // use the filter even when the boolean expression filter is used
-        testPlan("SELECT DISTINCT prefix1, prefix2 FROM "+testTable+ " WHERE col1 > 0.5", true);
         // do not use the filter when the distinct is on the entire key
         testPlan("SELECT DISTINCT prefix1, prefix2, prefix3 FROM "+testTable, false);
         testPlan("SELECT DISTINCT (prefix1, prefix2, prefix3) FROM "+testTable, false);
@@ -193,7 +191,6 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         testPlan("SELECT (prefix1, prefix2, prefix3) FROM "+testTable+" GROUP BY (prefix1, prefix2, prefix3)", false);
         testPlan("SELECT prefix1, 1, 2 FROM "+testTable+" GROUP BY prefix1", true);
         testPlan("SELECT prefix1 FROM "+testTable+" GROUP BY prefix1, col1", false);
-        testPlan("SELECT DISTINCT prefix1, prefix2 FROM "+testTable+" WHERE col1 > 0.5", true);
 
         testPlan("SELECT COUNT(DISTINCT prefix1) FROM "+testTable+" HAVING COUNT(col1) > 10", false);
         testPlan("SELECT COUNT(DISTINCT prefix1) FROM "+testTable+" ORDER BY COUNT(col1)", true);
@@ -204,6 +201,8 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         testPlan("SELECT COUNT(DISTINCT prefix1) FROM "+testTable+" HAVING COUNT(DISTINCT prefix2) > 10", false);
         testPlan("SELECT COUNT(DISTINCT prefix1) FROM "+testTable+" HAVING COUNT(DISTINCT prefix1) > 10", false);
         testPlan("SELECT COUNT(DISTINCT prefix1) / 10 FROM "+testTable, false);
+        // do not use the filter when the boolean expression filter is used
+        testPlan("SELECT DISTINCT prefix1, prefix2 FROM "+testTable+" WHERE col1 > 0.5", false);
     }
 
     private void testPlan(String query, boolean optimizable) throws Exception {
@@ -220,6 +219,7 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         testSkipRange("SELECT %s prefix1 FROM "+ testTableF + " GROUP BY prefix1, prefix2 HAVING prefix2 = 2147483647", 2);
         testSkipRange("SELECT %s prefix1 FROM "+ testTableF + " GROUP BY prefix1, prefix2 HAVING prefix1 = 2147483647", 1);
         testSkipRange("SELECT %s prefix1 FROM "+ testTableF + " WHERE col1 > 0.99 GROUP BY prefix1, prefix2 HAVING prefix2 = 2", -1);
+        testSkipRange("SELECT %s prefix1 FROM "+ testTableF + " WHERE col1 >=0 and col2 > 990 GROUP BY prefix1, prefix2 HAVING prefix2 = 2", -1);
 
         testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " GROUP BY prefix1, prefix2 HAVING prefix1 IN ('1','2')", 6);
         testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " GROUP BY prefix1, prefix2 HAVING prefix1 IN ('1','2') AND prefix2 IN ('1','2')", 4);
@@ -228,6 +228,7 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " GROUP BY prefix1, prefix2 HAVING prefix2 = '22'", 1);
         testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " GROUP BY prefix1, prefix2 HAVING prefix1 = '22'", 1);
         testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " WHERE col1 > 0.99 GROUP BY prefix1, prefix2 HAVING prefix2 = '2'", -1);
+        testSkipRange("SELECT %s prefix1 FROM "+ testTableV + " WHERE col1 >= 0 and col2 > 990 GROUP BY prefix1, prefix2 HAVING prefix2 = '2'", -1);
 
         testCommonGroupBy(testTableF);
         testCommonGroupBy(testTableV);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/60b95f96/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index d0ade72..ceba000 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -227,7 +227,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             }
 
             int cols = plan.getGroupBy().getOrderPreservingColumnCount();
-            if (cols > 0 &&
+            if (cols > 0 && context.getWhereConditionColumns().size() == 0 &&
                 !plan.getStatement().getHint().hasHint(HintNode.Hint.RANGE_SCAN) &&
                 cols < plan.getTableRef().getTable().getRowKeySchema().getFieldCount() &&
                 plan.getGroupBy().isOrderPreserving() &&


[07/50] [abbrv] phoenix git commit: PHOENIX-3111 Possible Deadlock/delay while building index, upsert select, delete rows at server-addendum(Rajeshbabu)

Posted by ma...@apache.org.
PHOENIX-3111 Possible Deadlock/delay while building index, upsert select, delete rows at server-addendum(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 27c4027fd72cec790975c810724f3a778388e426
Parents: 50b3f94
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed Aug 3 11:01:21 2016 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Wed Aug 3 11:01:21 2016 +0530

----------------------------------------------------------------------
 .../UngroupedAggregateRegionObserver.java       | 38 +++++++++-----------
 1 file changed, 17 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/27c4027f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index eda59d1..a7c6fde 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -187,28 +187,22 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
               m.setAttribute(PhoenixIndexCodec.INDEX_UUID, indexUUID);
           }
       }
+      
       Mutation[] mutationArray = new Mutation[mutations.size()];
+      // When memstore size reaches blockingMemstoreSize we are waiting 3 seconds for the
+      // flush happen which decrease the memstore size and then writes allowed on the region.
+      for (int i = 0; region.getMemstoreSize() > blockingMemstoreSize && i < 30; i++) {
+          try {
+              checkForRegionClosing();
+              Thread.sleep(100);
+          } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+              throw new IOException(e);
+          }
+      }
       // TODO: should we use the one that is all or none?
       logger.debug("Committing bactch of " + mutations.size() + " mutations for " + region.getRegionInfo().getTable().getNameAsString());
-      try {
-          region.batchMutate(mutations.toArray(mutationArray), HConstants.NO_NONCE, HConstants.NO_NONCE);
-      } catch (RegionTooBusyException rtbe) {
-            // When memstore size reaches blockingMemstoreSize we are waiting 3 seconds for the
-            // flush happen which decrease the memstore size and then writes allowed on the region.
-            for (int i = 0; region.getMemstoreSize() > blockingMemstoreSize && i < 30; i++) {
-                try {
-                    checkForRegionClosing();
-                    Thread.sleep(100);
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                    throw new IOException(e);
-                }
-            }
-            if (region.getMemstoreSize() > blockingMemstoreSize) {
-                throw rtbe;
-            }
-            region.batchMutate(mutationArray, HConstants.NO_NONCE, HConstants.NO_NONCE);
-      }
+      region.batchMutate(mutations.toArray(mutationArray), HConstants.NO_NONCE, HConstants.NO_NONCE);
     }
 
     /**
@@ -354,8 +348,10 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         }
 
         /**
-         * Upper bound of memstore size allowed for region. Updates will be blocked until the flush
-         * happen if the memstore reaches this threshold.
+         * Slow down the writes if the memstore size more than
+         * (hbase.hregion.memstore.block.multiplier - 1) times hbase.hregion.memstore.flush.size
+         * bytes. This avoids flush storm to hdfs for cases like index building where reads and
+         * write happen to all the table regions in the server.
          */
         final long blockingMemStoreSize = flushSize * (
                 conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,


[15/50] [abbrv] phoenix git commit: PHOENIX-2236 PHOENIX-2290 PHOENIX-2547 Various phoenix-spark fixes (Kalyan Hadoop)

Posted by ma...@apache.org.
PHOENIX-2236 PHOENIX-2290 PHOENIX-2547 Various phoenix-spark fixes (Kalyan Hadoop)


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

Branch: refs/heads/calcite
Commit: 2afb16dc2032f2be9de220946e97f87336218e80
Parents: ba82b1c
Author: Josh Mahonin <jm...@interset.com>
Authored: Mon Aug 15 11:55:56 2016 -0400
Committer: Josh Mahonin <jm...@interset.com>
Committed: Mon Aug 15 11:58:41 2016 -0400

----------------------------------------------------------------------
 phoenix-spark/src/it/resources/setup.sql        |  6 +++
 .../apache/phoenix/spark/PhoenixSparkIT.scala   | 49 ++++++++++++++++++++
 .../apache/phoenix/spark/PhoenixRelation.scala  | 26 ++++++-----
 3 files changed, 70 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2afb16dc/phoenix-spark/src/it/resources/setup.sql
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/resources/setup.sql b/phoenix-spark/src/it/resources/setup.sql
index aa2cee1..e56924f 100644
--- a/phoenix-spark/src/it/resources/setup.sql
+++ b/phoenix-spark/src/it/resources/setup.sql
@@ -48,3 +48,9 @@ CREATE TABLE TEST_SMALL_TINY (ID BIGINT NOT NULL PRIMARY KEY, COL1 SMALLINT, COL
 UPSERT INTO TEST_SMALL_TINY VALUES (1, 32767, 127)
 CREATE TABLE DATE_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 DATE)
 UPSERT INTO DATE_TEST VALUES(1, CURRENT_DATE())
+CREATE TABLE "space" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR)
+UPSERT INTO "space" VALUES ('key1', 'xyz')
+CREATE TABLE "small" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR, "salary" INTEGER )
+UPSERT INTO "small" VALUES ('key1', 'foo', 10000)
+UPSERT INTO "small" VALUES ('key2', 'bar', 20000)
+UPSERT INTO "small" VALUES ('key3', 'xyz', 30000)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2afb16dc/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index c216406..7d05f07 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -621,6 +621,55 @@ class PhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfterAll {
     assert(Math.abs(epoch - dt) < 86400000)
   }
 
+  test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") {
+    val sqlContext = new SQLContext(sc)
+    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("space"),
+      "zkUrl" -> quorumAddress))
+    val res = df.filter(df.col("first name").equalTo("xyz"))
+    // Make sure we got the right value back
+    assert(res.collectAsList().size() == 1L)
+  }
+
+  test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") {
+    val sqlContext = new SQLContext(sc)
+    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("small"),
+      "zkUrl" -> quorumAddress))
+    df.registerTempTable("temp")
+
+    // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions
+    // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle
+
+    val res1 = sqlContext.sql("select * from temp where salary = '10000' ")
+    assert(res1.collectAsList().size() == 1L)
+
+    val res2 = sqlContext.sql("select * from temp where \"salary\" = '10000' ")
+    assert(res2.collectAsList().size() == 0L)
+
+    val res3 = sqlContext.sql("select * from temp where salary > '10000' ")
+    assert(res3.collectAsList().size() == 2L)
+  }
+
+  test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") {
+    val sqlContext = new SQLContext(sc)
+    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("small"),
+      "zkUrl" -> quorumAddress))
+
+    // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions
+    // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle
+
+    val res1 = df.filter(df.col("first name").equalTo("foo"))
+    assert(res1.collectAsList().size() == 1L)
+
+    val res2 = df.filter("\"first name\" = 'foo'")
+    assert(res2.collectAsList().size() == 0L)
+
+    val res3 = df.filter("salary = '10000'")
+    assert(res3.collectAsList().size() == 1L)
+
+    val res4 = df.filter("salary > '10000'")
+    assert(res4.collectAsList().size() == 2L)
+  }
+
   test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") {
     val sqlContext = new SQLContext(sc)
     val df = sqlContext.read

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2afb16dc/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
index 8d7f9f7..d2eac8c 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{Row, SQLContext}
 import org.apache.spark.sql.sources._
 import org.apache.phoenix.util.StringUtil.escapeStringConstant
+import org.apache.phoenix.util.SchemaUtil
 
 case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Boolean = false)(@transient val sqlContext: SQLContext)
     extends BaseRelation with PrunedFilteredScan {
@@ -80,17 +81,17 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
         case And(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter, rightFilter)))
         case Or(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter)) + " OR " + buildFilter(Array(rightFilter)))
         case Not(aFilter) => filter.append(" NOT " + buildFilter(Array(aFilter)))
-        case EqualTo(attr, value) => filter.append(s" $attr = ${compileValue(value)}")
-        case GreaterThan(attr, value) => filter.append(s" $attr > ${compileValue(value)}")
-        case GreaterThanOrEqual(attr, value) => filter.append(s" $attr >= ${compileValue(value)}")
-        case LessThan(attr, value) => filter.append(s" $attr < ${compileValue(value)}")
-        case LessThanOrEqual(attr, value) => filter.append(s" $attr <= ${compileValue(value)}")
-        case IsNull(attr) => filter.append(s" $attr IS NULL")
-        case IsNotNull(attr) => filter.append(s" $attr IS NOT NULL")
-        case In(attr, values) => filter.append(s" $attr IN ${values.map(compileValue).mkString("(", ",", ")")}")
-        case StringStartsWith(attr, value) => filter.append(s" $attr LIKE ${compileValue(value + "%")}")
-        case StringEndsWith(attr, value) => filter.append(s" $attr LIKE ${compileValue("%" + value)}")
-        case StringContains(attr, value) => filter.append(s" $attr LIKE ${compileValue("%" + value + "%")}")
+        case EqualTo(attr, value) => filter.append(s" ${escapeKey(attr)} = ${compileValue(value)}")
+        case GreaterThan(attr, value) => filter.append(s" ${escapeKey(attr)} > ${compileValue(value)}")
+        case GreaterThanOrEqual(attr, value) => filter.append(s" ${escapeKey(attr)} >= ${compileValue(value)}")
+        case LessThan(attr, value) => filter.append(s" ${escapeKey(attr)} < ${compileValue(value)}")
+        case LessThanOrEqual(attr, value) => filter.append(s" ${escapeKey(attr)} <= ${compileValue(value)}")
+        case IsNull(attr) => filter.append(s" ${escapeKey(attr)} IS NULL")
+        case IsNotNull(attr) => filter.append(s" ${escapeKey(attr)} IS NOT NULL")
+        case In(attr, values) => filter.append(s" ${escapeKey(attr)} IN ${values.map(compileValue).mkString("(", ",", ")")}")
+        case StringStartsWith(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue(value + "%")}")
+        case StringEndsWith(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue("%" + value)}")
+        case StringContains(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue("%" + value + "%")}")
       }
 
       i = i + 1
@@ -99,6 +100,9 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
     filter.toString()
   }
 
+  // Helper function to escape column key to work with SQL queries
+  private def escapeKey(key: String): String = SchemaUtil.getEscapedArgument(key)
+
   // Helper function to escape string values in SQL queries
   private def compileValue(value: Any): Any = value match {
     case stringValue: String => s"'${escapeStringConstant(stringValue)}'"


[40/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
index cf45724..17c6b72 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayAppendFunctionIT.java
@@ -26,11 +26,12 @@ import java.sql.*;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.junit.Test;
 
-public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
-    private void initTables(Connection conn) throws Exception {
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
+public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
+    private String initTables(Connection conn) throws Exception {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO regions(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
+        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
                 "ARRAY['2345','46345','23234']," +
                 "ARRAY[2345,46345,23234,456]," +
                 "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
@@ -44,12 +45,14 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.execute();
         conn.commit();
+        return tableName;
     }
 
-    private void initTablesDesc(Connection conn, String type, String val) throws Exception {
-        String ddl = "CREATE TABLE regions (pk " + type + " PRIMARY KEY DESC,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],chars2 CHAR(15)[], bools BOOLEAN[])";
+    private String initTablesDesc(Connection conn, String type, String val) throws Exception {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (pk " + type + " PRIMARY KEY DESC,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],chars2 CHAR(15)[], bools BOOLEAN[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO regions(pk,varchars,integers,doubles,bigints,chars,chars2,bools) VALUES(" + val + "," +
+        String dml = "UPSERT INTO " + tableName + "(pk,varchars,integers,doubles,bigints,chars,chars2,bools) VALUES(" + val + "," +
                 "ARRAY['2345','46345','23234']," +
                 "ARRAY[2345,46345,23234,456]," +
                 "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
@@ -61,15 +64,17 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.execute();
         conn.commit();
+        return tableName;
+
     }
 
     @Test
     public void testArrayAppendFunctionVarchar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,'34567') FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,'34567') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"2345", "46345", "23234", "34567"};
@@ -83,10 +88,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInteger() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,1234) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,1234) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456, 1234};
@@ -100,10 +105,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionDouble() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,double1) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,double1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 23.45};
@@ -117,10 +122,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionDouble2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,23) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,23) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, new Double(23)};
@@ -134,9 +139,9 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionBigint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bigints,1112) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bigints,1112) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Long[] longs = new Long[]{12l, 34l, 56l, 78l, 910l, 1112l};
@@ -150,9 +155,9 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionChar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,'fac') FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,'fac') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"a", "bbbb", "c", "ddd", "e", "fac"};
@@ -166,28 +171,28 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test(expected = TypeMismatchException.class)
     public void testArrayAppendFunctionIntToCharArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,234) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,234) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
     }
 
     @Test(expected = TypeMismatchException.class)
     public void testArrayAppendFunctionVarcharToIntegerArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,'234') FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,'234') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
 
     }
 
     @Test(expected = SQLException.class)
     public void testArrayAppendFunctionChar2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,'facfacfacfacfacfacfac') FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,'facfacfacfacfacfacfac') FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         rs.next();
         rs.getArray(1);
     }
@@ -195,10 +200,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionIntegerToDoubleArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,45) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,45) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 45.0};
@@ -212,10 +217,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionWithNestedFunctions1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(ARRAY[23,45],integers[1]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(ARRAY[23,45],integers[1]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{23, 45, 2345};
@@ -229,10 +234,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionWithNestedFunctions2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,ARRAY_ELEM(ARRAY[2,4],1)) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,ARRAY_ELEM(ARRAY[2,4],1)) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456, 2};
@@ -246,10 +251,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionWithNestedFunctions3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,ARRAY_ELEM(doubles,2)) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,ARRAY_ELEM(doubles,2)) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 46.345};
@@ -263,16 +268,16 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionWithUpsert1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,varchars) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY['hello','world'],':-)'))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,varchars) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY['hello','world'],':-)'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"hello", "world", ":-)"};
@@ -287,15 +292,16 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayAppendFunctionWithUpsert2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,integers) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[4,5],6))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,integers) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[4,5],6))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT integers FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT integers FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{4, 5, 6};
@@ -310,15 +316,16 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayAppendFunctionWithUpsert3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,doubles) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[5.67,7.87],9.0))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[5.67,7.87],9.0))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{5.67, 7.87, new Double(9)};
@@ -333,25 +340,28 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayAppendFunctionWithUpsertSelect1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
 
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String sourceTableName = generateRandomString();
+        String targetTableName = generateRandomString();
+
+        String ddl = "CREATE TABLE " + sourceTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        ddl = "CREATE TABLE " + targetTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[5.67,7.87],9.0))";
+        String dml = "UPSERT INTO " + sourceTableName + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY[5.67,7.87],9.0))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area2',ARRAY_APPEND(ARRAY[56.7,7.87],9.2))";
+        dml = "UPSERT INTO " + sourceTableName + "(region_name,doubles) VALUES('SF Bay Area2',ARRAY_APPEND(ARRAY[56.7,7.87],9.2))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, doubles) SELECT region_name, ARRAY_APPEND(doubles,5) FROM source";
+        dml = "UPSERT INTO " + targetTableName + "(region_name, doubles) SELECT region_name, ARRAY_APPEND(doubles,5) FROM " + sourceTableName ;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM target");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + targetTableName );
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{5.67, 7.87, new Double(9), new Double(5)};
@@ -371,25 +381,27 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayAppendFunctionWithUpsertSelect2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
 
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String sourceTableName = generateRandomString();
+        String targetTableName = generateRandomString();
+        String ddl = "CREATE TABLE " + sourceTableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        ddl = "CREATE TABLE " + targetTableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY['abcd','b'],'c'))";
+        String dml = "UPSERT INTO " + sourceTableName + "(region_name,varchars) VALUES('SF Bay Area',ARRAY_APPEND(ARRAY['abcd','b'],'c'))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area2',ARRAY_APPEND(ARRAY['d','fgh'],'something'))";
+        dml = "UPSERT INTO " + sourceTableName + "(region_name,varchars) VALUES('SF Bay Area2',ARRAY_APPEND(ARRAY['d','fgh'],'something'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, varchars) SELECT region_name, ARRAY_APPEND(varchars,'stu') FROM source";
+        dml = "UPSERT INTO " + targetTableName + "(region_name, varchars) SELECT region_name, ARRAY_APPEND(varchars,'stu') FROM " + sourceTableName ;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM target");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + targetTableName );
         assertTrue(rs.next());
 
         String[] strings = new String[]{"abcd", "b", "c", "stu"};
@@ -408,10 +420,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[2345,46345,23234,456,123]=ARRAY_APPEND(integers,123)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[2345,46345,23234,456,123]=ARRAY_APPEND(integers,123)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -421,10 +433,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE varchars[1]=ANY(ARRAY_APPEND(ARRAY['2345','46345','23234'],'1234'))");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE varchars[1]=ANY(ARRAY_APPEND(ARRAY['2345','46345','23234'],'1234'))");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -434,10 +446,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['2345','46345','23234','1234']=ARRAY_APPEND(ARRAY['2345','46345','23234'],'1234')");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['2345','46345','23234','1234']=ARRAY_APPEND(ARRAY['2345','46345','23234'],'1234')");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -447,10 +459,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere4() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[23.45,4634.5,2.3234,123.4]=ARRAY_APPEND(ARRAY[23.45,4634.5,2.3234],123.4)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[23.45,4634.5,2.3234,123.4]=ARRAY_APPEND(ARRAY[23.45,4634.5,2.3234],123.4)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -460,10 +472,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere5() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['2345','46345','23234','foo']=ARRAY_APPEND(varchars,'foo')");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['2345','46345','23234','foo']=ARRAY_APPEND(varchars,'foo')");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -473,10 +485,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere6() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE chars2=ARRAY_APPEND(chars,'foo')");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE chars2=ARRAY_APPEND(chars,'foo')");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -486,10 +498,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionInWhere7() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[2,3,4]=ARRAY_APPEND(ARRAY[2,3],4)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[2,3,4]=ARRAY_APPEND(ARRAY[2,3],4)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -499,10 +511,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionVarcharWithNull() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,NULL) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"2345", "46345", "23234"};
@@ -516,10 +528,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionDoublesWithNull() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,NULL) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(doubles,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78};
@@ -533,10 +545,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionCharsWithNull() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,NULL) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,NULL) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"a", "bbbb", "c", "ddd", "e"};
@@ -550,10 +562,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionWithNull() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,nullcheck) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,nullcheck) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456};
@@ -567,10 +579,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test(expected = SQLException.class)
     public void testArrayAppendFunctionCharLimitCheck() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,char1) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(chars,char1) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"a", "bbbb", "c", "ddd", "e", "wert"};
@@ -584,10 +596,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionIntegerDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "INTEGER", "23");
+        String tableName = initTablesDesc(conn, "INTEGER", "23");
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,pk) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(integers,pk) FROM " + tableName + "");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456, 23};
@@ -602,10 +614,10 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionVarcharDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "VARCHAR", "'e'");
+        String tableName = initTablesDesc(conn, "VARCHAR", "'e'");
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,pk) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(varchars,pk) FROM " + tableName + "");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"2345", "46345", "23234", "e"};
@@ -619,9 +631,9 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionBigIntDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "BIGINT", "1112");
+        String tableName = initTablesDesc(conn, "BIGINT", "1112");
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bigints,pk) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bigints,pk) FROM " + tableName );
         assertTrue(rs.next());
 
         Long[] longs = new Long[]{12l, 34l, 56l, 78l, 910l, 1112l};
@@ -635,9 +647,9 @@ public class ArrayAppendFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayAppendFunctionBooleanDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "BOOLEAN", "false");
+        String tableName = initTablesDesc(conn, "BOOLEAN", "false");
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bools,pk) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_APPEND(bools,pk) FROM " + tableName );
         assertTrue(rs.next());
 
         Boolean[] booleans = new Boolean[]{true, false, false};

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayConcatFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayConcatFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayConcatFunctionIT.java
index 247bfb7..168000e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayConcatFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayConcatFunctionIT.java
@@ -26,12 +26,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
+public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
-    private void initTables(Connection conn) throws Exception {
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
+    private String initTables(Connection conn) throws Exception {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO regions(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
+        String dml = "UPSERT INTO " + tableName + "(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
                 "ARRAY['2345','46345','23234']," +
                 "ARRAY[2345,46345,23234,456]," +
                 "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
@@ -45,15 +46,16 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.execute();
         conn.commit();
+        return tableName;
     }
 
     @Test
     public void testArrayConcatFunctionVarchar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(varchars,varchars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(varchars,varchars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"2345", "46345", "23234", "2345", "46345", "23234"};
@@ -67,10 +69,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInteger() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,integers) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,integers) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456, 2345, 46345, 23234, 456};
@@ -84,10 +86,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionDouble() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,doubles) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,doubles) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 23.45, 46.345, 23.234, 45.6, 5.78};
@@ -101,10 +103,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionDouble2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[23]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[23]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, new Double(23)};
@@ -118,9 +120,9 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionBigint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(bigints,bigints) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(bigints,bigints) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Long[] longs = new Long[]{12l, 34l, 56l, 78l, 910l, 12l, 34l, 56l, 78l, 910l};
@@ -134,9 +136,9 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionChar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,chars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,chars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"a", "bbbb", "c", "ddd", "e", "a", "bbbb", "c", "ddd", "e"};
@@ -150,9 +152,9 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionChar3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,chars2) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,chars2) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"a", "bbbb", "c", "ddd", "e", "a", "bbbb", "c", "ddd", "e", "foo"};
@@ -166,28 +168,28 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test(expected = TypeMismatchException.class)
     public void testArrayConcatFunctionIntToCharArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(varchars,ARRAY[23,45]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(varchars,ARRAY[23,45]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
     }
 
     @Test(expected = TypeMismatchException.class)
     public void testArrayConcatFunctionVarcharToIntegerArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,ARRAY['a', 'b']) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,ARRAY['a', 'b']) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
 
     }
 
     @Test(expected = SQLException.class)
     public void testArrayConcatFunctionChar2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,ARRAY['facfacfacfacfacfacfac','facfacfacfacfacfacfac']) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(chars,ARRAY['facfacfacfacfacfacfac','facfacfacfacfacfacfac']) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         rs.next();
         rs.getArray(1);
     }
@@ -195,10 +197,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionIntegerArrayToDoubleArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[45, 55]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[45, 55]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 45.0, 55.0};
@@ -212,10 +214,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNestedFunctions1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(ARRAY[23,45],ARRAY[integers[1],integers[1]]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(ARRAY[23,45],ARRAY[integers[1],integers[1]]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{23, 45, 2345, 2345};
@@ -229,10 +231,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNestedFunctions2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,ARRAY[ARRAY_ELEM(ARRAY[2,4],1),ARRAY_ELEM(ARRAY[2,4],2)]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(integers,ARRAY[ARRAY_ELEM(ARRAY[2,4],1),ARRAY_ELEM(ARRAY[2,4],2)]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 46345, 23234, 456, 2, 4};
@@ -246,10 +248,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNestedFunctions3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[ARRAY_ELEM(doubles, 1), ARRAY_ELEM(doubles, 1)]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_CAT(doubles,ARRAY[ARRAY_ELEM(doubles, 1), ARRAY_ELEM(doubles, 1)]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 46.345, 23.234, 45.6, 5.78, 23.45, 23.45};
@@ -263,16 +265,17 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithUpsert1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,varchars) VALUES('SF Bay Area',ARRAY_CAT(ARRAY['hello','world'],ARRAY[':-)']))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,varchars) VALUES('SF Bay Area',ARRAY_CAT(ARRAY['hello','world'],ARRAY[':-)']))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"hello", "world", ":-)"};
@@ -286,16 +289,17 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithUpsert2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,integers) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[4,5],ARRAY[6, 7]))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,integers) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[4,5],ARRAY[6, 7]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT integers FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT integers FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{4, 5, 6, 7};
@@ -309,16 +313,17 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithUpsert3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,doubles) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[5.67,7.87],ARRAY[9.0, 8.0]))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[5.67,7.87],ARRAY[9.0, 8.0]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{5.67, 7.87, new Double(9), new Double(8)};
@@ -333,25 +338,27 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayConcatFunctionWithUpsertSelect1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
 
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String sourceTableName = generateRandomString();
+        String targetTableName = generateRandomString();
+        String ddl = "CREATE TABLE " + sourceTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        ddl = "CREATE TABLE " + targetTableName + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[5.67,7.87],ARRAY[9.0, 4.0]))";
+        String dml = "UPSERT INTO " + sourceTableName + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_CAT(ARRAY[5.67,7.87],ARRAY[9.0, 4.0]))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area2',ARRAY_CAT(ARRAY[56.7,7.87],ARRAY[9.2, 3.4]))";
+        dml = "UPSERT INTO " + sourceTableName + "(region_name,doubles) VALUES('SF Bay Area2',ARRAY_CAT(ARRAY[56.7,7.87],ARRAY[9.2, 3.4]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, doubles) SELECT region_name, ARRAY_CAT(doubles,doubles) FROM source";
+        dml = "UPSERT INTO " + targetTableName + "(region_name, doubles) SELECT region_name, ARRAY_CAT(doubles,doubles) FROM " + sourceTableName ;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM target");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + targetTableName );
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{5.67, 7.87, new Double(9), new Double(4), 5.67, 7.87, new Double(9), new Double(4)};
@@ -370,26 +377,27 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithUpsertSelect2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String sourceTableName = generateRandomString();
+        String targetTableName = generateRandomString();
+        String ddl = "CREATE TABLE " + sourceTableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        ddl = "CREATE TABLE " + targetTableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area',ARRAY_CAT(ARRAY['abcd','b'],ARRAY['c', 'd']))";
+        String dml = "UPSERT INTO " + sourceTableName + "(region_name,varchars) VALUES('SF Bay Area',ARRAY_CAT(ARRAY['abcd','b'],ARRAY['c', 'd']))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area2',ARRAY_CAT(ARRAY['d','fgh'],ARRAY['something','something']))";
+        dml = "UPSERT INTO " + sourceTableName + "(region_name,varchars) VALUES('SF Bay Area2',ARRAY_CAT(ARRAY['d','fgh'],ARRAY['something','something']))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, varchars) SELECT region_name, ARRAY_CAT(varchars,varchars) FROM source";
+        dml = "UPSERT INTO " + targetTableName + "(region_name, varchars) SELECT region_name, ARRAY_CAT(varchars,varchars) FROM " + sourceTableName ;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM target");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + targetTableName );
         assertTrue(rs.next());
 
         String[] strings = new String[]{"abcd", "b", "c", "d", "abcd", "b", "c", "d"};
@@ -408,10 +416,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[2345,46345,23234,456,123]=ARRAY_CAT(integers,ARRAY[123])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[2345,46345,23234,456,123]=ARRAY_CAT(integers,ARRAY[123])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -421,10 +429,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE varchars[1]=ANY(ARRAY_CAT(ARRAY['2345','46345','23234'],ARRAY['1234']))");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE varchars[1]=ANY(ARRAY_CAT(ARRAY['2345','46345','23234'],ARRAY['1234']))");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -434,10 +442,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['2345','46345','23234','1234','234']=ARRAY_CAT(ARRAY['2345','46345','23234'],ARRAY['1234', '234'])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['2345','46345','23234','1234','234']=ARRAY_CAT(ARRAY['2345','46345','23234'],ARRAY['1234', '234'])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -447,10 +455,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere4() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[23.45,4634.5,2.3234,123.4,12.0]=ARRAY_CAT(ARRAY[23.45,4634.5,2.3234],ARRAY[123.4,12.0])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[23.45,4634.5,2.3234,123.4,12.0]=ARRAY_CAT(ARRAY[23.45,4634.5,2.3234],ARRAY[123.4,12.0])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -460,10 +468,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere5() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['2345','46345','23234','foo','foo']=ARRAY_CAT(varchars,ARRAY['foo','foo'])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['2345','46345','23234','foo','foo']=ARRAY_CAT(varchars,ARRAY['foo','foo'])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -473,10 +481,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere6() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE chars2=ARRAY_CAT(chars,ARRAY['foo'])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE chars2=ARRAY_CAT(chars,ARRAY['foo'])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -486,10 +494,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionInWhere7() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[2,3,4,5]=ARRAY_CAT(ARRAY[2,3],ARRAY[4,5])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[2,3,4,5]=ARRAY_CAT(ARRAY[2,3],ARRAY[4,5])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -499,10 +507,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNulls1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM regions WHERE region_name = 'SF Bay Area'");
+        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         Array array1 = conn.createArrayOf("VARCHAR", new Object[]{"a", "b", "c", null});
         st.setArray(1, array1);
         Array array2 = conn.createArrayOf("VARCHAR", new Object[]{"a", "b", "c"});
@@ -519,10 +527,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNulls2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM regions WHERE region_name = 'SF Bay Area'");
+        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         Array array1 = conn.createArrayOf("VARCHAR", new Object[]{"a", "b", "c"});
         st.setArray(1, array1);
         Array array2 = conn.createArrayOf("VARCHAR", new Object[]{null, "a", "b", "c"});
@@ -539,10 +547,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNulls3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM regions WHERE region_name = 'SF Bay Area'");
+        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         Array array1 = conn.createArrayOf("VARCHAR", new Object[]{"a", "b", "c", null});
         st.setArray(1, array1);
         Array array2 = conn.createArrayOf("VARCHAR", new Object[]{null, "a", "b", "c"});
@@ -559,10 +567,10 @@ public class ArrayConcatFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayConcatFunctionWithNulls4() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = initTables(conn);
 
         ResultSet rs;
-        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM regions WHERE region_name = 'SF Bay Area'");
+        PreparedStatement st = conn.prepareStatement("SELECT ARRAY_CAT(?,?) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         Array array1 = conn.createArrayOf("VARCHAR", new Object[]{null, "a", null, "b", "c", null, null});
         st.setArray(1, array1);
         Array array2 = conn.createArrayOf("VARCHAR", new Object[]{null, null, "a", null, "b", null, "c", null});


[27/50] [abbrv] phoenix git commit: PHOENIX-3203 Tenant cache lookup in Global Cache fails in certain conditions

Posted by ma...@apache.org.
PHOENIX-3203 Tenant cache lookup in Global Cache fails in certain conditions


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

Branch: refs/heads/calcite
Commit: 64cb9b0130188b1407093b8bc62447bd2ecf0d3c
Parents: 9cc77c8
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Aug 25 15:25:18 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Fri Aug 26 12:08:07 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/index/MutableIndexIT.java   | 25 +++++++++++++-------
 .../org/apache/phoenix/cache/GlobalCache.java   |  4 ++--
 .../cache/aggcache/SpillableGroupByCache.java   |  5 ++--
 .../phoenix/coprocessor/GroupByCache.java       |  5 ++--
 .../GroupedAggregateRegionObserver.java         | 15 ++++++------
 .../coprocessor/HashJoinRegionScanner.java      |  5 ++--
 .../phoenix/coprocessor/ScanRegionObserver.java |  9 ++++---
 .../phoenix/index/PhoenixIndexMetaData.java     |  3 +--
 .../java/org/apache/phoenix/util/ScanUtil.java  |  5 ++--
 9 files changed, 40 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 29057db..6a49076 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -824,9 +824,9 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
           conn.createStatement().execute(
               "CREATE TABLE IF NOT EXISTS " + fullTableName + 
               "(TENANT_ID CHAR(15) NOT NULL,"+
-              "TYPE VARCHAR(25) NOT NULL,"+
+              "TYPE VARCHAR(25),"+
               "ENTITY_ID CHAR(15) NOT NULL,"+
-              "CONSTRAINT PK_CONSTRAINT PRIMARY KEY (TENANT_ID, TYPE, ENTITY_ID)) MULTI_TENANT=TRUE "
+              "CONSTRAINT PK_CONSTRAINT PRIMARY KEY (TENANT_ID, ENTITY_ID)) MULTI_TENANT=TRUE "
               + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") );
           // create index
           conn.createStatement().execute("CREATE INDEX IF NOT EXISTS " + indexName + " ON " + fullTableName + " (ENTITY_ID, TYPE)");
@@ -836,14 +836,23 @@ public class MutableIndexIT extends BaseHBaseManagedTimeIT {
           props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "tenant1");
           // connection is tenant-specific
           try (Connection tenantConn = DriverManager.getConnection(getUrl(), props)) {
-              for (int i=0; i<2; ++i) {
-                  PreparedStatement stmt = tenantConn.prepareStatement(dml);
-                  stmt.setString(1, "00000000000000" + String.valueOf(i));
-                  stmt.setString(2, String.valueOf(i));
-                  assertEquals(1,stmt.executeUpdate());
-              }
+              // upsert one row
+              upsertRow(dml, tenantConn, 0);
+              tenantConn.commit();
+              ResultSet rs = tenantConn.createStatement().executeQuery("SELECT ENTITY_ID FROM " + fullTableName + " ORDER BY TYPE LIMIT 5");
+              assertTrue(rs.next());
+              // upsert two rows which ends up using the tenant cache
+              upsertRow(dml, tenantConn, 1);
+              upsertRow(dml, tenantConn, 2);
               tenantConn.commit();
           }
       }
   }
+
+private void upsertRow(String dml, Connection tenantConn, int i) throws SQLException {
+    PreparedStatement stmt = tenantConn.prepareStatement(dml);
+      stmt.setString(1, "00000000000000" + String.valueOf(i));
+      stmt.setString(2, String.valueOf(i));
+      assertEquals(1,stmt.executeUpdate());
+}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
index 319597c..0c3a87a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -136,7 +136,7 @@ public class GlobalCache extends TenantCacheImpl {
      * @param tenantId the tenant ID or null if not applicable.
      * @return TenantCache
      */
-    public static TenantCache getTenantCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId) {
+    public static TenantCache getTenantCache(RegionCoprocessorEnvironment env, ImmutableBytesPtr tenantId) {
         GlobalCache globalCache = GlobalCache.getInstance(env);
         TenantCache tenantCache = tenantId == null ? globalCache : globalCache.getChildTenantCache(tenantId);      
         return tenantCache;
@@ -165,7 +165,7 @@ public class GlobalCache extends TenantCacheImpl {
      * @param tenantId the ID that identifies the tenant
      * @return the existing or newly created TenantCache
      */
-    public TenantCache getChildTenantCache(ImmutableBytesWritable tenantId) {
+    public TenantCache getChildTenantCache(ImmutableBytesPtr tenantId) {
         TenantCache tenantCache = perTenantCacheMap.get(tenantId);
         if (tenantCache == null) {
             int maxTenantMemoryPerc = config.getInt(MAX_TENANT_MEMORY_PERC_ATTRIB, QueryServicesOptions.DEFAULT_MAX_TENANT_MEMORY_PERC);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
index 8edeb3a..dc0ae21 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
@@ -35,7 +35,6 @@ import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -126,7 +125,7 @@ public class SpillableGroupByCache implements GroupByCache {
      * @param aggs
      * @param ctxt
      */
-    public SpillableGroupByCache(final RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId,
+    public SpillableGroupByCache(final RegionCoprocessorEnvironment env, ImmutableBytesPtr tenantId,
             ServerAggregators aggs, final int estSizeNum) {
         totalNumElements = 0;
         this.aggregators = aggs;
@@ -217,7 +216,7 @@ public class SpillableGroupByCache implements GroupByCache {
      * implements an implicit put() of a new key/value tuple and loads it into the cache
      */
     @Override
-    public Aggregator[] cache(ImmutableBytesWritable cacheKey) {
+    public Aggregator[] cache(ImmutableBytesPtr cacheKey) {
         ImmutableBytesPtr key = new ImmutableBytesPtr(cacheKey);
         Aggregator[] rowAggregators = cache.get(key);
         if (rowAggregators == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
index 38c4ca0..68d07a9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupByCache.java
@@ -19,10 +19,9 @@ package org.apache.phoenix.coprocessor;
 
 import java.io.Closeable;
 
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-
 import org.apache.phoenix.expression.aggregator.Aggregator;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 /**
  * 
@@ -34,6 +33,6 @@ import org.apache.phoenix.expression.aggregator.Aggregator;
  */
 public interface GroupByCache extends Closeable {
     long size();
-    Aggregator[] cache(ImmutableBytesWritable key);
+    Aggregator[] cache(ImmutableBytesPtr key);
     RegionScanner getScanner(RegionScanner s);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 2d7c291..2c194c9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -138,7 +137,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
                 viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
             }
-            ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+            ImmutableBytesPtr tempPtr = new ImmutableBytesPtr();
             innerScanner =
                     getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector,
                         c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
@@ -239,7 +238,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
 
         private int estDistVals;
 
-        InMemoryGroupByCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
+        InMemoryGroupByCache(RegionCoprocessorEnvironment env, ImmutableBytesPtr tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
             int estValueSize = aggregators.getEstimatedByteSize();
             long estSize = sizeOfUnorderedGroupByMap(estDistVals, estValueSize);
             TenantCache tenantCache = GlobalCache.getTenantCache(env, tenantId);
@@ -257,7 +256,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         }
 
         @Override
-        public Aggregator[] cache(ImmutableBytesWritable cacheKey) {
+        public Aggregator[] cache(ImmutableBytesPtr cacheKey) {
             ImmutableBytesPtr key = new ImmutableBytesPtr(cacheKey);
             Aggregator[] rowAggregators = aggregateMap.get(key);
             if (rowAggregators == null) {
@@ -348,7 +347,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         private GroupByCacheFactory() {
         }
 
-        GroupByCache newCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
+        GroupByCache newCache(RegionCoprocessorEnvironment env, ImmutableBytesPtr tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
             Configuration conf = env.getConfiguration();
             boolean spillableEnabled =
                     conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
@@ -414,7 +413,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                         hasMore = scanner.nextRaw(results);
                         if (!results.isEmpty()) {
                             result.setKeyValues(results);
-                            ImmutableBytesWritable key =
+                            ImmutableBytesPtr key =
                                 TupleUtil.getConcatenatedValue(result, expressions);
                             Aggregator[] rowAggregators = groupByCache.cache(key);
                             // Aggregate values here
@@ -457,7 +456,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         }
         return new BaseRegionScanner(scanner) {
             private long rowCount = 0;
-            private ImmutableBytesWritable currentKey = null;
+            private ImmutableBytesPtr currentKey = null;
 
             @Override
             public boolean next(List<Cell> results) throws IOException {
@@ -465,7 +464,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 boolean atLimit;
                 boolean aggBoundary = false;
                 MultiKeyValueTuple result = new MultiKeyValueTuple();
-                ImmutableBytesWritable key = null;
+                ImmutableBytesPtr key = null;
                 Aggregator[] rowAggregators = aggregators.getAggregators();
                 // If we're calculating no aggregate functions, we can exit at the
                 // start of a new row. Otherwise, we have to wait until an agg

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index bd9c5ec..4340886 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -65,7 +64,7 @@ public class HashJoinRegionScanner implements RegionScanner {
     private ValueBitSet[] tempSrcBitSet;
 
     @SuppressWarnings("unchecked")
-    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesWritable tenantId, RegionCoprocessorEnvironment env) throws IOException {
+    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env) throws IOException {
         this.env = env;
         this.scanner = scanner;
         this.projector = projector;
@@ -198,7 +197,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                 for (Iterator<Tuple> iter = resultQueue.iterator(); iter.hasNext();) {
                     Tuple t = iter.next();
                     postFilter.reset();
-                    ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
+                    ImmutableBytesPtr tempPtr = new ImmutableBytesPtr();
                     try {
                         if (!postFilter.evaluate(t, tempPtr)) {
                             iter.remove();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index 48e3704..ade88db 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.TenantCache;
@@ -47,6 +46,7 @@ import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.OrderByExpression;
 import org.apache.phoenix.expression.function.ArrayIndexFunction;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.iterate.OffsetResultIterator;
 import org.apache.phoenix.iterate.OrderedResultIterator;
@@ -64,12 +64,11 @@ import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.tephra.Transaction;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-import org.apache.tephra.Transaction;
-
 
 /**
  *
@@ -225,7 +224,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                     dataColumns, tupleProjector, dataRegion, indexMaintainer, tx,
                     viewConstants, kvSchema, kvSchemaBitSet, j == null ? p : null, ptr);
 
-        final ImmutableBytesWritable tenantId = ScanUtil.getTenantId(scan);
+        final ImmutableBytesPtr tenantId = ScanUtil.getTenantId(scan);
         if (j != null) {
             innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, c.getEnvironment());
         }
@@ -311,7 +310,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
      *  getting the first Tuple (which forces running through the entire region)
      *  since after this everything is held in memory
      */
-    private RegionScanner getTopNScanner(final ObserverContext<RegionCoprocessorEnvironment> c, final RegionScanner s, final OrderedResultIterator iterator, ImmutableBytesWritable tenantId) throws Throwable {
+    private RegionScanner getTopNScanner(final ObserverContext<RegionCoprocessorEnvironment> c, final RegionScanner s, final OrderedResultIterator iterator, ImmutableBytesPtr tenantId) throws Throwable {
         final Tuple firstTuple;
         TenantCache tenantCache = GlobalCache.getTenantCache(c.getEnvironment(), tenantId);
         long estSize = iterator.getEstimatedByteSize();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
index 2679f1c..818713b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.IndexMetaDataCache;
 import org.apache.phoenix.cache.ServerCacheClient;
@@ -71,7 +70,7 @@ public class PhoenixIndexMetaData implements IndexMetaData {
             };
         } else {
             byte[] tenantIdBytes = attributes.get(PhoenixRuntime.TENANT_ID_ATTRIB);
-            ImmutableBytesWritable tenantId = tenantIdBytes == null ? null : new ImmutableBytesWritable(tenantIdBytes);
+            ImmutableBytesPtr tenantId = tenantIdBytes == null ? null : new ImmutableBytesPtr(tenantIdBytes);
             TenantCache cache = GlobalCache.getTenantCache(env, tenantId);
             IndexMetaDataCache indexCache = (IndexMetaDataCache)cache.getServerCache(new ImmutableBytesPtr(uuid));
             if (indexCache == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/64cb9b01/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index d7f6f2f..b0e8a99 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -55,6 +55,7 @@ import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
 import org.apache.phoenix.filter.BooleanExpressionFilter;
 import org.apache.phoenix.filter.DistinctPrefixFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
@@ -115,13 +116,13 @@ public class ScanUtil {
     // Use getTenantId and pass in column name to match against
     // in as PSchema attribute. If column name matches in 
     // KeyExpressions, set on scan as attribute
-    public static ImmutableBytesWritable getTenantId(Scan scan) {
+    public static ImmutableBytesPtr getTenantId(Scan scan) {
         // Create Scan with special aggregation column over which to aggregate
         byte[] tenantId = scan.getAttribute(PhoenixRuntime.TENANT_ID_ATTRIB);
         if (tenantId == null) {
             return null;
         }
-        return new ImmutableBytesWritable(tenantId);
+        return new ImmutableBytesPtr(tenantId);
     }
     
     public static void setCustomAnnotations(Scan scan, byte[] annotations) {


[48/50] [abbrv] phoenix git commit: Fix compilation errors from sync with master

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
index 1b83be2,0000000..0915d09
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixTableModify.java
@@@ -1,395 -1,0 +1,395 @@@
 +package org.apache.phoenix.calcite.rel;
 +
 +import static org.apache.phoenix.execute.MutationState.RowTimestampColInfo.NULL_ROWTIMESTAMP_INFO;
 +
 +import java.sql.ParameterMetaData;
 +import java.sql.ResultSet;
 +import java.sql.SQLException;
 +import java.util.Arrays;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +
 +import org.apache.calcite.plan.RelOptCluster;
 +import org.apache.calcite.plan.RelOptTable;
 +import org.apache.calcite.plan.RelTraitSet;
 +import org.apache.calcite.prepare.Prepare.CatalogReader;
 +import org.apache.calcite.rel.RelNode;
 +import org.apache.calcite.rel.core.TableModify;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.phoenix.calcite.PhoenixTable;
 +import org.apache.phoenix.compile.ColumnResolver;
 +import org.apache.phoenix.compile.ExplainPlan;
 +import org.apache.phoenix.compile.FromCompiler;
 +import org.apache.phoenix.compile.MutationPlan;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.compile.SequenceManager;
 +import org.apache.phoenix.compile.StatementContext;
 +import org.apache.phoenix.compile.StatementPlan;
 +import org.apache.phoenix.exception.SQLExceptionCode;
 +import org.apache.phoenix.exception.SQLExceptionInfo;
 +import org.apache.phoenix.execute.MutationState;
 +import org.apache.phoenix.execute.MutationState.RowMutationState;
 +import org.apache.phoenix.execute.MutationState.RowTimestampColInfo;
 +import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 +import org.apache.phoenix.iterate.ResultIterator;
 +import org.apache.phoenix.jdbc.PhoenixConnection;
 +import org.apache.phoenix.jdbc.PhoenixResultSet;
 +import org.apache.phoenix.jdbc.PhoenixStatement;
 +import org.apache.phoenix.query.ConnectionQueryServices;
 +import org.apache.phoenix.query.QueryServices;
 +import org.apache.phoenix.query.QueryServicesOptions;
 +import org.apache.phoenix.schema.IllegalDataException;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PName;
 +import org.apache.phoenix.schema.PRow;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.SortOrder;
 +import org.apache.phoenix.schema.TableRef;
 +import org.apache.phoenix.schema.types.PLong;
 +import org.apache.phoenix.util.ByteUtil;
 +import org.apache.phoenix.util.MetaDataUtil;
 +import org.apache.phoenix.util.ScanUtil;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.Maps;
 +
 +public class PhoenixTableModify extends TableModify implements PhoenixRel {
 +
 +    public PhoenixTableModify(RelOptCluster cluster, RelTraitSet traits,
 +            RelOptTable table, CatalogReader catalogReader, RelNode child,
 +            Operation operation, List<String> updateColumnList, boolean flattened) {
 +        super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
 +        assert operation == Operation.INSERT || operation == Operation.DELETE;
 +    }
 +
 +    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
 +      return new PhoenixTableModify(
 +          getCluster(),
 +          traitSet,
 +          getTable(),
 +          getCatalogReader(),
 +          sole(inputs),
 +          getOperation(),
 +          getUpdateColumnList(),
 +          isFlattened());
 +    }
 +
 +    @Override
 +    public StatementPlan implement(PhoenixRelImplementor implementor) {
 +        final QueryPlan queryPlan = implementor.visitInput(0, (PhoenixQueryRel) input);
 +        final RowProjector projector = implementor.getTableMapping().createRowProjector();
 +
 +        final PhoenixTable targetTable = getTable().unwrap(PhoenixTable.class);
 +        final PhoenixConnection connection = targetTable.pc;
 +        final TableRef targetTableRef = targetTable.tableMapping.getTableRef();
 +        
 +        if (getOperation() == Operation.INSERT) {
 +            return upsert(connection, targetTable, targetTableRef, queryPlan, projector);
 +        }
 +        
 +        // delete
 +        return delete(connection, targetTable, targetTableRef, queryPlan, projector);
 +    }
 +    
 +    private static MutationPlan upsert(final PhoenixConnection connection,
 +            final PhoenixTable targetTable, final TableRef targetTableRef,
 +            final QueryPlan queryPlan, final RowProjector projector) {
 +        try (PhoenixStatement stmt = new PhoenixStatement(connection)) {
 +            final ColumnResolver resolver = FromCompiler.getResolver(targetTableRef);
 +            final StatementContext context = new StatementContext(stmt, resolver, new Scan(), new SequenceManager(stmt));
 +
 +            // TODO TenantId, ViewIndexId, UpdatableViewColumns
 +            final List<PColumn> mappedColumns = targetTable.tableMapping.getMappedColumns();
 +            final int[] columnIndexes = new int[mappedColumns.size()];
 +            final int[] pkSlotIndexes = new int[mappedColumns.size()];
 +            for (int i = 0; i < columnIndexes.length; i++) {
 +                PColumn column = mappedColumns.get(i);
 +                if (SchemaUtil.isPKColumn(column)) {
 +                    pkSlotIndexes[i] = column.getPosition();
 +                }
 +                columnIndexes[i] = column.getPosition();
 +            }
 +            // TODO
 +            final boolean useServerTimestamp = false;
 +            
 +            return new MutationPlan() {
 +                @Override
 +                public ParameterMetaData getParameterMetaData() {
 +                    return queryPlan.getContext().getBindManager().getParameterMetaData();
 +                }
 +
 +                @Override
 +                public StatementContext getContext() {
 +                    return context;
 +                }
 +
 +                @Override
 +                public TableRef getTargetRef() {
 +                    return targetTableRef;
 +                }
 +
 +                @Override
 +                public Set<TableRef> getSourceRefs() {
 +                    // TODO return originalQueryPlan.getSourceRefs();
 +                    return queryPlan.getSourceRefs();
 +                }
 +
 +                @Override
 +                public org.apache.phoenix.jdbc.PhoenixStatement.Operation getOperation() {
 +                    return org.apache.phoenix.jdbc.PhoenixStatement.Operation.UPSERT;
 +                }
 +
 +                @Override
 +                public MutationState execute() throws SQLException {
 +                    ResultIterator iterator = queryPlan.iterator();
 +                    // simplest version, no run-on-server, no pipelined update
 +                    StatementContext childContext = queryPlan.getContext();
 +                    ConnectionQueryServices services = connection.getQueryServices();
 +                    int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,
 +                            QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
 +                    int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
 +                    boolean isAutoCommit = connection.getAutoCommit();
 +                    byte[][] values = new byte[columnIndexes.length][];
 +                    int rowCount = 0;
 +                    Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
 +                    PTable table = targetTableRef.getTable();
 +                    try (ResultSet rs = new PhoenixResultSet(iterator, projector, childContext)) {
 +                        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +                        while (rs.next()) {
 +                            for (int i = 0; i < values.length; i++) {
 +                                PColumn column = table.getColumns().get(columnIndexes[i]);
 +                                byte[] bytes = rs.getBytes(i + 1);
 +                                ptr.set(bytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : bytes);
 +                                Object value = rs.getObject(i + 1);
 +                                int rsPrecision = rs.getMetaData().getPrecision(i + 1);
 +                                Integer precision = rsPrecision == 0 ? null : rsPrecision;
 +                                int rsScale = rs.getMetaData().getScale(i + 1);
 +                                Integer scale = rsScale == 0 ? null : rsScale;
 +                                // We are guaranteed that the two column will have compatible types,
 +                                // as we checked that before.
 +                                if (!column.getDataType().isSizeCompatible(ptr, value, column.getDataType(), precision, scale,
 +                                        column.getMaxLength(), column.getScale())) { throw new SQLExceptionInfo.Builder(
 +                                        SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(column.getName().getString())
 +                                        .setMessage("value=" + column.getDataType().toStringLiteral(ptr, null)).build()
 +                                        .buildException(); }
 +                                column.getDataType().coerceBytes(ptr, value, column.getDataType(), 
 +                                        precision, scale, SortOrder.getDefault(), 
 +                                        column.getMaxLength(), column.getScale(), column.getSortOrder(),
 +                                        table.rowKeyOrderOptimizable());
 +                                values[i] = ByteUtil.copyKeyBytesIfNecessary(ptr);
 +                            }
 +                            setValues(values, pkSlotIndexes, columnIndexes, table, mutation, connection, useServerTimestamp);
 +                            rowCount++;
 +                            // Commit a batch if auto commit is true and we're at our batch size
 +                            if (isAutoCommit && rowCount % batchSize == 0) {
 +                                MutationState state = new MutationState(targetTableRef, mutation, 0, maxSize, connection);
 +                                connection.getMutationState().join(state);
 +                                connection.getMutationState().send();
 +                                mutation.clear();
 +                            }
 +                        }
 +                        // If auto commit is true, this last batch will be committed upon return
 +                        return new MutationState(targetTableRef, mutation, rowCount / batchSize * batchSize, maxSize, connection);
 +                    }
 +                }
 +
 +                @Override
 +                public ExplainPlan getExplainPlan() throws SQLException {
 +                    List<String> queryPlanSteps =  queryPlan.getExplainPlan().getPlanSteps();
 +                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
 +                    planSteps.add("UPSERT SELECT");
 +                    planSteps.addAll(queryPlanSteps);
 +                    return new ExplainPlan(planSteps);
 +                }                
 +            };
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes, PTable table, Map<ImmutableBytesPtr,RowMutationState> mutation, PhoenixConnection connection, boolean useServerTimestamp) {
 +        Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
 +        byte[][] pkValues = new byte[table.getPKColumns().size()][];
 +        // If the table uses salting, the first byte is the salting byte, set to an empty array
 +        // here and we will fill in the byte later in PRowImpl.
 +        if (table.getBucketNum() != null) {
 +            pkValues[0] = new byte[] {0};
 +        }
 +        Long rowTimestamp = null; // case when the table doesn't have a row timestamp column
 +        RowTimestampColInfo rowTsColInfo = new RowTimestampColInfo(useServerTimestamp, rowTimestamp);
 +        for (int i = 0; i < values.length; i++) {
 +            byte[] value = values[i];
 +            PColumn column = table.getColumns().get(columnIndexes[i]);
 +            if (SchemaUtil.isPKColumn(column)) {
 +                pkValues[pkSlotIndex[i]] = value;
 +                if (SchemaUtil.getPKPosition(table, column) == table.getRowTimestampColPos()) {
 +                    if (!useServerTimestamp) {
 +                        PColumn rowTimestampCol = table.getPKColumns().get(table.getRowTimestampColPos());
 +                        rowTimestamp = PLong.INSTANCE.getCodec().decodeLong(value, 0, rowTimestampCol.getSortOrder());
 +                        if (rowTimestamp < 0) {
 +                            throw new IllegalDataException("Value of a column designated as ROW_TIMESTAMP cannot be less than zero");
 +                        }
 +                        rowTsColInfo = new RowTimestampColInfo(useServerTimestamp, rowTimestamp);
 +                    } 
 +                }
 +            } else {
 +                columnValues.put(column, value);
 +            }
 +        }
 +        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
 +        table.newKey(ptr, pkValues);
 +        mutation.put(ptr, new RowMutationState(columnValues, connection.getStatementExecutionCounter(), rowTsColInfo));
 +    }
 +
 +    private static MutationPlan delete(final PhoenixConnection connection,
 +            final PhoenixTable targetTable, final TableRef targetTableRef,
 +            final QueryPlan queryPlan, final RowProjector projector) {
 +        final StatementContext context = queryPlan.getContext();
 +        // TODO
 +        final boolean deleteFromImmutableIndexToo = false;
 +        return new MutationPlan() {
 +            @Override
 +            public ParameterMetaData getParameterMetaData() {
 +                return context.getBindManager().getParameterMetaData();
 +            }
 +
 +            @Override
 +            public StatementContext getContext() {
 +                return context;
 +            }
 +
 +            @Override
 +            public TableRef getTargetRef() {
 +                return targetTableRef;
 +            }
 +
 +            @Override
 +            public Set<TableRef> getSourceRefs() {
 +                // TODO dataPlan.getSourceRefs();
 +                return queryPlan.getSourceRefs();
 +            }
 +
 +            @Override
 +            public org.apache.phoenix.jdbc.PhoenixStatement.Operation getOperation() {
 +                return org.apache.phoenix.jdbc.PhoenixStatement.Operation.DELETE;
 +            }
 +
 +            @Override
 +            public MutationState execute() throws SQLException {
 +                ResultIterator iterator = queryPlan.iterator();
 +                try {
 +                    // TODO hasLimit??
 +                    return deleteRows(context, targetTableRef, deleteFromImmutableIndexToo ? queryPlan.getTableRef() : null, iterator, projector, queryPlan.getTableRef());
 +                } finally {
 +                    iterator.close();
 +                }
 +            }
 +
 +            @Override
 +            public ExplainPlan getExplainPlan() throws SQLException {
 +                List<String> queryPlanSteps =  queryPlan.getExplainPlan().getPlanSteps();
 +                List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
 +                planSteps.add("DELETE ROWS");
 +                planSteps.addAll(queryPlanSteps);
 +                return new ExplainPlan(planSteps);
 +            }
 +        };
 +    }
 +    
 +    private static MutationState deleteRows(StatementContext childContext, TableRef targetTableRef, TableRef indexTableRef, ResultIterator iterator, RowProjector projector, TableRef sourceTableRef) throws SQLException {
 +        PTable table = targetTableRef.getTable();
 +        PhoenixStatement statement = childContext.getStatement();
 +        PhoenixConnection connection = statement.getConnection();
 +        PName tenantId = connection.getTenantId();
 +        byte[] tenantIdBytes = null;
 +        if (tenantId != null) {
-             tenantIdBytes = ScanUtil.getTenantIdBytes(table.getRowKeySchema(), table.getBucketNum() != null, tenantId);
++            tenantIdBytes = ScanUtil.getTenantIdBytes(table.getRowKeySchema(), table.getBucketNum() != null, tenantId, table.getViewIndexId() != null);
 +        }
 +        final boolean isAutoCommit = connection.getAutoCommit();
 +        ConnectionQueryServices services = connection.getQueryServices();
 +        final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
 +        final int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
 +        Map<ImmutableBytesPtr,RowMutationState> mutations = Maps.newHashMapWithExpectedSize(batchSize);
 +        Map<ImmutableBytesPtr,RowMutationState> indexMutations = null;
 +        // If indexTableRef is set, we're deleting the rows from both the index table and
 +        // the data table through a single query to save executing an additional one.
 +        if (indexTableRef != null) {
 +            indexMutations = Maps.newHashMapWithExpectedSize(batchSize);
 +        }
 +        List<PColumn> pkColumns = table.getPKColumns();
 +        boolean isMultiTenant = table.isMultiTenant() && tenantIdBytes != null;
 +        boolean isSharedViewIndex = table.getViewIndexId() != null;
 +        int offset = (table.getBucketNum() == null ? 0 : 1);
 +        byte[][] values = new byte[pkColumns.size()][];
 +        if (isMultiTenant) {
 +            values[offset++] = tenantIdBytes;
 +        }
 +        if (isSharedViewIndex) {
 +            values[offset++] = MetaDataUtil.getViewIndexIdDataType().toBytes(table.getViewIndexId());
 +        }
 +        try (PhoenixResultSet rs = new PhoenixResultSet(iterator, projector, childContext)) {
 +            int rowCount = 0;
 +            while (rs.next()) {
 +                ImmutableBytesPtr ptr = new ImmutableBytesPtr();  // allocate new as this is a key in a Map
 +                // Use tuple directly, as projector would not have all the PK columns from
 +                // our index table inside of our projection. Since the tables are equal,
 +                // there's no transation required.
 +                if (sourceTableRef.equals(targetTableRef)) {
 +                    rs.getCurrentRow().getKey(ptr);
 +                } else {
 +                    for (int i = offset; i < values.length; i++) {
 +                        byte[] byteValue = rs.getBytes(i+1-offset);
 +                        // The ResultSet.getBytes() call will have inverted it - we need to invert it back.
 +                        // TODO: consider going under the hood and just getting the bytes
 +                        if (pkColumns.get(i).getSortOrder() == SortOrder.DESC) {
 +                            byte[] tempByteValue = Arrays.copyOf(byteValue, byteValue.length);
 +                            byteValue = SortOrder.invert(byteValue, 0, tempByteValue, 0, byteValue.length);
 +                        }
 +                        values[i] = byteValue;
 +                    }
 +                    table.newKey(ptr, values);
 +                }
 +                // When issuing deletes, we do not care about the row time ranges. Also, if the table had a row timestamp column, then the
 +                // row key will already have its value. 
 +                mutations.put(ptr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO));
 +                if (indexTableRef != null) {
 +                    ImmutableBytesPtr indexPtr = new ImmutableBytesPtr(); // allocate new as this is a key in a Map
 +                    rs.getCurrentRow().getKey(indexPtr);
 +                    indexMutations.put(indexPtr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO));
 +                }
 +                if (mutations.size() > maxSize) {
 +                    throw new IllegalArgumentException("MutationState size of " + mutations.size() + " is bigger than max allowed size of " + maxSize);
 +                }
 +                rowCount++;
 +                // Commit a batch if auto commit is true and we're at our batch size
 +                if (isAutoCommit && rowCount % batchSize == 0) {
 +                    MutationState state = new MutationState(targetTableRef, mutations, 0, maxSize, connection);
 +                    connection.getMutationState().join(state);
 +                    if (indexTableRef != null) {
 +                        MutationState indexState = new MutationState(indexTableRef, indexMutations, 0, maxSize, connection);
 +                        connection.getMutationState().join(indexState);
 +                    }
 +                    connection.getMutationState().send();
 +                    mutations.clear();
 +                    if (indexMutations != null) {
 +                        indexMutations.clear();
 +                    }
 +                }
 +            }
 +
 +            // If auto commit is true, this last batch will be committed upon return
 +            int nCommittedRows = rowCount / batchSize * batchSize;
 +            MutationState state = new MutationState(targetTableRef, mutations, nCommittedRows, maxSize, connection);
 +            if (indexTableRef != null) {
 +                // To prevent the counting of these index rows, we have a negative for remainingRows.
 +                MutationState indexState = new MutationState(indexTableRef, indexMutations, 0, maxSize, connection);
 +                state.join(indexState);
 +            }
 +            return state;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
index a72793a,0000000..5bc2c0d
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixToEnumerableConverter.java
@@@ -1,138 -1,0 +1,143 @@@
 +package org.apache.phoenix.calcite.rel;
 +
 +import java.sql.SQLException;
 +import java.util.List;
 +
 +import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 +import org.apache.calcite.adapter.enumerable.EnumerableRel;
 +import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
 +import org.apache.calcite.adapter.enumerable.JavaRowFormat;
 +import org.apache.calcite.adapter.enumerable.PhysType;
 +import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
 +import org.apache.calcite.linq4j.tree.BlockBuilder;
 +import org.apache.calcite.linq4j.tree.Expression;
 +import org.apache.calcite.linq4j.tree.Expressions;
 +import org.apache.calcite.linq4j.tree.MethodCallExpression;
 +import org.apache.calcite.linq4j.tree.ParameterExpression;
 +import org.apache.calcite.plan.ConventionTraitDef;
 +import org.apache.calcite.plan.RelOptCluster;
 +import org.apache.calcite.plan.RelOptCost;
 +import org.apache.calcite.plan.RelOptPlanner;
 +import org.apache.calcite.plan.RelTraitSet;
 +import org.apache.calcite.rel.RelNode;
 +import org.apache.calcite.rel.convert.ConverterImpl;
 +import org.apache.calcite.rel.metadata.RelMetadataQuery;
 +import org.apache.calcite.rel.type.RelDataType;
 +import org.apache.calcite.util.ImmutableIntList;
++import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.phoenix.calcite.BuiltInMethod;
 +import org.apache.phoenix.calcite.rel.PhoenixRelImplementor.ImplementorContext;
 +import org.apache.phoenix.compile.ExplainPlan;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.compile.StatementPlan;
 +import org.apache.phoenix.execute.DelegateQueryPlan;
 +import org.apache.phoenix.execute.RuntimeContext;
 +import org.apache.phoenix.execute.RuntimeContextImpl;
 +import org.apache.phoenix.iterate.DefaultParallelScanGrouper;
 +import org.apache.phoenix.iterate.ParallelScanGrouper;
 +import org.apache.phoenix.iterate.ResultIterator;
 +
 +/**
 + * Scan of a Phoenix table.
 + */
 +public class PhoenixToEnumerableConverter extends ConverterImpl implements EnumerableRel {
 +
 +    public static PhoenixToEnumerableConverter create(RelNode input) {
 +        RelOptCluster cluster = input.getCluster();
 +        RelTraitSet traits = input.getTraitSet().replace(EnumerableConvention.INSTANCE);
 +        return new PhoenixToEnumerableConverter(cluster, traits, input);
 +    }
 +
 +    private PhoenixToEnumerableConverter(
 +        RelOptCluster cluster,
 +        RelTraitSet traits,
 +        RelNode input) {
 +        super(cluster, ConventionTraitDef.INSTANCE, traits, input);
 +    }
 +
 +    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
 +        return create(sole(inputs));
 +    }
 +
 +    @Override public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
 +        return super.computeSelfCost(planner, mq)
 +                .multiplyBy(.1)
 +                .multiplyBy(PhoenixRel.PHOENIX_FACTOR)
 +                .multiplyBy(PhoenixRel.SERVER_FACTOR);
 +    }
 +
 +    @Override
 +    public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
 +        // Generates code that instantiates a result iterator, then converts it
 +        // to an enumerable.
 +        //
 +        //   ResultIterator iterator = root.get("x");
 +        //   return CalciteRuntime.toEnumerable(iterator);
 +        final BlockBuilder list = new BlockBuilder();
 +        StatementPlan plan = makePlan((PhoenixRel)getInput());
 +        Expression var = stash(implementor, plan, StatementPlan.class);
 +        final RelDataType rowType = getRowType();
 +        final PhysType physType =
 +            PhysTypeImpl.of(
 +                implementor.getTypeFactory(), rowType,
 +                pref.prefer(JavaRowFormat.ARRAY));
 +        final Expression iterator_ =
 +            list.append("iterator", var);
 +        final Expression enumerable_ =
 +            list.append("enumerable",
 +                Expressions.call(BuiltInMethod.TO_ENUMERABLE.method,
 +                    iterator_));
 +        list.add(Expressions.return_(null, enumerable_));
 +        return implementor.result(physType, list.toBlock());
 +    }
 +    
 +    static StatementPlan makePlan(PhoenixRel rel) {
 +        RuntimeContext runtimeContext = new RuntimeContextImpl();
 +        RuntimeContext.THREAD_LOCAL.get().add(runtimeContext);
 +        final PhoenixRelImplementor phoenixImplementor = new PhoenixRelImplementorImpl(runtimeContext);
 +        phoenixImplementor.pushContext(new ImplementorContext(true, false, ImmutableIntList.identity(rel.getRowType().getFieldCount())));
 +        final StatementPlan plan = rel.implement(phoenixImplementor);
 +        if (!(plan instanceof QueryPlan)) {
 +            return plan;
 +        }
 +            
 +        return new DelegateQueryPlan((QueryPlan) plan) {
 +            @Override
 +            public ResultIterator iterator() throws SQLException {
 +                return iterator(DefaultParallelScanGrouper.getInstance());
 +            }
 +            @Override
 +            public ExplainPlan getExplainPlan() throws SQLException {
 +                return delegate.getExplainPlan();
 +            }
 +            @Override
 +            public RowProjector getProjector() {
 +                return phoenixImplementor.getTableMapping().createRowProjector();
 +            }
 +            @Override
 +            public ResultIterator iterator(ParallelScanGrouper scanGrouper)
 +                    throws SQLException {
 +                return delegate.iterator(scanGrouper);
 +            }
 +            @Override
 +            public QueryPlan limit(Integer limit) {
 +                return delegate.limit(limit);
 +            }
++            @Override
++            public ResultIterator iterator(ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
++                return delegate.iterator(scanGrouper, scan);
++            }
 +        };
 +    }
 +
 +    @SuppressWarnings({ "rawtypes", "unchecked" })
 +    static Expression stash(EnumerableRelImplementor implementor, Object o, Class clazz) {
 +        ParameterExpression x = (ParameterExpression) implementor.stash(o, clazz);
 +        MethodCallExpression e =
 +            Expressions.call(implementor.getRootExpression(),
 +                org.apache.calcite.util.BuiltInMethod.DATA_CONTEXT_GET.method,
 +                Expressions.constant(x.name));
 +        return Expressions.convert_(e, clazz);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 6909b23,4d3c0cf..bbc995c
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@@ -175,12 -177,12 +177,12 @@@ public class TupleProjectionCompiler 
              projectedColumns.add(column);
          }
          return PTableImpl.makePTable(table.getTenantId(), PROJECTED_TABLE_SCHEMA, table.getName(), PTableType.PROJECTED,
 -                null, table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
 -                retainPKColumns ? table.getBucketNum() : null, projectedColumns, null, null,
 -                Collections.<PTable> emptyList(), table.isImmutableRows(), Collections.<PName> emptyList(), null, null,
 -                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
 -                table.getViewIndexId(), null, table.rowKeyOrderOptimizable(), table.isTransactional(),
 -                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
 +                    null, table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
 +                    retainPKColumns ? table.getBucketNum() : null, projectedColumns, null,
 +                    null, Collections.<PTable>emptyList(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
 +                    table.isWALDisabled(), retainPKColumns ? table.isMultiTenant() : false, table.getStoreNulls(), table.getViewType(),
 +                    retainPKColumns ? table.getViewIndexId() : null, null, table.rowKeyOrderOptimizable(), table.isTransactional(),
-                     table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp());
++                    table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
      }
  
      // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
index 5ab8e3a,00d478a..cd83c4d
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
@@@ -77,10 -80,9 +80,12 @@@ public class AggregatePlan extends Base
      private final Expression having;
      private List<KeyRange> splits;
      private List<List<Scan>> scans;
-     
+     private static final Logger logger = LoggerFactory.getLogger(AggregatePlan.class);
+     private boolean isSerial;
 -    
++        
 +    public static AggregatePlan create(AggregatePlan plan, OrderBy newOrderBy) {
 +        return new AggregatePlan(plan.getContext(), plan.getStatement(), plan.getTableRef(), plan.getSourceRefs().iterator().next(), plan.getProjector(), null, null, newOrderBy, plan.parallelIteratorFactory, plan.getGroupBy(), plan.getHaving(), plan.dynamicFilter);
 +    }
  
      public AggregatePlan(StatementContext context, FilterableStatement statement, TableRef table,
              RowProjector projector, Integer limit, Integer offset, OrderBy orderBy,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
index 56930ba,fc5a04d..9e6dad3
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
@@@ -21,9 -21,9 +21,10 @@@ import java.io.IOException
  import java.sql.SQLException;
  import java.util.List;
  
+ import org.apache.hadoop.hbase.client.Scan;
  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
  import org.apache.phoenix.compile.ExplainPlan;
 +import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
  import org.apache.phoenix.compile.QueryPlan;
  import org.apache.phoenix.exception.SQLExceptionCode;
  import org.apache.phoenix.exception.SQLExceptionInfo;
@@@ -105,10 -104,14 +106,15 @@@ public class CorrelatePlan extends Dele
      }
  
      @Override
-     public ResultIterator iterator(ParallelScanGrouper scanGrouper)
+     public ResultIterator iterator(ParallelScanGrouper scanGrouper) 
+                 throws SQLException {
+         return iterator(scanGrouper, null);
+     }
+     @Override
+     public ResultIterator iterator(ParallelScanGrouper scanGrouper, Scan scan)
              throws SQLException {
          return new ResultIterator() {
 +            private final CorrelateVariable variable = runtimeContext.getCorrelateVariable(variableId);
              private final ValueBitSet destBitSet = ValueBitSet.newInstance(joinedSchema);
              private final ValueBitSet lhsBitSet = ValueBitSet.newInstance(lhsSchema);
              private final ValueBitSet rhsBitSet = 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
index 8f4711c,0000000..31ad0e9
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/DegenerateQueryPlan.java
@@@ -1,70 -1,0 +1,70 @@@
 +/*
 + * 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.execute;
 +
 +import java.sql.SQLException;
 +import java.util.Collections;
 +import java.util.List;
 +
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 +import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.compile.ScanRanges;
 +import org.apache.phoenix.compile.StatementContext;
 +import org.apache.phoenix.iterate.ParallelScanGrouper;
 +import org.apache.phoenix.iterate.ResultIterator;
 +import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 +import org.apache.phoenix.parse.FilterableStatement;
 +import org.apache.phoenix.query.KeyRange;
 +import org.apache.phoenix.schema.TableRef;
 +
 +public class DegenerateQueryPlan extends BaseQueryPlan {
 +
 +    public DegenerateQueryPlan(StatementContext context, FilterableStatement statement, TableRef table) {
 +        super(context, statement, table, table, RowProjector.EMPTY_PROJECTOR, PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA, null,null, OrderBy.EMPTY_ORDER_BY, GroupBy.EMPTY_GROUP_BY, null, null);
 +        context.setScanRanges(ScanRanges.NOTHING);
 +    }
 +
 +    @Override
 +    public List<KeyRange> getSplits() {
 +        return Collections.emptyList();
 +    }
 +
 +    @Override
 +    public List<List<Scan>> getScans() {
 +        return Collections.emptyList();
 +    }
 +
 +    @Override
-     protected ResultIterator newIterator(ParallelScanGrouper scanGrouper) throws SQLException {
++    protected ResultIterator newIterator(ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
 +        return null;
 +    }
 +
 +    @Override
 +    public boolean useRoundRobinIterator() {
 +        return false;
 +    }
 +
 +    @Override
 +    public QueryPlan limit(Integer limit) {
 +        return this;
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
index b085f08,7f735b7..d3d000b
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
@@@ -81,17 -80,15 +81,19 @@@ public class ScanPlan extends BaseQuery
      private List<KeyRange> splits;
      private List<List<Scan>> scans;
      private boolean allowPageFilter;
+     private boolean isSerial;
+     private boolean isDataToScanWithinThreshold;
 +
 +    public static ScanPlan create(ScanPlan plan, OrderBy newOrderBy) throws SQLException {
 +        return new ScanPlan(plan.getContext(), plan.getStatement(), plan.getTableRef(), plan.getSourceRefs().iterator().next(), plan.getProjector(), null, null, newOrderBy, plan.parallelIteratorFactory, plan.allowPageFilter, plan.dynamicFilter);
 +    }
      
      public ScanPlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector, Integer limit, Integer offset, OrderBy orderBy, ParallelIteratorFactory parallelIteratorFactory, boolean allowPageFilter) throws SQLException {
 -        this(context, statement, table, projector, limit, offset, orderBy, parallelIteratorFactory, allowPageFilter, null);
 +        this(context, statement, table, table, projector, limit, offset, orderBy, parallelIteratorFactory, allowPageFilter, null);
      }
      
 -    private ScanPlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector, Integer limit, Integer offset, OrderBy orderBy, ParallelIteratorFactory parallelIteratorFactory, boolean allowPageFilter, Expression dynamicFilter) throws SQLException {
 -        super(context, statement, table, projector, context.getBindManager().getParameterMetaData(), limit,offset, orderBy, GroupBy.EMPTY_GROUP_BY,
 +    public ScanPlan(StatementContext context, FilterableStatement statement, TableRef table, TableRef srcRef, RowProjector projector, Integer limit, Integer offset, OrderBy orderBy, ParallelIteratorFactory parallelIteratorFactory, boolean allowPageFilter, Expression dynamicFilter) throws SQLException {
 +        super(context, statement, table, srcRef, projector, context.getBindManager().getParameterMetaData(), limit,offset, orderBy, GroupBy.EMPTY_GROUP_BY,
                  parallelIteratorFactory != null ? parallelIteratorFactory :
                          buildResultIteratorFactory(context, statement, table, orderBy, limit, offset, allowPageFilter), dynamicFilter);
          this.allowPageFilter = allowPageFilter;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjectionPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
index 12db6d9,94c59df..137a632
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
@@@ -20,9 -20,9 +20,10 @@@ package org.apache.phoenix.execute
  import java.sql.SQLException;
  import java.util.List;
  
+ import org.apache.hadoop.hbase.client.Scan;
  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
  import org.apache.phoenix.compile.ExplainPlan;
 +import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
  import org.apache.phoenix.compile.QueryPlan;
  import org.apache.phoenix.expression.BaseSingleExpression;
  import org.apache.phoenix.expression.BaseTerminalExpression;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index e26660e,8b9adfd..20bda5e
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@@ -114,11 -119,12 +117,11 @@@ public abstract class BaseResultIterato
  
      private final List<List<Scan>> scans;
      private final List<KeyRange> splits;
 -    private final PTableStats tableStats;
      private final byte[] physicalTableName;
-     private final QueryPlan plan;
+     protected final QueryPlan plan;
      protected final String scanId;
      protected final MutationState mutationState;
-     private final ParallelScanGrouper scanGrouper;
+     protected final ParallelScanGrouper scanGrouper;
      // TODO: too much nesting here - breakup into new classes.
      private final List<List<List<Pair<Scan,Future<PeekingResultIterator>>>>> allFutures;
      private Long estimatedRows;
@@@ -136,17 -143,14 +140,20 @@@
          return plan.getTableRef().getTable();
      }
      
+     protected boolean useStats() {
++        return useStats(this.context);
++    }
++    
 +    private static boolean useStats(StatementContext context) {
 +        Scan scan = context.getScan();
 +        boolean isPointLookup = context.getScanRanges().isPointLookup();
          /*
-          *  Don't use guide posts if:
-          *  1) We're doing a point lookup, as HBase is fast enough at those
-          *     to not need them to be further parallelized. TODO: perf test to verify
-          *  2) We're collecting stats, as in this case we need to scan entire
-          *     regions worth of data to track where to put the guide posts.
+          * Don't use guide posts:
+          * 1) If we're collecting stats, as in this case we need to scan entire
+          * regions worth of data to track where to put the guide posts.
+          * 2) If the query is going to be executed serially.
           */
-         if (isPointLookup || ScanUtil.isAnalyzeTable(scan)) {
+         if (ScanUtil.isAnalyzeTable(scan)) {
              return false;
          }
          return true;
@@@ -342,10 -358,17 +361,10 @@@
          TableRef tableRef = plan.getTableRef();
          PTable table = tableRef.getTable();
          physicalTableName = table.getPhysicalName().getBytes();
 -        Long currentSCN = context.getConnection().getSCN();
 -        if (null == currentSCN) {
 -          currentSCN = HConstants.LATEST_TIMESTAMP;
 -        }
 -        tableStats = useStats() && StatisticsUtil.isStatsEnabled(TableName.valueOf(physicalTableName))
 -                ? context.getConnection().getQueryServices().getTableStats(physicalTableName, currentSCN)
 -                : PTableStats.EMPTY_STATS;
          // Used to tie all the scans together during logging
-         scanId = UUID.randomUUID().toString();
+         scanId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
          
-         initializeScan(plan, perScanLimit, offset);
+         initializeScan(plan, perScanLimit, offset, scan);
          
          this.scans = getParallelScans();
          List<KeyRange> splitRanges = Lists.newArrayListWithExpectedSize(scans.size() * ESTIMATED_GUIDEPOSTS_PER_REGION);
@@@ -401,18 -424,13 +420,13 @@@
          return guideIndex;
      }
  
-     private static GuidePostsInfo getGuidePosts(StatementContext context, PTable table, Set<byte[]> whereConditions) throws SQLException {
-         /*
-          * Don't use guide posts if: 1) We're doing a point lookup, as HBase is fast enough at those to not need them to
-          * be further parallelized. TODO: pref test to verify 2) We're collecting stats, as in this case we need to scan
-          * entire regions worth of data to track where to put the guide posts.
-          */
-         if (!useStats(context)) { return GuidePostsInfo.NO_GUIDEPOST; }
 -    private GuidePostsInfo getGuidePosts(Set<byte[]> whereConditions) {
 -        if (!useStats()) { return GuidePostsInfo.NO_GUIDEPOST; }
++    private static GuidePostsInfo getGuidePosts(StatementContext context, PTable table, Set<byte[]> whereConditions, boolean useStats) throws SQLException {
++        if (!useStats) { return GuidePostsInfo.NO_GUIDEPOST; }
  
          GuidePostsInfo gps = null;
 -        PTable table = getTable();
 +        PTableStats tableStats = new MetaDataClient(context.getConnection()).getTableStats(table);
          Map<byte[], GuidePostsInfo> guidePostMap = tableStats.getGuidePosts();
 -        byte[] defaultCF = SchemaUtil.getEmptyColumnFamily(getTable());
 +        byte[] defaultCF = SchemaUtil.getEmptyColumnFamily(table);
          if (table.getColumnFamilies().isEmpty()) {
              // For sure we can get the defaultCF from the table
              gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
@@@ -495,7 -571,7 +567,7 @@@
                  whereConditions.add(cf);
              }
          }
-         GuidePostsInfo gps = getGuidePosts(context, table, whereConditions);
 -        GuidePostsInfo gps = getGuidePosts(whereConditions);
++        GuidePostsInfo gps = getGuidePosts(context, table, whereConditions, useStats());
          hasGuidePosts = gps != GuidePostsInfo.NO_GUIDEPOST;
          boolean traverseAllRegions = isSalted || isLocalIndex;
          if (!traverseAllRegions) {
@@@ -612,130 -686,6 +682,130 @@@
          return parallelScans;
      }
  
 +
 +    /**
 +     * Compute the estimated count of rows and bytes that will be scanned.
 +     * @return the estimated row count and the byte count.
 +     * @throws SQLException
 +     */
 +    public static Pair<Long, Long> getEstimatedCount(StatementContext context, PTable table) throws SQLException {
 +        if (table.getName() == null) { // empty table
 +            return new Pair<Long, Long>(null, null);
 +        }
 +        
 +        if (context.getScanRanges().isPointLookup()) {
 +            return new Pair<Long, Long>(1L, SchemaUtil.estimateRowSize(table));
 +        }
 +        
 +        TreeSet<byte[]> whereConditions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 +        for(Pair<byte[], byte[]> where : context.getWhereConditionColumns()) {
 +            byte[] cf = where.getFirst();
 +            if (cf != null) {
 +                whereConditions.add(cf);
 +            }
 +        }
-         GuidePostsInfo gps = getGuidePosts(context, table, whereConditions);
++        GuidePostsInfo gps = getGuidePosts(context, table, whereConditions, useStats(context));
 +        if (gps == GuidePostsInfo.NO_GUIDEPOST) {
 +            return new Pair<Long, Long>(null, null);
 +        }
 +        
 +        byte[] startKey = ByteUtil.EMPTY_BYTE_ARRAY;
 +        byte[] stopKey = ByteUtil.EMPTY_BYTE_ARRAY;
 +        Scan scan = context.getScan();
 +        List<HRegionLocation> regionLocations = context.getConnection().getQueryServices()
 +                .getAllTableRegions(table.getPhysicalName().getBytes());
 +        List<byte[]> regionBoundaries = toBoundaries(regionLocations);
 +        ScanRanges scanRanges = context.getScanRanges();
 +        boolean isSalted = table.getBucketNum() != null;
 +        boolean isLocalIndex = table.getIndexType() == IndexType.LOCAL;
 +        boolean traverseAllRegions = isSalted || isLocalIndex;
 +        if (!traverseAllRegions) {
 +            byte[] scanStartRow = scan.getStartRow();
 +            if (scanStartRow.length != 0 && Bytes.compareTo(scanStartRow, startKey) > 0) {
 +                startKey = scanStartRow;
 +            }
 +            byte[] scanStopRow = scan.getStopRow();
 +            if (stopKey.length == 0
 +                    || (scanStopRow.length != 0 && Bytes.compareTo(scanStopRow, stopKey) < 0)) {
 +                stopKey = scanStopRow;
 +            }
 +        }
 +        
 +        int regionIndex = 0;
 +        int stopIndex = regionBoundaries.size();
 +        if (startKey.length > 0) {
 +            regionIndex = getIndexContainingInclusive(regionBoundaries, startKey);
 +        }
 +        if (stopKey.length > 0) {
 +            stopIndex = Math.min(stopIndex, regionIndex + getIndexContainingExclusive(regionBoundaries.subList(regionIndex, stopIndex), stopKey));
 +            if (isLocalIndex) {
 +                stopKey = regionLocations.get(stopIndex).getRegionInfo().getEndKey();
 +            }
 +        }
 +        
 +        ImmutableBytesWritable currentKey = new ImmutableBytesWritable(startKey);
 +        
 +        int gpsSize = gps.getGuidePostsCount();
 +        int keyOffset = 0;
 +        ImmutableBytesWritable currentGuidePost = ByteUtil.EMPTY_IMMUTABLE_BYTE_ARRAY;
 +        ImmutableBytesWritable guidePosts = gps.getGuidePosts();
 +        ByteArrayInputStream stream = null;
 +        DataInput input = null;
 +        PrefixByteDecoder decoder = null;
 +        int guideIndex = 0;
 +        long estimatedRows = 0;
 +        long estimatedSize = 0;
 +        try {
 +            if (gpsSize > 0) {
 +                stream = new ByteArrayInputStream(guidePosts.get(), guidePosts.getOffset(), guidePosts.getLength());
 +                input = new DataInputStream(stream);
 +                decoder = new PrefixByteDecoder(gps.getMaxLength());
 +                try {
 +                    while (currentKey.compareTo(currentGuidePost = PrefixByteCodec.decode(decoder, input)) >= 0
 +                            && currentKey.getLength() != 0) {
 +                        guideIndex++;
 +                    }
 +                } catch (EOFException e) {}
 +            }
 +            byte[] currentKeyBytes = currentKey.copyBytes();
 +    
 +            // Merge bisect with guideposts for all but the last region
 +            while (regionIndex <= stopIndex) {
 +                byte[] currentGuidePostBytes = currentGuidePost.copyBytes();
 +                byte[] endKey, endRegionKey = EMPTY_BYTE_ARRAY;
 +                if (regionIndex == stopIndex) {
 +                    endKey = stopKey;
 +                } else {
 +                    endKey = regionBoundaries.get(regionIndex);
 +                }
 +                HRegionLocation regionLocation = regionLocations.get(regionIndex);
 +                if (isLocalIndex) {
 +                    HRegionInfo regionInfo = regionLocation.getRegionInfo();
 +                    endRegionKey = regionInfo.getEndKey();
 +                    keyOffset = ScanUtil.getRowKeyOffset(regionInfo.getStartKey(), endRegionKey);
 +                }
 +                try {
 +                    while (guideIndex < gpsSize && (currentGuidePost.compareTo(endKey) <= 0 || endKey.length == 0)) {
 +                        Scan newScan = scanRanges.intersectScan(scan, currentKeyBytes, currentGuidePostBytes, keyOffset,
 +                                false);
 +                        if (newScan != null) {
 +                            estimatedRows += gps.getRowCounts().get(guideIndex);
 +                            estimatedSize += gps.getByteCounts().get(guideIndex);
 +                        }
 +                        currentKeyBytes = currentGuidePost.copyBytes();
 +                        currentGuidePost = PrefixByteCodec.decode(decoder, input);
 +                        currentGuidePostBytes = currentGuidePost.copyBytes();
 +                        guideIndex++;
 +                    }
 +                } catch (EOFException e) {}
 +                currentKeyBytes = endKey;
 +                regionIndex++;
 +            }
 +        } finally {
 +            if (stream != null) Closeables.closeQuietly(stream);
 +        }
 +        return new Pair<Long, Long>(estimatedRows, estimatedSize);
 +    }
     
      public static <T> List<T> reverseIfNecessary(List<T> list, boolean reverse) {
          if (!reverse) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/parse/ColumnDef.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 755e509,daef367..cb0bceb
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@@ -1484,9 -1593,9 +1594,9 @@@ public abstract class BaseTest 
          if (ts != null) {
              props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
          }
-         Connection conn = DriverManager.getConnection(getUrl(), props);
+         Connection conn = DriverManager.getConnection(url, props);
          try {
 -            conn.createStatement().execute("CREATE SEQUENCE my.seq");
 +            conn.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS my.seq");
              // Insert into customer table
              PreparedStatement stmt = conn.prepareStatement(
                      "upsert into " + JOIN_CUSTOMER_TABLE_FULL_NAME +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
index 75ff40d,05fbf81..ecf31ab
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
@@@ -455,13 -471,8 +471,13 @@@ public class ParallelIteratorsSplitTes
              public boolean useRoundRobinIterator() {
                  return false;
              }
 +
 +            @Override
 +            public QueryPlan limit(Integer limit) {
 +                return this;
 +            }
              
-         }, null, new SpoolingResultIterator.SpoolingResultIteratorFactory(context.getConnection().getQueryServices()));
+         }, null, new SpoolingResultIterator.SpoolingResultIteratorFactory(context.getConnection().getQueryServices()), context.getScan(), false);
          List<KeyRange> keyRanges = parallelIterators.getSplits();
          return keyRanges;
      }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --cc phoenix-core/src/test/resources/log4j.properties
index 96ebd73,85706b4..575ee67
--- a/phoenix-core/src/test/resources/log4j.properties
+++ b/phoenix-core/src/test/resources/log4j.properties
@@@ -61,4 -61,5 +61,6 @@@ log4j.logger.org.mortbay.log=WAR
  log4j.logger.org.apache.hadoop=WARN
  log4j.logger.org.apache.zookeeper=ERROR
  log4j.logger.org.apache.hadoop.hbase=DEBUG
 +log4j.logger.org.apache.calcite=INFO
+ log4j.logger.org.apache.directory=WARN
+ log4j.logger.net.sf.ehcache=WARN

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index c21fbe9,b5edb6c..49af7f2
--- a/pom.xml
+++ b/pom.xml
@@@ -92,10 -97,9 +97,10 @@@
      <jodatime.version>1.6</jodatime.version>
      <joni.version>2.1.2</joni.version>
      <avatica.version>1.8.0</avatica.version>
 +    <calcite.version>1.9.0-SNAPSHOT</calcite.version>
      <jettyVersion>8.1.7.v20120910</jettyVersion>
-     <tephra.version>0.7.0</tephra.version>
-     <spark.version>1.5.2</spark.version>
+     <tephra.version>0.8.0-incubating</tephra.version>
+     <spark.version>1.6.1</spark.version>
      <scala.version>2.10.4</scala.version>
      <scala.binary.version>2.10</scala.binary.version>
  


[39/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayPrependFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayPrependFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayPrependFunctionIT.java
index 3145d95..f3386d9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayPrependFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayPrependFunctionIT.java
@@ -27,12 +27,12 @@ import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.types.PhoenixArray;
 import org.junit.Test;
 
-public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
+public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
-    private void initTableWithVarArray(Connection conn, String type, Object[] objectArray, String value) throws SQLException {
-        conn.createStatement().execute("CREATE TABLE t ( k VARCHAR PRIMARY KEY, a " + type + "[],b " + type + ")");
+    private void initTableWithVarArray(Connection conn, String tableName, String type, Object[] objectArray, String value) throws SQLException {
+        conn.createStatement().execute("CREATE TABLE " + tableName + " ( k VARCHAR PRIMARY KEY, a " + type + "[],b " + type + ")");
         conn.commit();
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?," + value + ")");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?," + value + ")");
         PhoenixArray array = (PhoenixArray) conn.createArrayOf(type, objectArray);
         stmt.setString(1, "a");
         stmt.setArray(2, array);
@@ -41,10 +41,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
 
     }
 
-    private void initTables(Connection conn) throws Exception {
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
+    private void initTables(Connection conn, String tableName) throws Exception {
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO regions(region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
+        String dml = "UPSERT INTO " + tableName + " (region_name,varchars,integers,doubles,bigints,chars,double1,char1,nullcheck,chars2) VALUES('SF Bay Area'," +
                 "ARRAY['2345','46345','23234']," +
                 "ARRAY[2345,46345,23234,456]," +
                 "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
@@ -60,10 +60,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
         conn.commit();
     }
 
-    private void initTablesDesc(Connection conn, String type, String val) throws Exception {
-        String ddl = "CREATE TABLE regions (pk " + type + " PRIMARY KEY DESC,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],chars2 CHAR(15)[], bools BOOLEAN[])";
+    private void initTablesDesc(Connection conn, String tableName, String type, String val) throws Exception {
+        String ddl = "CREATE TABLE " + tableName + " (pk " + type + " PRIMARY KEY DESC,varchars VARCHAR[],integers INTEGER[],doubles DOUBLE[],bigints BIGINT[],chars CHAR(15)[],chars2 CHAR(15)[], bools BOOLEAN[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO regions(pk,varchars,integers,doubles,bigints,chars,chars2,bools) VALUES(" + val + "," +
+        String dml = "UPSERT INTO " + tableName + "(pk,varchars,integers,doubles,bigints,chars,chars2,bools) VALUES(" + val + "," +
                 "ARRAY['2345','46345','23234']," +
                 "ARRAY[2345,46345,23234,456]," +
                 "ARRAY[23.45,46.345,23.234,45.6,5.78]," +
@@ -80,10 +80,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInteger() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(1234,integers) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(1234,integers) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{1234, 2345, 46345, 23234, 456};
@@ -97,10 +98,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionVarchar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('34567',varchars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('34567',varchars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"34567", "2345", "46345", "23234"};
@@ -115,12 +117,13 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayPrependFunctionNulls1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String[] s = new String[]{null, null, "1", "2"};
-        initTableWithVarArray(conn, "VARCHAR", s, null);
+        String tableName = generateRandomString();
+        initTableWithVarArray(conn, tableName, "VARCHAR", s, null);
         String[] s2 = new String[]{null, null, null, "1", "2"};
         PhoenixArray array2 = (PhoenixArray) conn.createArrayOf("VARCHAR", s2);
         conn = DriverManager.getConnection(getUrl());
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM t WHERE k = 'a'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM " + tableName + " WHERE k = 'a'");
         assertTrue(rs.next());
         assertEquals(array2, rs.getArray(1));
     }
@@ -129,12 +132,13 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayPrependFunctionNulls2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String[] s = new String[]{"1", "2"};
-        initTableWithVarArray(conn, "VARCHAR", s, null);
+        String tableName = generateRandomString();
+        initTableWithVarArray(conn, tableName, "VARCHAR", s, null);
         String[] s2 = new String[]{null, "1", "2"};
         PhoenixArray array2 = (PhoenixArray) conn.createArrayOf("VARCHAR", s2);
         conn = DriverManager.getConnection(getUrl());
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM t WHERE k = 'a'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM " + tableName + " WHERE k = 'a'");
         assertTrue(rs.next());
         assertEquals(array2, rs.getArray(1));
     }
@@ -143,12 +147,13 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayPrependFunctionNulls3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String[] s = new String[]{"176", null, "212"};
-        initTableWithVarArray(conn, "VARCHAR", s, null);
+        String tableName = generateRandomString();
+        initTableWithVarArray(conn, tableName, "VARCHAR", s, null);
         String[] s2 = new String[]{null, "176", null, "212"};
         PhoenixArray array2 = (PhoenixArray) conn.createArrayOf("VARCHAR", s2);
         conn = DriverManager.getConnection(getUrl());
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM t WHERE k = 'a'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM " + tableName + " WHERE k = 'a'");
         assertTrue(rs.next());
         assertEquals(array2, rs.getArray(1));
     }
@@ -157,12 +162,13 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     public void testArrayPrependFunctionNulls4() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String[] s = new String[]{"176", null, "212"};
-        initTableWithVarArray(conn, "VARCHAR", s, "'foo'");
+        String tableName = generateRandomString();
+        initTableWithVarArray(conn, tableName, "VARCHAR", s, "'foo'");
         String[] s2 = new String[]{"foo", "176", null, "212"};
         PhoenixArray array2 = (PhoenixArray) conn.createArrayOf("VARCHAR", s2);
         conn = DriverManager.getConnection(getUrl());
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM t WHERE k = 'a'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(b,a) FROM " + tableName + " WHERE k = 'a'");
         assertTrue(rs.next());
         assertEquals(array2, rs.getArray(1));
     }
@@ -170,10 +176,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionDouble() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(double1,doubles) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(double1,doubles) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{23.45, 23.45, 46.345, 23.234, 45.6, 5.78};
@@ -187,10 +194,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionDouble2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(23,doubles) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(23,doubles) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{new Double(23), 23.45, 46.345, 23.234, 45.6, 5.78};
@@ -204,9 +212,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionBigint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(1112,bigints) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(1112,bigints) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Long[] longs = new Long[]{1112l, 12l, 34l, 56l, 78l, 910l};
@@ -220,9 +229,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionChar() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('fac',chars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('fac',chars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"fac", "a", "bbbb", "c", "ddd", "e"};
@@ -236,28 +246,31 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test(expected = TypeMismatchException.class)
     public void testArrayPrependFunctionIntToCharArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(234,varchars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(234,varchars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
     }
 
     @Test(expected = TypeMismatchException.class)
     public void testArrayPrependFunctionVarcharToIntegerArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('234',integers) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('234',integers) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
 
     }
 
     @Test(expected = SQLException.class)
     public void testArrayPrependFunctionChar2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('facfacfacfacfacfacfac',chars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND('facfacfacfacfacfacfac',chars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         rs.next();
         rs.getArray(1);
     }
@@ -265,10 +278,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionIntegerToDoubleArray() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(45,doubles) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(45,doubles) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{45.0, 23.45, 46.345, 23.234, 45.6, 5.78};
@@ -282,10 +296,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithNestedFunctions1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(integers[1],ARRAY[23,45]) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(integers[1],ARRAY[23,45]) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2345, 23, 45};
@@ -299,10 +314,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithNestedFunctions2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(ARRAY_ELEM(ARRAY[2,4],1),integers) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(ARRAY_ELEM(ARRAY[2,4],1),integers) FROM " + tableName+ " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{2, 2345, 46345, 23234, 456};
@@ -316,10 +332,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithNestedFunctions3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(ARRAY_ELEM(doubles,2),doubles) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(ARRAY_ELEM(doubles,2),doubles) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{46.345, 23.45, 46.345, 23.234, 45.6, 5.78};
@@ -333,16 +350,16 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithUpsert1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,varchars) VALUES('SF Bay Area',ARRAY_PREPEND(':-)',ARRAY['hello','world']))";
+        String dml = "UPSERT INTO " + tableName + " (region_name,varchars) VALUES('SF Bay Area',ARRAY_PREPEND(':-)',ARRAY['hello','world']))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{":-)", "hello", "world"};
@@ -356,16 +373,17 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithUpsert2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
+        String ddl = "CREATE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY,integers INTEGER[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,integers) VALUES('SF Bay Area',ARRAY_PREPEND(6,ARRAY[4,5]))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,integers) VALUES('SF Bay Area',ARRAY_PREPEND(6,ARRAY[4,5]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT integers FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT integers FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{6, 4, 5};
@@ -379,16 +397,16 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithUpsert3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE regions (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + "(region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO regions(region_name,doubles) VALUES('SF Bay Area',ARRAY_PREPEND(9.0,ARRAY[5.67,7.87]))";
+        String dml = "UPSERT INTO " + tableName + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_PREPEND(9.0,ARRAY[5.67,7.87]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{new Double(9), 5.67, 7.87};
@@ -402,26 +420,28 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithUpsertSelect1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        String baseTable = generateRandomString();
+        String source = baseTable + "_SOURCE";
+        String target = baseTable + "_TARGET";
+        String ddl = "CREATE TABLE " + source + " (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
+        ddl = "CREATE TABLE " + target + "(region_name VARCHAR PRIMARY KEY,doubles DOUBLE[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area',ARRAY_PREPEND(9.0,ARRAY[5.67,7.87]))";
+        String dml = "UPSERT INTO " + source + "(region_name,doubles) VALUES('SF Bay Area',ARRAY_PREPEND(9.0,ARRAY[5.67,7.87]))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,doubles) VALUES('SF Bay Area2',ARRAY_PREPEND(9.2,ARRAY[56.7,7.87]))";
+        dml = "UPSERT INTO " + source + "(region_name,doubles) VALUES('SF Bay Area2',ARRAY_PREPEND(9.2,ARRAY[56.7,7.87]))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, doubles) SELECT region_name, ARRAY_PREPEND(5,doubles) FROM source";
+        dml = "UPSERT INTO " + target + "(region_name, doubles) SELECT region_name, ARRAY_PREPEND(5,doubles) FROM " + source;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT doubles FROM target");
+        rs = conn.createStatement().executeQuery("SELECT doubles FROM " + target);
         assertTrue(rs.next());
 
         Double[] doubles = new Double[]{new Double(5), new Double(9), 5.67, 7.87};
@@ -440,26 +460,28 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionWithUpsertSelect2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE source (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        String baseTable = generateRandomString();
+        String source = baseTable + "_SOURCE";
+        String target = baseTable + "_TARGET";
+        String ddl = "CREATE TABLE " + source + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE TABLE target (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
+        ddl = "CREATE TABLE " + target + " (region_name VARCHAR PRIMARY KEY,varchars VARCHAR[])";
         conn.createStatement().execute(ddl);
 
-        String dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area',ARRAY_PREPEND('c',ARRAY['abcd','b']))";
+        String dml = "UPSERT INTO " + source + "(region_name,varchars) VALUES('SF Bay Area',ARRAY_PREPEND('c',ARRAY['abcd','b']))";
         conn.createStatement().execute(dml);
 
-        dml = "UPSERT INTO source(region_name,varchars) VALUES('SF Bay Area2',ARRAY_PREPEND('something',ARRAY['d','fgh']))";
+        dml = "UPSERT INTO " + source + "(region_name,varchars) VALUES('SF Bay Area2',ARRAY_PREPEND('something',ARRAY['d','fgh']))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        dml = "UPSERT INTO target(region_name, varchars) SELECT region_name, ARRAY_PREPEND('stu',varchars) FROM source";
+        dml = "UPSERT INTO " + target + "(region_name, varchars) SELECT region_name, ARRAY_PREPEND('stu',varchars) FROM " + source;
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT varchars FROM target");
+        rs = conn.createStatement().executeQuery("SELECT varchars FROM " + target);
         assertTrue(rs.next());
 
         String[] strings = new String[]{"stu", "c", "abcd", "b"};
@@ -478,10 +500,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere1() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[123,2345,46345,23234,456]=ARRAY_PREPEND(123,integers)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[123,2345,46345,23234,456]=ARRAY_PREPEND(123,integers)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -491,10 +514,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere2() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE varchars[1]=ANY(ARRAY_PREPEND('1234',ARRAY['2345','46345','23234']))");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE varchars[1]=ANY(ARRAY_PREPEND('1234',ARRAY['2345','46345','23234']))");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -504,10 +528,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere3() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['1234','2345','46345','23234']=ARRAY_PREPEND('1234',ARRAY['2345','46345','23234'])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['1234','2345','46345','23234']=ARRAY_PREPEND('1234',ARRAY['2345','46345','23234'])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -517,10 +542,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere4() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[123.4,23.45,4634.5,2.3234]=ARRAY_PREPEND(123.4,ARRAY[23.45,4634.5,2.3234])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[123.4,23.45,4634.5,2.3234]=ARRAY_PREPEND(123.4,ARRAY[23.45,4634.5,2.3234])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -530,10 +556,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere5() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY['foo','2345','46345','23234']=ARRAY_PREPEND('foo',varchars)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY['foo','2345','46345','23234']=ARRAY_PREPEND('foo',varchars)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -543,10 +570,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere6() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE chars2=ARRAY_PREPEND('foo',chars)");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE chars2=ARRAY_PREPEND('foo',chars)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -556,10 +584,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionInWhere7() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String  tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM regions WHERE ARRAY[4,2,3]=ARRAY_PREPEND(4,ARRAY[2,3])");
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName + " WHERE ARRAY[4,2,3]=ARRAY_PREPEND(4,ARRAY[2,3])");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -569,10 +598,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test(expected = SQLException.class)
     public void testArrayPrependFunctionCharLimitCheck() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTables(conn);
+        String tableName = generateRandomString();
+        initTables(conn, tableName);
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(char1,chars) FROM regions WHERE region_name = 'SF Bay Area'");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(char1,chars) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"wert", "a", "bbbb", "c", "ddd", "e"};
@@ -586,10 +616,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionIntegerDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "INTEGER", "23");
+        String tableName = generateRandomString();
+        initTablesDesc(conn, tableName, "INTEGER", "23");
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,integers) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,integers) FROM " + tableName);
         assertTrue(rs.next());
 
         Integer[] integers = new Integer[]{23, 2345, 46345, 23234, 456};
@@ -604,10 +635,11 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionVarcharDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "VARCHAR", "'e'");
+        String tableName = generateRandomString();
+        initTablesDesc(conn, tableName, "VARCHAR", "'e'");
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,varchars) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,varchars) FROM " + tableName);
         assertTrue(rs.next());
 
         String[] strings = new String[]{"e", "2345", "46345", "23234"};
@@ -621,9 +653,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionBigIntDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "BIGINT", "1112");
+        String  tableName = generateRandomString();
+        initTablesDesc(conn, tableName, "BIGINT", "1112");
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,bigints) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,bigints) FROM " +  tableName);
         assertTrue(rs.next());
 
         Long[] longs = new Long[]{1112l, 12l, 34l, 56l, 78l, 910l};
@@ -637,9 +670,10 @@ public class ArrayPrependFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testArrayPrependFunctionBooleanDesc() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTablesDesc(conn, "BOOLEAN", "false");
+        String tableName = generateRandomString();
+        initTablesDesc(conn, tableName, "BOOLEAN", "false");
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,bools) FROM regions");
+        rs = conn.createStatement().executeQuery("SELECT ARRAY_PREPEND(pk,bools) FROM " + tableName);
         assertTrue(rs.next());
 
         Boolean[] booleans = new Boolean[]{false, true, false};

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoPartitionViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoPartitionViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoPartitionViewsIT.java
index c66a7c8..95ac2cb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoPartitionViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutoPartitionViewsIT.java
@@ -47,7 +47,7 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
-public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
+public class AutoPartitionViewsIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private String tableDDLOptions;
     private boolean isMultiTenant;
@@ -64,7 +64,7 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
 
     public AutoPartitionViewsIT(boolean salted, boolean isMultiTenant) {
         this.isMultiTenant = isMultiTenant;
-        StringBuilder optionBuilder = new StringBuilder(" AUTO_PARTITION_SEQ=\"TSDB.METRIC_ID_SEQ\"");
+        StringBuilder optionBuilder = new StringBuilder(" AUTO_PARTITION_SEQ=\"%s\"");
         if (salted) optionBuilder.append(", SALTED=4 ");
         if (isMultiTenant) optionBuilder.append(", MULTI_TENANT=true ");
         this.tableDDLOptions = optionBuilder.toString();
@@ -79,13 +79,17 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
                 Connection viewConn2 =
                         isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1)
                                 : DriverManager.getConnection(getUrl())) {
+            String tableName = generateRandomString();
+            String autoSeqName = generateRandomString();
+
             try {
+
                 String ddl =
                         String.format(
-                            "CREATE TABLE metric_table (%s metricId VARCHAR, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
+                            "CREATE TABLE " + tableName + " (%s metricId VARCHAR, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
                                 isMultiTenant ? "tenantId VARCHAR, " : "", 
-                                isMultiTenant ? "tenantId, ": "", 
-                                tableDDLOptions);
+                                isMultiTenant ? "tenantId, ": "", String.format(tableDDLOptions, autoSeqName)
+                                );
                 conn.createStatement().execute(ddl);
                 fail("Sequence value must be castable to the auto partition id column data type");
             } catch (SQLException e) {
@@ -94,36 +98,41 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
                             .getErrorCode(),
                     e.getErrorCode());
             }
-
             String ddl =
                     String.format(
-                        "CREATE TABLE metric_table (%s metricId INTEGER NOT NULL, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
+                        "CREATE TABLE " + tableName + " (%s metricId INTEGER NOT NULL, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
                             isMultiTenant ? "tenantId VARCHAR NOT NULL, " : "", 
-                            isMultiTenant ? "tenantId, ": "", 
-                            tableDDLOptions);
+                            isMultiTenant ? "tenantId, ": "",
+                            String.format(tableDDLOptions, autoSeqName));
             conn.createStatement().execute(ddl);
-            
+
+
+            String baseViewName = generateRandomString();
+            String metricView1 = baseViewName + "_VIEW1";
+            String metricView2 = baseViewName + "_VIEW2";
+            String metricView3 = baseViewName + "_VIEW3";
+            String metricView4 = baseViewName + "_VIEW4";
             try {
                 viewConn1.createStatement().execute(
-                    "CREATE VIEW metric1 AS SELECT * FROM metric_table");
+                    "CREATE VIEW " + metricView1 + "  AS SELECT * FROM " + tableName);
                 fail("Auto-partition sequence must be created before view is created");
             } catch (SequenceNotFoundException e) {
             }
 
             conn.createStatement().execute(
-                "CREATE SEQUENCE TSDB.metric_id_seq start with " + (Integer.MAX_VALUE-2) + " cache 1");
+                "CREATE SEQUENCE " + autoSeqName + " start with " + (Integer.MAX_VALUE-2) + " cache 1");
             viewConn1.createStatement().execute(
-                "CREATE VIEW metric1 AS SELECT * FROM metric_table WHERE val2=1.2");
+                "CREATE VIEW " + metricView1 + " AS SELECT * FROM " + tableName + " WHERE val2=1.2");
             // create a view without a where clause
             viewConn1.createStatement().execute(
-                    "CREATE VIEW metric2 AS SELECT * FROM metric_table");
+                    "CREATE VIEW " + metricView2 + " AS SELECT * FROM " + tableName);
             // create a view with a complex where clause
             viewConn1.createStatement().execute(
-                "CREATE VIEW metric3 AS SELECT * FROM metric_table WHERE val1=1.0 OR val2=2.0");
+                "CREATE VIEW " + metricView3 + " AS SELECT * FROM " + tableName + " WHERE val1=1.0 OR val2=2.0");
 
             try {
                 viewConn1.createStatement().execute(
-                    "CREATE VIEW metric4 AS SELECT * FROM metric_table");
+                    "CREATE VIEW " + metricView4 + " AS SELECT * FROM " + tableName);
                 fail("Creating a view with a partition id that is too large should fail");
             } catch (SQLException e) {
                 assertEquals(SQLExceptionCode.CANNOT_COERCE_AUTO_PARTITION_ID.getErrorCode(),
@@ -132,23 +141,23 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
 
             if (isMultiTenant) {
                 // load tables into cache
-                viewConn1.createStatement().execute("SELECT * FROM METRIC1");
-                viewConn1.createStatement().execute("SELECT * FROM METRIC2");
-                viewConn1.createStatement().execute("SELECT * FROM METRIC3");
+                viewConn1.createStatement().execute("SELECT * FROM " + metricView1);
+                viewConn1.createStatement().execute("SELECT * FROM " + metricView2);
+                viewConn1.createStatement().execute("SELECT * FROM " + metricView3);
             }
             PhoenixConnection pconn = viewConn1.unwrap(PhoenixConnection.class);
-            PTable view1 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC1"));
-            PTable view2 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC2"));
-            PTable view3 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC3"));
+            PTable view1 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView1));
+            PTable view2 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView2));
+            PTable view3 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView3));
             
             // verify the view statement was set correctly 
             String expectedViewStatement1 =
-                    "SELECT * FROM \"METRIC_TABLE\" WHERE VAL2 = 1.2 AND METRICID = "
+                    "SELECT * FROM \"" + tableName + "\" WHERE VAL2 = 1.2 AND METRICID = "
                             + (Integer.MAX_VALUE - 2);
             String expectedViewStatement2 =
-                    "SELECT * FROM \"METRIC_TABLE\" WHERE METRICID = " + (Integer.MAX_VALUE - 1);
+                    "SELECT * FROM \"" + tableName + "\" WHERE METRICID = " + (Integer.MAX_VALUE - 1);
             String expectedViewStatement3 =
-                    "SELECT * FROM \"METRIC_TABLE\" WHERE (VAL1 = 1.0 OR VAL2 = 2.0) AND METRICID = " + Integer.MAX_VALUE;
+                    "SELECT * FROM \"" + tableName + "\" WHERE (VAL1 = 1.0 OR VAL2 = 2.0) AND METRICID = " + Integer.MAX_VALUE;
             assertEquals("Unexpected view statement", expectedViewStatement1,
                 view1.getViewStatement());
             assertEquals("Unexpected view statement", expectedViewStatement2,
@@ -181,13 +190,13 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
                 expectedPartition3, partitionCol3.getViewConstant());
 
             // verify that the table was created correctly on the server
-            viewConn2.createStatement().execute("SELECT * FROM METRIC1");
-            viewConn2.createStatement().execute("SELECT * FROM METRIC2");
-            viewConn2.createStatement().execute("SELECT * FROM METRIC3");
+            viewConn2.createStatement().execute("SELECT * FROM " + metricView1);
+            viewConn2.createStatement().execute("SELECT * FROM " + metricView2 );
+            viewConn2.createStatement().execute("SELECT * FROM " + metricView3);
             pconn = viewConn2.unwrap(PhoenixConnection.class);
-            view1 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC1"));
-            view2 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC2"));
-            view3 = pconn.getTable(new PTableKey(pconn.getTenantId(), "METRIC3"));
+            view1 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView1));
+            view2 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView2));
+            view3 = pconn.getTable(new PTableKey(pconn.getTenantId(), metricView3));
             
             // verify the view statement was set correctly 
             assertEquals("Unexpected view statement", expectedViewStatement1,
@@ -225,37 +234,44 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
                 Connection viewConn2 =
                         isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL2)
                                 : DriverManager.getConnection(getUrl())) {
+            String tableName = generateRandomString();
+            String autoSeqName = generateRandomString();
+
             String ddl =
                     String.format(
-                        "CREATE TABLE hbase.metric_table (%s metricId INTEGER NOT NULL, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
+                        "CREATE TABLE " + tableName + " (%s metricId INTEGER NOT NULL, val1 DOUBLE, val2 DOUBLE CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
                             isMultiTenant ? "tenantId VARCHAR NOT NULL, " : "", 
                             isMultiTenant ? "tenantId, ": "", 
-                            tableDDLOptions);
+                            String.format(tableDDLOptions, autoSeqName));
             conn.createStatement().execute(ddl);
-            conn.createStatement().execute("CREATE SEQUENCE TSDB.metric_id_seq CACHE 1");
+            conn.createStatement().execute("CREATE SEQUENCE " + autoSeqName + " CACHE 1");
+
+            String baseViewName = generateRandomString();
+            String metricView1 = baseViewName + "_VIEW1";
+            String metricView2 = baseViewName + "_VIEW2";
             // create a view
             viewConn1.createStatement().execute(
-                "CREATE VIEW metric1 AS SELECT * FROM hbase.metric_table WHERE val2=1.2");
+                "CREATE VIEW " + metricView1 + " AS SELECT * FROM " + tableName + " WHERE val2=1.2");
             try {
                 // create the same view which should fail
                 viewConn1.createStatement()
-                        .execute("CREATE VIEW metric1 AS SELECT * FROM hbase.metric_table");
+                        .execute("CREATE VIEW " + metricView1 + " AS SELECT * FROM " + tableName);
                 fail("view should already exist");
             } catch (TableAlreadyExistsException e) {
             }
 
             // create a second view (without a where clause)
             viewConn2.createStatement().execute(
-                "CREATE VIEW metric2 AS SELECT * FROM hbase.metric_table");
+                "CREATE VIEW " + metricView2 + " AS SELECT * FROM " +  tableName);
 
             // upsert a row into each view
-            viewConn1.createStatement().execute("UPSERT INTO metric1(val1) VALUES(1.1)");
+            viewConn1.createStatement().execute("UPSERT INTO " + metricView1 + "(val1) VALUES(1.1)");
             viewConn1.commit();
-            viewConn2.createStatement().execute("UPSERT INTO metric2(val1,val2) VALUES(2.1,2.2)");
+            viewConn2.createStatement().execute("UPSERT INTO " + metricView2 + "(val1,val2) VALUES(2.1,2.2)");
             viewConn2.commit();
 
             // query the base table
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM hbase.metric_table");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             int offset = 0;
             if (isMultiTenant) {
@@ -276,7 +292,7 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // query the first view
-            rs = viewConn1.createStatement().executeQuery("SELECT * FROM metric1");
+            rs = viewConn1.createStatement().executeQuery("SELECT * FROM " + metricView1);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(1.1, rs.getDouble(2), 1e-6);
@@ -284,7 +300,7 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // query the second view
-            rs = viewConn2.createStatement().executeQuery("SELECT * FROM metric2");
+            rs = viewConn2.createStatement().executeQuery("SELECT * FROM " + metricView2);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             assertEquals(2.1, rs.getDouble(2), 1e-6);
@@ -299,32 +315,37 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
                 Connection viewConn1 =
                         isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1)
                                 : DriverManager.getConnection(getUrl())) {
+            String tableName = generateRandomString();
+            String autoSeqName = generateRandomString();
+
             String ddl =
                     String.format(
-                        "CREATE TABLE hbase.metric_table (%s metricId INTEGER NOT NULL, val1 DOUBLE, CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
+                        "CREATE TABLE " + tableName + " (%s metricId INTEGER NOT NULL, val1 DOUBLE, CONSTRAINT PK PRIMARY KEY( %s metricId)) %s",
                             isMultiTenant ? "tenantId VARCHAR NOT NULL, " : "", 
                             isMultiTenant ? "tenantId, ": "", 
-                            tableDDLOptions);
+                            String.format(tableDDLOptions, autoSeqName));
             conn.createStatement().execute(ddl);
-            conn.createStatement().execute("CREATE SEQUENCE TSDB.metric_id_seq CACHE 1");
+            conn.createStatement().execute("CREATE SEQUENCE " + autoSeqName + " CACHE 1");
+
+            String metricView = generateRandomString() + "_VIEW";
             // create a view
             viewConn1.createStatement().execute(
-                "CREATE VIEW metric1 AS SELECT * FROM hbase.metric_table");
+                "CREATE VIEW " + metricView + " AS SELECT * FROM " + tableName);
             
             // add a column to the base table
             conn.createStatement().execute(
-                    "ALTER TABLE hbase.metric_table add val2 DOUBLE");
+                    "ALTER TABLE " + tableName + " add val2 DOUBLE");
             
             // add a column to the view
             viewConn1.createStatement().execute(
-                    "ALTER VIEW metric1 add val3 DOUBLE");
+                    "ALTER VIEW " + metricView + " add val3 DOUBLE");
 
             // upsert a row into the view
-            viewConn1.createStatement().execute("UPSERT INTO metric1(val1,val2,val3) VALUES(1.1,1.2,1.3)");
+            viewConn1.createStatement().execute("UPSERT INTO " + metricView + "(val1,val2,val3) VALUES(1.1,1.2,1.3)");
             viewConn1.commit();
 
             // query the base table
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM hbase.metric_table");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName);
             assertTrue(rs.next());
             int offset = 0;
             if (isMultiTenant) {
@@ -337,7 +358,7 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             // query the view
-            rs = viewConn1.createStatement().executeQuery("SELECT * FROM metric1");
+            rs = viewConn1.createStatement().executeQuery("SELECT * FROM " + metricView);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(1.1, rs.getDouble(2), 1e-6);
@@ -347,21 +368,21 @@ public class AutoPartitionViewsIT extends BaseHBaseManagedTimeIT {
 
             // drop a column from the base table
             conn.createStatement().execute(
-                    "ALTER TABLE hbase.metric_table DROP COLUMN val2");
+                    "ALTER TABLE " + tableName + " DROP COLUMN val2");
             
             // add a column to the view
             viewConn1.createStatement().execute(
-                    "ALTER VIEW metric1 DROP COLUMN val3");
+                    "ALTER VIEW " + metricView + " DROP COLUMN val3");
             
             // verify columns don't exist
             try {
-                viewConn1.createStatement().executeQuery("SELECT val2 FROM metric1");
+                viewConn1.createStatement().executeQuery("SELECT val2 FROM " + metricView);
                 fail("column should have been dropped");
             }
             catch (ColumnNotFoundException e) {
             }
             try {
-                viewConn1.createStatement().executeQuery("SELECT val3 FROM metric1");
+                viewConn1.createStatement().executeQuery("SELECT val3 FROM " + metricView);
                 fail("column should have been dropped");
             }
             catch (ColumnNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeTableReuseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeTableReuseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeTableReuseIT.java
index 02ebf98..c40ec59 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeTableReuseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeTableReuseIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 
 import javax.annotation.concurrent.NotThreadSafe;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.After;
@@ -47,6 +48,10 @@ import org.junit.experimental.categories.Category;
 @NotThreadSafe
 @Category(HBaseManagedTimeTableReuseTest.class)
 public class BaseHBaseManagedTimeTableReuseIT extends BaseTest {
+    protected static Configuration getTestClusterConfig() {
+        // don't want callers to modify config.
+        return new Configuration(config);
+    }
 
     @BeforeClass
     public static void doSetup() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index aa5068b..9ad12e5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 
 import java.io.IOException;
 import java.sql.Connection;
@@ -29,7 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -61,7 +61,7 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
     protected static final String tenantId = getOrganizationId();
     protected static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
     protected static final long BATCH_SIZE = 3;
-    
+
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
@@ -90,7 +90,7 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
     @Before
     public void initTable() throws Exception {
          ts = nextTimestamp();
-        initATableValues(tenantId, getDefaultSplits(tenantId), date=new Date(System.currentTimeMillis()), ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), date=new Date(System.currentTimeMillis()), ts, getUrl());
         if (indexDDL != null && indexDDL.length() > 0) {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
index 04f4268..e0916a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificViewIndexIT.java
@@ -35,7 +35,7 @@ import org.apache.phoenix.util.QueryUtil;
 
 import com.google.common.collect.Lists;
 
-public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
+public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     
     public static final String TENANT1_ID = "tenant1";
     public static final String TENANT2_ID = "tenant2";
@@ -48,24 +48,26 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
     }
     
     protected void testUpdatableView(Integer saltBuckets, boolean localIndex) throws Exception {
-        createBaseTable("t", saltBuckets, true);
+        String tableName = generateRandomString();
+        createBaseTable(tableName, saltBuckets, true);
         Connection conn = createTenantConnection(TENANT1_ID);
         try {
-            createAndPopulateTenantView(conn, TENANT1_ID, "t", "");
-            createAndVerifyIndex(conn, saltBuckets, TENANT1_ID, "", localIndex);
-            verifyViewData(conn, "");
+            String viewName = createAndPopulateTenantView(conn, TENANT1_ID, tableName, "");
+            createAndVerifyIndex(conn, viewName, tableName, saltBuckets, TENANT1_ID, "", localIndex);
+            verifyViewData(conn, viewName, "");
         } finally {
             try { conn.close();} catch (Exception ignored) {}
         }
     }
 
     protected void testUpdatableViewNonString(Integer saltBuckets, boolean localIndex) throws Exception {
-        createBaseTable("t", saltBuckets, false);
+        String tableName = generateRandomString();
+        createBaseTable(tableName, saltBuckets, false);
         Connection conn = createTenantConnection(NON_STRING_TENANT_ID);
         try {
-            createAndPopulateTenantView(conn, NON_STRING_TENANT_ID, "t", "");
-            createAndVerifyIndexNonStringTenantId(conn, NON_STRING_TENANT_ID, "");
-            verifyViewData(conn, "");
+            String viewName = createAndPopulateTenantView(conn, NON_STRING_TENANT_ID, tableName, "");
+            createAndVerifyIndexNonStringTenantId(conn, viewName, tableName, NON_STRING_TENANT_ID, "");
+            verifyViewData(conn, viewName, "");
         } finally {
             try { conn.close();} catch (Exception ignored) {}
         }
@@ -76,7 +78,8 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void testUpdatableViewsWithSameNameDifferentTenants(Integer saltBuckets, boolean localIndex) throws Exception {
-        createBaseTable("t", saltBuckets, true);
+        String tableName = generateRandomString();
+        createBaseTable(tableName, saltBuckets, true);
         Connection conn1 = createTenantConnection(TENANT1_ID);
         Connection conn2 = createTenantConnection(TENANT2_ID);
         try {
@@ -84,14 +87,14 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
             String prefixForTenant2Data = "TII";
             
             // tenant views with same name for two different tables
-            createAndPopulateTenantView(conn1, TENANT1_ID, "t", prefixForTenant1Data);
-            createAndPopulateTenantView(conn2, TENANT2_ID, "t", prefixForTenant2Data);
+            String viewName1 = createAndPopulateTenantView(conn1, TENANT1_ID, tableName, prefixForTenant1Data);
+            String viewName2 = createAndPopulateTenantView(conn2, TENANT2_ID, tableName, prefixForTenant2Data);
             
-            createAndVerifyIndex(conn1, saltBuckets, TENANT1_ID, prefixForTenant1Data, localIndex);
-            createAndVerifyIndex(conn2, saltBuckets, TENANT2_ID, prefixForTenant2Data, localIndex);
+            createAndVerifyIndex(conn1, viewName1, tableName, saltBuckets, TENANT1_ID, prefixForTenant1Data, localIndex);
+            createAndVerifyIndex(conn2, viewName2, tableName, saltBuckets, TENANT2_ID, prefixForTenant2Data, localIndex);
             
-            verifyViewData(conn1, prefixForTenant1Data);
-            verifyViewData(conn2, prefixForTenant2Data);
+            verifyViewData(conn1, viewName1, prefixForTenant1Data);
+            verifyViewData(conn2, viewName2, prefixForTenant2Data);
         } finally {
             try { conn1.close();} catch (Exception ignored) {}
             try { conn2.close();} catch (Exception ignored) {}
@@ -111,51 +114,56 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
         conn.close();
     }
     
-    private void createAndPopulateTenantView(Connection conn, String tenantId, String baseTable, String valuePrefix) throws SQLException {
-        String ddl = "CREATE VIEW v(v2 VARCHAR) AS SELECT * FROM " + baseTable + " WHERE k1 = 1";
+    private String createAndPopulateTenantView(Connection conn, String tenantId, String baseTable, String valuePrefix) throws SQLException {
+        String viewName = generateRandomString();
+        String ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR) AS SELECT * FROM " + baseTable + " WHERE k1 = 1";
         conn.createStatement().execute(ddl);
-        tenantViewsToDelete.add(new Pair<String, String>(tenantId, "v"));
+        tenantViewsToDelete.add(new Pair<String, String>(tenantId, viewName ));
         for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO v(k2,v1,v2) VALUES(" + i + ",'" + valuePrefix + "v1-" + (i%5) + "','" + valuePrefix + "v2-" + (i%2) + "')");
+            conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,v1,v2) VALUES(" + i + ",'" + valuePrefix + "v1-" + (i%5) + "','" + valuePrefix + "v2-" + (i%2) + "')");
         }
         conn.commit();
+        return viewName;
     }
     
-    private void createAndVerifyIndex(Connection conn, Integer saltBuckets, String tenantId, String valuePrefix, boolean localIndex) throws SQLException {
+    private void createAndVerifyIndex(Connection conn, String viewName, String tableName, Integer saltBuckets, String tenantId, String valuePrefix, boolean localIndex) throws SQLException {
+        String indexName = generateRandomString();
         if(localIndex){
-            conn.createStatement().execute("CREATE LOCAL INDEX i ON v(v2)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + viewName + "(v2)");
         } else {
-            conn.createStatement().execute("CREATE INDEX i ON v(v2)");
+            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + viewName + "(v2)");
         }
-        conn.createStatement().execute("UPSERT INTO v(k2,v1,v2) VALUES (-1, 'blah', 'superblah')"); // sanity check that we can upsert after index is there
+        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,v1,v2) VALUES (-1, 'blah', 'superblah')"); // sanity check that we can upsert after index is there
         conn.commit();
-        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'");
+        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM " + viewName + " WHERE v2='" + valuePrefix + "v2-1'");
         if(localIndex){
             assertEquals(saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER T [1,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER " + tableName + " [1,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY\n"
                             + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         } else {
             String expected = saltBuckets == null ? 
-                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + tableName + " [-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
                             + "    SERVER FILTER BY FIRST KEY ONLY" :
-                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_T [0,-32768,'" + tenantId + "','" + valuePrefix + "v2-1'] - ["+(saltBuckets.intValue()-1)+",-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+                    "CLIENT PARALLEL 3-WAY RANGE SCAN OVER _IDX_" + tableName + " [0,-32768,'" + tenantId + "','" + valuePrefix + "v2-1'] - ["+(saltBuckets.intValue()-1)+",-32768,'" + tenantId + "','" + valuePrefix + "v2-1']\n"
+
                   + "    SERVER FILTER BY FIRST KEY ONLY\n"
                   + "CLIENT MERGE SORT";
             assertEquals(expected, QueryUtil.getExplainPlan(rs));
         }
     }
 
-    private void createAndVerifyIndexNonStringTenantId(Connection conn, String tenantId, String valuePrefix) throws SQLException {
-        conn.createStatement().execute("CREATE LOCAL INDEX i ON v(v2)");
-        conn.createStatement().execute("UPSERT INTO v(k2,v1,v2) VALUES (-1, 'blah', 'superblah')"); // sanity check that we can upsert after index is there
+    private void createAndVerifyIndexNonStringTenantId(Connection conn, String viewName, String tableName, String tenantId, String valuePrefix) throws SQLException {
+        String indexName = generateRandomString();
+        conn.createStatement().execute("CREATE LOCAL INDEX " + indexName + " ON " + viewName + "(v2)");
+        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,v1,v2) VALUES (-1, 'blah', 'superblah')"); // sanity check that we can upsert after index is there
         conn.commit();
-        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'");
+        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT k1, k2, v2 FROM " + viewName + " WHERE v2='" + valuePrefix + "v2-1'");
         assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1," + tenantId + ",'" + valuePrefix + "v2-1']\n"
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1," + tenantId + ",'" + valuePrefix + "v2-1']\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY\n"
                         + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
     }
@@ -167,8 +175,8 @@ public class BaseTenantSpecificViewIndexIT extends BaseHBaseManagedTimeIT {
     }
     
     @SuppressWarnings("unchecked")
-    private void verifyViewData(Connection conn, String valuePrefix) throws SQLException {
-        String query = "SELECT k1, k2, v2 FROM v WHERE v2='" + valuePrefix + "v2-1'";
+    private void verifyViewData(Connection conn, String viewName, String valuePrefix) throws SQLException {
+        String query = "SELECT k1, k2, v2 FROM " + viewName + " WHERE v2='" + valuePrefix + "v2-1'";
         ResultSet rs = conn.createStatement().executeQuery(query);
         List<List<Object>> expectedResultsA = Lists.newArrayList(
             Arrays.<Object>asList(1,1, valuePrefix + "v2-1"),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
index 544284d..5f656e9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BinaryRowKeyIT.java
@@ -34,15 +34,16 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
 
-public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
+public class BinaryRowKeyIT extends BaseHBaseManagedTimeTableReuseIT {
 
-    private static void initTableValues() throws SQLException {
+    private static void initTableValues(String tableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         
         try {
-            String ddl = "CREATE TABLE test_table" +
+
+            String ddl = "CREATE TABLE " + tableName +
                     "   (a_binary binary(10) not null, \n" +
                     "    a_string varchar not null, \n" +
                     "    b_binary varbinary \n" +
@@ -52,7 +53,7 @@ public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
             String query;
             PreparedStatement stmt;
             
-            query = "UPSERT INTO test_table"
+            query = "UPSERT INTO " + tableName
                     + "(a_binary, a_string) "
                     + "VALUES(?,?)";
             stmt = conn.prepareStatement(query);
@@ -76,11 +77,12 @@ public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
-            initTableValues();
+            String tableName = generateRandomString();
+            initTableValues(tableName);
             conn.setAutoCommit(true);
-            conn.createStatement().execute("DELETE FROM test_table");
+            conn.createStatement().execute("DELETE FROM " + tableName );
            
-            String query = "UPSERT INTO test_table"
+            String query = "UPSERT INTO " + tableName
                     + "(a_binary, a_string) "
                     + "VALUES(?,?)";
             PreparedStatement stmt = conn.prepareStatement(query);
@@ -88,7 +90,7 @@ public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
             stmt.setString(2, "a");
             stmt.execute();
             
-            ResultSet rs = conn.createStatement().executeQuery("SELECT a_string FROM test_table");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT a_string FROM " + tableName);
             assertTrue(rs.next());
             assertEquals("a",rs.getString(1));
             assertFalse(rs.next());
@@ -103,9 +105,9 @@ public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         
         try {
-            initTableValues();
-            
-            String query = "SELECT * FROM test_table";
+            String tableName = generateRandomString();
+            initTableValues(tableName);
+            String query = "SELECT * FROM " + tableName;
             PreparedStatement stmt = conn.prepareStatement(query);
             ResultSet rs = stmt.executeQuery();
             
@@ -129,15 +131,16 @@ public class BinaryRowKeyIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         
         try {
-            initTableValues();
+            String tableName = generateRandomString();
+            initTableValues(tableName);
             
-            String query = "UPSERT INTO test_table (a_binary, a_string, b_binary) "
-                    + " SELECT a_binary, a_string, a_binary FROM test_table";
+            String query = "UPSERT INTO " + tableName + " (a_binary, a_string, b_binary) "
+                    + " SELECT a_binary, a_string, a_binary FROM " + tableName;
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.execute();
             conn.commit();
             
-            query = "SELECT a_binary, b_binary FROM test_table";
+            query = "SELECT a_binary, b_binary FROM " + tableName;
             stmt = conn.prepareStatement(query);
             ResultSet rs = stmt.executeQuery();
             


[06/50] [abbrv] phoenix git commit: PHOENIX-3141 UnnestArrayIT causes builds to hang

Posted by ma...@apache.org.
PHOENIX-3141 UnnestArrayIT causes builds to hang


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

Branch: refs/heads/calcite
Commit: 50b3f948480286089896d81ff006720e6049a118
Parents: 243e5e2
Author: Samarth <sa...@salesforce.com>
Authored: Tue Aug 2 21:41:47 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Aug 2 21:41:47 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/UnnestArrayIT.java | 18 ++++++++----------
 1 file changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/50b3f948/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnnestArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnnestArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnnestArrayIT.java
index 4ca73e6..9c80cd5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnnestArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnnestArrayIT.java
@@ -17,10 +17,10 @@
  */
 package org.apache.phoenix.end2end;
 
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.junit.Ignore;
-import org.junit.Test;
+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.assertTrue;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -28,13 +28,11 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Properties;
 
-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.assertTrue;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
 
-@Ignore
-public class UnnestArrayIT extends BaseClientManagedTimeIT {
+public abstract class UnnestArrayIT extends BaseClientManagedTimeIT {
 
     private static long timestamp;
 


[28/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6c96293..142595f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,5 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" 
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
@@ -9,17 +9,17 @@
   <description>A SQL layer over HBase</description>
 
   <licenses>
-      <license>
-          <name>The Apache Software License, Version 2.0</name>
-          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
-          <distribution>repo</distribution>
-          <comments />
-      </license>
+    <license>
+      <name>The Apache Software License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+      <distribution>repo</distribution>
+      <comments />
+    </license>
   </licenses>
 
   <organization>
-      <name>Apache Software Foundation</name>
-      <url>http://www.apache.org</url>
+    <name>Apache Software Foundation</name>
+    <url>http://www.apache.org</url>
   </organization>
 
   <modules>
@@ -112,14 +112,14 @@
     <maven-build-helper-plugin.version>1.9.1</maven-build-helper-plugin.version>
     <maven-surefire-plugin.version>2.19.1</maven-surefire-plugin.version>
     <maven-failsafe-plugin.version>2.19.1</maven-failsafe-plugin.version>
-    
+
     <maven-dependency-plugin.version>2.1</maven-dependency-plugin.version>
     <maven.assembly.version>2.5.2</maven.assembly.version>
 
     <!-- Plugin options -->
     <numForkedUT>3</numForkedUT>
     <numForkedIT>5</numForkedIT>
-    
+
     <!-- Set default encoding so multi-byte tests work correctly on the Mac -->
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
@@ -227,40 +227,40 @@
           <version>${maven-failsafe-plugin.version}</version>
           <executions>
             <execution>
-               <id>ClientManagedTimeTests</id>
-                 <configuration>
-                 <encoding>UTF-8</encoding>
-                 <forkCount>${numForkedIT}</forkCount>
-                 <runOrder>alphabetical</runOrder>
-                 <reuseForks>true</reuseForks>
-                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
-                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
-                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
-                 <groups>org.apache.phoenix.end2end.ClientManagedTimeTest</groups>
-                 <shutdown>kill</shutdown>
-               </configuration>
-               <goals>
-                 <goal>integration-test</goal>
-                 <goal>verify</goal>
-               </goals>
+              <id>ClientManagedTimeTests</id>
+              <configuration>
+                <encoding>UTF-8</encoding>
+                <forkCount>${numForkedIT}</forkCount>
+                <runOrder>alphabetical</runOrder>
+                <reuseForks>true</reuseForks>
+                <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
+                <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                <groups>org.apache.phoenix.end2end.ClientManagedTimeTest</groups>
+                <shutdown>kill</shutdown>
+              </configuration>
+              <goals>
+                <goal>integration-test</goal>
+                <goal>verify</goal>
+              </goals>
             </execution>
             <execution>
-               <id>HBaseManagedTimeTests</id>
-                 <configuration>
-                 <encoding>UTF-8</encoding>
-                 <forkCount>${numForkedIT}</forkCount>
-                 <runOrder>alphabetical</runOrder>
-                 <reuseForks>true</reuseForks>
-                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
-                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
-                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
-                 <groups>org.apache.phoenix.end2end.HBaseManagedTimeTest</groups>
-                 <shutdown>kill</shutdown>
-               </configuration>
-               <goals>
-                 <goal>integration-test</goal>
-                 <goal>verify</goal>
-               </goals>
+              <id>HBaseManagedTimeTests</id>
+              <configuration>
+                <encoding>UTF-8</encoding>
+                <forkCount>${numForkedIT}</forkCount>
+                <runOrder>alphabetical</runOrder>
+                <reuseForks>true</reuseForks>
+                <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
+                <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                <groups>org.apache.phoenix.end2end.HBaseManagedTimeTest</groups>
+                <shutdown>kill</shutdown>
+              </configuration>
+              <goals>
+                <goal>integration-test</goal>
+                <goal>verify</goal>
+              </goals>
             </execution>
             <execution>
               <id>HBaseManagedTimeTableReuseTest</id>
@@ -280,23 +280,23 @@
                 <goal>verify</goal>
               </goals>
             </execution>
-	        <execution>
-               <id>NeedTheirOwnClusterTests</id>
-                 <configuration>
-                 <encoding>UTF-8</encoding>
-                 <forkCount>${numForkedIT}</forkCount>
-                 <runOrder>alphabetical</runOrder>
-                 <reuseForks>false</reuseForks>
-                 <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
-                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
-                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
-                 <groups>org.apache.phoenix.end2end.NeedsOwnMiniClusterTest</groups>
-                 <shutdown>kill</shutdown>
-               </configuration>
-               <goals>
-                 <goal>integration-test</goal>
-                 <goal>verify</goal>
-               </goals>
+            <execution>
+              <id>NeedTheirOwnClusterTests</id>
+              <configuration>
+                <encoding>UTF-8</encoding>
+                <forkCount>${numForkedIT}</forkCount>
+                <runOrder>alphabetical</runOrder>
+                <reuseForks>false</reuseForks>
+                <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=256m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
+                <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+                <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
+                <groups>org.apache.phoenix.end2end.NeedsOwnMiniClusterTest</groups>
+                <shutdown>kill</shutdown>
+              </configuration>
+              <goals>
+                <goal>integration-test</goal>
+                <goal>verify</goal>
+              </goals>
             </execution>
           </executions>
         </plugin>
@@ -390,7 +390,7 @@
         <configuration>
           <forkCount>${numForkedUT}</forkCount>
           <reuseForks>true</reuseForks>
-          <argLine>-enableassertions -Xmx2250m -XX:MaxPermSize=128m 
+          <argLine>-enableassertions -Xmx2250m -XX:MaxPermSize=128m
             -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
           <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
           <shutdown>kill</shutdown>
@@ -404,7 +404,7 @@
         <executions>
           <execution>
             <phase>prepare-package
-                </phase>
+            </phase>
             <goals>
               <goal>test-jar</goal>
             </goals>
@@ -433,7 +433,7 @@
             <!-- Header on changelog isn't normal -->
             <exclude>CHANGES</exclude>
             <!-- IDE configuration -->
-	          <exclude>dev/phoenix.importorder</exclude>
+            <exclude>dev/phoenix.importorder</exclude>
             <!-- Release L&N -->
             <exclude>dev/release_files/LICENSE</exclude>
             <exclude>dev/release_files/NOTICE</exclude>
@@ -897,8 +897,8 @@
             <version>1.6</version>
             <executions>
               <execution>
-              <id>sign-artifacts</id>
-              <phase>verify</phase>
+                <id>sign-artifacts</id>
+                <phase>verify</phase>
                 <goals>
                   <goal>sign</goal>
                 </goals>
@@ -909,4 +909,4 @@
       </build>
     </profile>
   </profiles>
-</project>
+</project>
\ No newline at end of file


[46/50] [abbrv] phoenix git commit: PHOENIX-3189 Perform Kerberos login before ConnectionInfo is constructed

Posted by ma...@apache.org.
PHOENIX-3189 Perform Kerberos login before ConnectionInfo is constructed

Now that ConnectionInfo has the current User/UGI stored inside, we must
make sure that any automatic Kerberos login occurs before the ConnectionInfo
object is constructed. Otherwise, we will have multiple instances of
ConnectionInfo that differ only by the User, which will leak HBase/ZK
connections in the connectionQueryServicesMap. Also, protect the area
in which we perform logins to prevent concurrent clients from colliding.

Closes apache/phoenix#191


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

Branch: refs/heads/calcite
Commit: 2dc6be4e9ffb205aaa5b0f6a1f2746eed8d23426
Parents: 93a9c91
Author: Josh Elser <el...@apache.org>
Authored: Wed Aug 17 13:34:59 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 31 16:27:30 2016 -0400

----------------------------------------------------------------------
 phoenix-core/pom.xml                            |   5 +
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |  10 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  89 ++++-
 .../query/ConnectionQueryServicesImpl.java      |  24 +-
 .../apache/phoenix/util/InstanceResolver.java   |   7 +
 .../phoenix/jdbc/SecureUserConnectionsTest.java | 369 +++++++++++++++++++
 .../src/test/resources/log4j.properties         |   2 +
 pom.xml                                         |  18 +
 8 files changed, 504 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index ab2f80e..cf383f2 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -461,6 +461,11 @@
       <artifactId>hadoop-minicluster</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
         <groupId>org.jruby.joni</groupId>
         <artifactId>joni</artifactId>
         <version>${joni.version}</version>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
index 91d25ca..fa31dd9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -47,10 +47,10 @@ import org.apache.phoenix.query.HBaseFactoryProvider;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesImpl;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 
@@ -212,7 +212,8 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
             checkClosed();
             ConnectionInfo connInfo = ConnectionInfo.create(url);
             QueryServices services = getQueryServices();
-            ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps());
+            // Also performs the Kerberos login if the URL/properties request this
+            ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps(), info);
             ConnectionQueryServices connectionQueryServices = connectionQueryServicesMap.get(normalizedConnInfo);
             if (connectionQueryServices == null) {
                 if (normalizedConnInfo.isConnectionless()) {
@@ -317,4 +318,9 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
             closeLock.writeLock().unlock();
         }
     }
+
+    @VisibleForTesting
+    protected ConcurrentMap<ConnectionInfo,ConnectionQueryServices> getCachedConnections() {
+        return this.connectionQueryServicesMap;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index 375388a..272fb22 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -28,6 +28,7 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.util.Map;
 import java.util.Properties;
 import java.util.StringTokenizer;
+import java.util.Map.Entry;
 import java.util.logging.Logger;
 
 import javax.annotation.concurrent.Immutable;
@@ -37,6 +38,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -48,6 +50,7 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SQLCloseable;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -194,6 +197,8 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
      * @since 0.1.1
      */
     public static class ConnectionInfo {
+        private static final org.slf4j.Logger logger = LoggerFactory.getLogger(ConnectionInfo.class);
+        private static final Object KERBEROS_LOGIN_LOCK = new Object();
         private static SQLException getMalFormedUrlException(String url) {
             return new SQLExceptionInfo.Builder(SQLExceptionCode.MALFORMED_CONNECTION_URL)
             .setMessage(url).build().buildException();
@@ -283,7 +288,7 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             return new ConnectionInfo(quorum,port,rootNode, principal, keytabFile);
         }
         
-        public ConnectionInfo normalize(ReadOnlyProps props) throws SQLException {
+        public ConnectionInfo normalize(ReadOnlyProps props, Properties info) throws SQLException {
             String zookeeperQuorum = this.getZookeeperQuorum();
             Integer port = this.getPort();
             String rootNode = this.getRootNode();
@@ -333,8 +338,77 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             		 keytab = props.get(QueryServices.HBASE_CLIENT_KEYTAB);
             	 }
             }
+            if (!isConnectionless()) {
+                boolean credsProvidedInUrl = null != principal && null != keytab;
+                boolean credsProvidedInProps = info.containsKey(QueryServices.HBASE_CLIENT_PRINCIPAL) && info.containsKey(QueryServices.HBASE_CLIENT_KEYTAB);
+                if (credsProvidedInUrl || credsProvidedInProps) {
+                    // PHOENIX-3189 Because ConnectionInfo is immutable, we must make sure all parts of it are correct before
+                    // construction; this also requires the Kerberos user credentials object (since they are compared by reference
+                    // and not by value. If the user provided a principal and keytab via the JDBC url, we must make sure that the
+                    // Kerberos login happens *before* we construct the ConnectionInfo object. Otherwise, the use of ConnectionInfo
+                    // to determine when ConnectionQueryServices impl's should be reused will be broken.
+                    try {
+                        // Check if we need to authenticate with kerberos so that we cache the correct ConnectionInfo
+                        UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+                        if (!currentUser.hasKerberosCredentials() || !currentUser.getUserName().equals(principal)) {
+                            synchronized (KERBEROS_LOGIN_LOCK) {
+                                // Double check the current user, might have changed since we checked last. Don't want
+                                // to re-login if it's the same user.
+                                currentUser = UserGroupInformation.getCurrentUser();
+                                if (!currentUser.hasKerberosCredentials() || !currentUser.getUserName().equals(principal)) {
+                                    final Configuration config = getConfiguration(props, info, principal, keytab);
+                                    logger.info("Trying to connect to a secure cluster as {} with keytab {}", config.get(QueryServices.HBASE_CLIENT_PRINCIPAL),
+                                            config.get(QueryServices.HBASE_CLIENT_KEYTAB));
+                                    UserGroupInformation.setConfiguration(config);
+                                    User.login(config, QueryServices.HBASE_CLIENT_KEYTAB, QueryServices.HBASE_CLIENT_PRINCIPAL, null);
+                                    logger.info("Successful login to secure cluster");
+                                }
+                            }
+                        } else {
+                            // The user already has Kerberos creds, so there isn't anything to change in the ConnectionInfo.
+                            logger.debug("Already logged in as {}", currentUser);
+                        }
+                    } catch (IOException e) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
+                            .setRootCause(e).build().buildException();
+                    }
+                } else {
+                    logger.debug("Principal and keytab not provided, not attempting Kerberos login");
+                }
+            } // else, no connection, no need to login
+            // Will use the current User from UGI
             return new ConnectionInfo(zookeeperQuorum, port, rootNode, principal, keytab);
         }
+
+        /**
+         * Constructs a Configuration object to use when performing a Kerberos login.
+         * @param props QueryServices properties
+         * @param info User-provided properties
+         * @param principal Kerberos user principal
+         * @param keytab Path to Kerberos user keytab
+         * @return Configuration object suitable for Kerberos login
+         */
+        private Configuration getConfiguration(ReadOnlyProps props, Properties info, String principal, String keytab) {
+            final Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+            // Add QueryServices properties
+            for (Entry<String,String> entry : props) {
+                config.set(entry.getKey(), entry.getValue());
+            }
+            // Add any user-provided properties (via DriverManager)
+            if (info != null) {
+                for (Object key : info.keySet()) {
+                    config.set((String) key, info.getProperty((String) key));
+                }
+            }
+            // Set the principal and keytab if provided from the URL (overriding those provided in Properties)
+            if (null != principal) {
+                config.set(QueryServices.HBASE_CLIENT_PRINCIPAL, principal);
+            }
+            if (null != keytab) {
+                config.set(QueryServices.HBASE_CLIENT_KEYTAB, keytab);
+            }
+            return config;
+        }
         
         private final Integer port;
         private final String rootNode;
@@ -365,6 +439,15 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
         	this(zookeeperQuorum, port, rootNode, null, null);
         }
 
+        /**
+         * Copy constructor for all members except {@link #user}.
+         *
+         * @param other The instance to copy
+         */
+        public ConnectionInfo(ConnectionInfo other) {
+            this(other.zookeeperQuorum, other.port, other.rootNode, other.principal, other.keytab);
+        }
+
         public ReadOnlyProps asProps() {
             Map<String, String> connectionProps = Maps.newHashMapWithExpectedSize(3);
             if (getZookeeperQuorum() != null) {
@@ -408,6 +491,10 @@ public abstract class PhoenixEmbeddedDriver implements Driver, SQLCloseable {
             return principal;
         }
 
+        public User getUser() {
+            return user;
+        }
+
         @Override
         public int hashCode() {
             final int prime = 31;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 524067d..04c2c7b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -96,7 +96,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.phoenix.compile.MutationPlan;
 import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
@@ -374,22 +373,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     
     private void openConnection() throws SQLException {
         try {
-            // check if we need to authenticate with kerberos
-            String clientKeytab = this.getProps().get(HBASE_CLIENT_KEYTAB);
-            String clientPrincipal = this.getProps().get(HBASE_CLIENT_PRINCIPAL);
-            if (clientKeytab != null && clientPrincipal != null) {
-                logger.info("Trying to connect to a secure cluster with keytab:" + clientKeytab);
-                UserGroupInformation.setConfiguration(config);
-                User.login(config, HBASE_CLIENT_KEYTAB, HBASE_CLIENT_PRINCIPAL, null);
-                logger.info("Successfull login to secure cluster!!");
-            }
-			boolean transactionsEnabled = props.getBoolean(
-					QueryServices.TRANSACTIONS_ENABLED,
-					QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
-			// only initialize the tx service client if needed
-			if (transactionsEnabled) {
-				initTxServiceClient();
-			}
+            boolean transactionsEnabled = props.getBoolean(
+                    QueryServices.TRANSACTIONS_ENABLED,
+                    QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+            // only initialize the tx service client if needed
+            if (transactionsEnabled) {
+                initTxServiceClient();
+            }
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java b/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
index dd99d1e..4757e46 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/InstanceResolver.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.util;
 
 import org.apache.commons.collections.IteratorUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import java.util.Iterator;
 import java.util.List;
 import java.util.ServiceLoader;
@@ -85,4 +87,9 @@ public class InstanceResolver {
         }
         return defaultInstance;
     }
+
+    @VisibleForTesting
+    public static void clearSingletons() {
+        RESOLVED_SINGLETONS.clear();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
new file mode 100644
index 0000000..6a33142
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
@@ -0,0 +1,369 @@
+/*
+ * 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.jdbc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.query.ConfigurationFactory;
+import org.apache.phoenix.util.InstanceResolver;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests ConnectionQueryServices caching when Kerberos authentication is enabled. It's not
+ * trivial to directly test this, so we exploit the knowledge that the caching is driven by
+ * a ConcurrentHashMap. We can use a HashSet to determine when instances of ConnectionInfo
+ * collide and when they do not.
+ */
+public class SecureUserConnectionsTest {
+    private static final File TEMP_DIR = new File(getClassTempDir());
+    private static final File KEYTAB_DIR = new File(TEMP_DIR, "keytabs");
+    private static final File KDC_DIR = new File(TEMP_DIR, "kdc");
+    private static final List<File> USER_KEYTAB_FILES = new ArrayList<>();
+    private static final int NUM_USERS = 3;
+    private static final Properties EMPTY_PROPERTIES = new Properties();
+    private static final String BASE_URL = PhoenixRuntime.JDBC_PROTOCOL + ":localhost:2181";
+
+    private static MiniKdc KDC;
+
+    @BeforeClass
+    public static void setupKdc() throws Exception {
+        ensureIsEmptyDirectory(KDC_DIR);
+        ensureIsEmptyDirectory(KEYTAB_DIR);
+        // Create and start the KDC
+        Properties kdcConf = MiniKdc.createConf();
+        kdcConf.put(MiniKdc.DEBUG, true);
+        KDC = new MiniKdc(kdcConf, KDC_DIR);
+        KDC.start();
+
+        createUsers(NUM_USERS);
+
+        final Configuration conf = new Configuration(false);
+        conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+        conf.set(User.HBASE_SECURITY_CONF_KEY, "kerberos");
+        conf.setBoolean(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, true);
+        UserGroupInformation.setConfiguration(conf);
+
+        // Clear the cached singletons so we can inject our own.
+        InstanceResolver.clearSingletons();
+        // Make sure the ConnectionInfo doesn't try to pull a default Configuration
+        InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() {
+            @Override
+            public Configuration getConfiguration() {
+                return conf;
+            }
+            @Override
+            public Configuration getConfiguration(Configuration confToClone) {
+                Configuration copy = new Configuration(conf);
+                copy.addResource(confToClone);
+                return copy;
+            }
+        });
+    }
+
+    @AfterClass
+    public static void stopKdc() throws Exception {
+        // Remove our custom ConfigurationFactory for future tests
+        InstanceResolver.clearSingletons();
+        if (null != KDC) {
+            KDC.stop();
+            KDC = null;
+        }
+    }
+
+    private static String getClassTempDir() {
+        StringBuilder sb = new StringBuilder(32);
+        sb.append(System.getProperty("user.dir")).append(File.separator);
+        sb.append("target").append(File.separator);
+        sb.append(SecureUserConnectionsTest.class.getSimpleName());
+        return sb.toString();
+    }
+
+    private static void ensureIsEmptyDirectory(File f) throws IOException {
+        if (f.exists()) {
+            if (f.isDirectory()) {
+                FileUtils.deleteDirectory(f);
+            } else {
+                assertTrue("Failed to delete keytab directory", f.delete());
+            }
+        }
+        assertTrue("Failed to create keytab directory", f.mkdirs());
+    }
+
+    private static void createUsers(int numUsers) throws Exception {
+        assertNotNull("KDC is null, was setup method called?", KDC);
+        for (int i = 1; i <= numUsers; i++) {
+            String principal = "user" + i;
+            File keytabFile = new File(KEYTAB_DIR, principal + ".keytab");
+            KDC.createPrincipal(keytabFile, principal);
+            USER_KEYTAB_FILES.add(keytabFile);
+        }
+    }
+
+    /**
+     * Returns the principal for a user.
+     *
+     * @param offset The "number" user to return, based on one, not zero.
+     */
+    private static String getUserPrincipal(int offset) {
+        return "user" + offset + "@" + KDC.getRealm();
+    }
+
+    /**
+     * Returns the keytab file for the corresponding principal with the same {@code offset}.
+     * Requires {@link #createUsers(int)} to have been called with a value greater than {@code offset}.
+     *
+     * @param offset The "number" for the principal whose keytab should be returned. One-based, not zero-based.
+     */
+    public static File getUserKeytabFile(int offset) {
+        assertTrue("Invalid offset: " + offset, (offset - 1) >= 0 && (offset - 1) < USER_KEYTAB_FILES.size());
+        return USER_KEYTAB_FILES.get(offset - 1);
+    }
+
+    private String joinUserAuthentication(String origUrl, String principal, File keytab) {
+        StringBuilder sb = new StringBuilder(64);
+        // Knock off the trailing terminator if one exists
+        if (origUrl.charAt(origUrl.length() - 1) == PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR) {
+            sb.append(origUrl, 0, origUrl.length() - 1);
+        } else {
+            sb.append(origUrl);
+        }
+
+        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(principal);
+        sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR).append(keytab.getPath());
+        return sb.append(PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR).toString();
+    }
+
+    @Test
+    public void testMultipleInvocationsBySameUserAreEquivalent() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+
+        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        };
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testMultipleUniqueUGIInstancesAreDisjoint() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+
+        PrivilegedExceptionAction<Void> callable = new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        };
+
+        ugi.doAs(callable);
+        assertEquals(1, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // A second, but equivalent, call from the same "real" user but a different UGI instance
+        // is expected functionality (programmer error).
+        UserGroupInformation ugiCopy = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+        ugiCopy.doAs(callable);
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testAlternatingLogins() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+
+        UserGroupInformation ugi1 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ1, keytab1.getPath());
+        UserGroupInformation ugi2 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(princ2, keytab2.getPath());
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi2.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ2, keytab2);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        ugi1.doAs(new PrivilegedExceptionAction<Void>() {
+            public Void run() throws Exception {
+                String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+                connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+                return null;
+            }
+        });
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testAlternatingDestructiveLogins() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
+
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        UserGroupInformation.loginUserFromKeytab(princ2, keytab2.getPath());
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(3, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    @Test
+    public void testMultipleConnectionsAsSameUser() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+
+        UserGroupInformation.loginUserFromKeytab(princ1, keytab1.getPath());
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+    }
+
+    @Test
+    public void testMultipleConnectionsAsSameUserWithoutLogin() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        final String url = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+    }
+
+    @Test
+    public void testAlternatingConnectionsWithoutLogin() throws Exception {
+        final HashSet<ConnectionInfo> connections = new HashSet<>();
+        final String princ1 = getUserPrincipal(1);
+        final File keytab1 = getUserKeytabFile(1);
+        final String princ2 = getUserPrincipal(2);
+        final File keytab2 = getUserKeytabFile(2);
+        final String url1 = joinUserAuthentication(BASE_URL, princ1, keytab1);
+        final String url2 = joinUserAuthentication(BASE_URL, princ2, keytab2);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(1, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Because the UGI instances are unique, so are the connections
+        connections.add(ConnectionInfo.create(url2).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(2, connections.size());
+        verifyAllConnectionsAreKerberosBased(connections);
+
+        // Using the same UGI should result in two equivalent ConnectionInfo objects
+        connections.add(ConnectionInfo.create(url1).normalize(ReadOnlyProps.EMPTY_PROPS, EMPTY_PROPERTIES));
+        assertEquals(3, connections.size());
+        // Sanity check
+        verifyAllConnectionsAreKerberosBased(connections);
+    }
+
+    private void verifyAllConnectionsAreKerberosBased(Collection<ConnectionInfo> connections) {
+        for (ConnectionInfo cnxnInfo : connections) {
+            assertTrue("ConnectionInfo does not have kerberos credentials: " + cnxnInfo, cnxnInfo.getUser().getUGI().hasKerberosCredentials());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/phoenix-core/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/resources/log4j.properties b/phoenix-core/src/test/resources/log4j.properties
index 8e54793..85706b4 100644
--- a/phoenix-core/src/test/resources/log4j.properties
+++ b/phoenix-core/src/test/resources/log4j.properties
@@ -61,3 +61,5 @@ log4j.logger.org.mortbay.log=WARN
 log4j.logger.org.apache.hadoop=WARN
 log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.apache.hadoop.hbase=DEBUG
+log4j.logger.org.apache.directory=WARN
+log4j.logger.net.sf.ehcache=WARN

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2dc6be4e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 142595f..b5edb6c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -322,6 +322,12 @@
           <artifactId>maven-shade-plugin</artifactId>
           <version>2.4.3</version>
         </plugin>
+        <plugin>
+          <!-- Allows us to get the apache-ds bundle artifacts -->
+          <groupId>org.apache.felix</groupId>
+          <artifactId>maven-bundle-plugin</artifactId>
+          <version>2.5.3</version>
+        </plugin>
       </plugins>
     </pluginManagement>
 
@@ -449,6 +455,13 @@
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <!-- Allows us to get the apache-ds bundle artifacts -->
+        <groupId>org.apache.felix</groupId>
+        <artifactId>maven-bundle-plugin</artifactId>
+        <extensions>true</extensions>
+        <inherited>true</inherited>
+      </plugin>
     </plugins>
   </build>
 
@@ -659,6 +672,11 @@
         <type>test-jar</type> <!-- this does not work which is typical for maven.-->
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-minikdc</artifactId>
+        <version>${hadoop-two.version}</version>
+      </dependency>
 
       <!-- General Dependencies -->
       <dependency>


[35/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
index c896a93..f0b4b24 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
@@ -56,14 +56,14 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
 
-public class PercentileIT extends BaseHBaseManagedTimeIT {
+public class PercentileIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testPercentile() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable";
+        String query = "SELECT PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -83,9 +83,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentileDesc() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM aTable";
+        String query = "SELECT PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -105,9 +105,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentileWithGroupby() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT A_STRING, PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable GROUP BY A_STRING";
+        String query = "SELECT A_STRING, PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName + " GROUP BY A_STRING";
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -138,9 +138,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentileWithGroupbyAndOrderBy() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT A_STRING, PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) AS PC FROM aTable GROUP BY A_STRING ORDER BY PC";
+        String query = "SELECT A_STRING, PERCENTILE_CONT(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) AS PC FROM " + tableName + " GROUP BY A_STRING ORDER BY PC";
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -171,9 +171,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
 	public void testPercentileDiscAsc() throws Exception {
 		String tenantId = getOrganizationId();
-		initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-		String query = "SELECT PERCENTILE_DISC(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable";
+		String query = "SELECT PERCENTILE_DISC(0.9) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName;
 
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -192,9 +192,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
 	@Test
 	public void testPercentileDiscDesc() throws Exception {
 		String tenantId = getOrganizationId();
-		initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-		String query = "SELECT PERCENTILE_DISC(0.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM aTable";
+		String query = "SELECT PERCENTILE_DISC(0.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM " + tableName;
 
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -213,9 +213,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentileDiscWithGroupby() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT A_STRING, PERCENTILE_DISC(0.5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable GROUP BY A_STRING";
+        String query = "SELECT A_STRING, PERCENTILE_DISC(0.5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName + " GROUP BY A_STRING";
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -243,9 +243,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentileDiscWithGroupbyAndOrderBy() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT A_STRING, PERCENTILE_DISC(0.5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable GROUP BY A_STRING ORDER BY A_STRING DESC";
+        String query = "SELECT A_STRING, PERCENTILE_DISC(0.5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName + " GROUP BY A_STRING ORDER BY A_STRING DESC";
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -273,9 +273,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentRank() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK(5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable";
+        String query = "SELECT PERCENT_RANK(5) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -295,9 +295,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentRankWithNegativeNumeric() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK(-2) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM aTable";
+        String query = "SELECT PERCENT_RANK(-2) WITHIN GROUP (ORDER BY A_INTEGER ASC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -317,9 +317,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentRankDesc() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK(8.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM aTable";
+        String query = "SELECT PERCENT_RANK(8.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -339,9 +339,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentRankDescOnVARCHARColumn() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK('ba') WITHIN GROUP (ORDER BY A_STRING DESC) FROM aTable";
+        String query = "SELECT PERCENT_RANK('ba') WITHIN GROUP (ORDER BY A_STRING DESC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -361,9 +361,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testPercentRankDescOnDECIMALColumn() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK(2) WITHIN GROUP (ORDER BY x_decimal ASC) FROM aTable";
+        String query = "SELECT PERCENT_RANK(2) WITHIN GROUP (ORDER BY x_decimal ASC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -383,9 +383,9 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMultiplePercentRanksOnSelect() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
+        String tableName = initATableValues(tenantId, null, getDefaultSplits(tenantId), null, null);
 
-        String query = "SELECT PERCENT_RANK(2) WITHIN GROUP (ORDER BY x_decimal ASC), PERCENT_RANK(8.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM aTable";
+        String query = "SELECT PERCENT_RANK(2) WITHIN GROUP (ORDER BY x_decimal ASC), PERCENT_RANK(8.9) WITHIN GROUP (ORDER BY A_INTEGER DESC) FROM " + tableName;
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -407,11 +407,12 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testPercentileContOnDescPKColumn() throws Exception {
-        ensureTableCreated(getUrl(), INDEX_DATA_TABLE);
-        populateINDEX_DATA_TABLETable();
+        String indexDataTableName = generateRandomString();
+        ensureTableCreated(getUrl(), indexDataTableName, INDEX_DATA_TABLE);
+        populateINDEX_DATA_TABLETable(indexDataTableName);
 
         String query = "SELECT PERCENTILE_CONT(1) WITHIN GROUP (ORDER BY long_pk ASC) FROM " + INDEX_DATA_SCHEMA
-                + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE;
+                + QueryConstants.NAME_SEPARATOR + indexDataTableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -429,11 +430,12 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testPercentRankOnDescPKColumn() throws Exception {
-        ensureTableCreated(getUrl(), INDEX_DATA_TABLE);
-        populateINDEX_DATA_TABLETable();
+        String indexDataTableName = generateRandomString();
+        ensureTableCreated(getUrl(), indexDataTableName, INDEX_DATA_TABLE);
+        populateINDEX_DATA_TABLETable(indexDataTableName);
 
         String query = "SELECT PERCENT_RANK(2) WITHIN GROUP (ORDER BY long_pk ASC) FROM " + INDEX_DATA_SCHEMA
-                + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE;
+                + QueryConstants.NAME_SEPARATOR + indexDataTableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -451,11 +453,12 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testPercentileDiscOnDescPKColumn() throws Exception {
-        ensureTableCreated(getUrl(), INDEX_DATA_TABLE);
-        populateINDEX_DATA_TABLETable();
+        String indexDataTableName = generateRandomString();
+        ensureTableCreated(getUrl(), indexDataTableName, INDEX_DATA_TABLE);
+        populateINDEX_DATA_TABLETable(indexDataTableName);
 
         String query = "SELECT PERCENTILE_DISC(0.4) WITHIN GROUP (ORDER BY long_pk DESC) FROM " + INDEX_DATA_SCHEMA
-                + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE;
+                + QueryConstants.NAME_SEPARATOR + indexDataTableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -470,12 +473,12 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private static void populateINDEX_DATA_TABLETable() throws SQLException {
+    private static void populateINDEX_DATA_TABLETable(String indexDataTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         Date date = DateUtil.parseDate("2015-01-01 00:00:00");
         try {
-            String upsert = "UPSERT INTO " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE
+            String upsert = "UPSERT INTO " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTableName
                     + " VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(upsert);
             stmt.setString(1, "varchar1");
@@ -508,12 +511,13 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private static void initATableValues(String tenantId1, String tenantId2, byte[][] splits,
+    private static String initATableValues(String tenantId1, String tenantId2, byte[][] splits,
             Date date, Long ts) throws Exception {
+        String tableName = generateRandomString();
         if (ts == null) {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits);
         } else {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits, ts - 2);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, ts - 2);
         }
 
         Properties props = new Properties();
@@ -523,7 +527,7 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             // Insert all rows at ts
-            PreparedStatement stmt = conn.prepareStatement("upsert into " + "ATABLE("
+            PreparedStatement stmt = conn.prepareStatement("upsert into " + tableName + "("
                     + "    ORGANIZATION_ID, " + "    ENTITY_ID, " + "    A_STRING, "
                     + "    B_STRING, " + "    A_INTEGER, " + "    A_DATE, " + "    X_DECIMAL, "
                     + "    X_LONG, " + "    X_INTEGER," + "    Y_INTEGER)"
@@ -669,6 +673,7 @@ public class PercentileIT extends BaseHBaseManagedTimeIT {
             conn.commit();
         } finally {
             conn.close();
+            return tableName;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index 3ba43e9..579eb19 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -29,6 +29,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -54,7 +55,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
-public class PhoenixRuntimeIT extends BaseHBaseManagedTimeIT {
+public class PhoenixRuntimeIT extends BaseHBaseManagedTimeTableReuseIT {
     private static void assertTenantIds(Expression e, HTableInterface htable, Filter filter, String[] tenantIds) throws IOException {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         Scan scan = new Scan();
@@ -95,23 +96,28 @@ public class PhoenixRuntimeIT extends BaseHBaseManagedTimeIT {
     }
     
     private void testGetTenantIdExpression(boolean isSalted) throws Exception {
+        //Have to delete metaData tables because BaseHBaseManagedTimeTableReuseIT doesn't delete them after each test case , and tenant list will create issues between test cases
+        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
+
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(true);
-        String tableName = "FOO_" + (isSalted ? "SALTED" : "UNSALTED");
+        String tableName = generateRandomString() ;
+        String sequenceName = generateRandomString() ;
         conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2)) MULTI_TENANT=true" + (isSalted ? ",SALT_BUCKETS=3" : ""));
-        conn.createStatement().execute("CREATE SEQUENCE s1");
+        conn.createStatement().execute("CREATE SEQUENCE "  + sequenceName);
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t1','x')");
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t2','y')");
         
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "t1");
         Connection tsconn = DriverManager.getConnection(getUrl(), props);
-        tsconn.createStatement().execute("CREATE SEQUENCE s1");
+        tsconn.createStatement().execute("CREATE SEQUENCE " + sequenceName);
         Expression e1 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME);
         HTableInterface htable1 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
         assertTenantIds(e1, htable1, new FirstKeyOnlyFilter(), new String[] {"", "t1"} );
-        
-        tsconn.createStatement().execute("CREATE VIEW A.BAR(V1 VARCHAR) AS SELECT * FROM " + tableName);
+
+        String viewName = generateRandomString();
+        tsconn.createStatement().execute("CREATE VIEW " + viewName + "(V1 VARCHAR) AS SELECT * FROM " + tableName);
         Expression e2 = PhoenixRuntime.getTenantIdExpression(tsconn, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
         HTableInterface htable2 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
         assertTenantIds(e2, htable2, getUserTableAndViewsFilter(), new String[] {"", "t1"} );
@@ -119,29 +125,33 @@ public class PhoenixRuntimeIT extends BaseHBaseManagedTimeIT {
         Expression e3 = PhoenixRuntime.getTenantIdExpression(conn, tableName);
         HTableInterface htable3 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
         assertTenantIds(e3, htable3, new FirstKeyOnlyFilter(), new String[] {"t1", "t2"} );
-        
-        conn.createStatement().execute("CREATE TABLE BAS (k1 VARCHAR PRIMARY KEY)");
-        Expression e4 = PhoenixRuntime.getTenantIdExpression(conn, "BAS");
+
+        String basTableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + basTableName + " (k1 VARCHAR PRIMARY KEY)");
+        Expression e4 = PhoenixRuntime.getTenantIdExpression(conn, basTableName);
         assertNull(e4);
 
-        tsconn.createStatement().execute("CREATE INDEX I1 ON A.BAR(V1)");
-        Expression e5 = PhoenixRuntime.getTenantIdExpression(tsconn, "A.I1");
+        String indexName1 = generateRandomString();
+        tsconn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + viewName + "(V1)");
+        Expression e5 = PhoenixRuntime.getTenantIdExpression(tsconn, indexName1);
         HTableInterface htable5 = tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX + tableName));
         assertTenantIds(e5, htable5, new FirstKeyOnlyFilter(), new String[] {"t1"} );
 
-        conn.createStatement().execute("CREATE INDEX I2 ON " + tableName + "(k2)");
-        Expression e6 = PhoenixRuntime.getTenantIdExpression(conn, "I2");
-        HTableInterface htable6 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("I2"));
+        String indexName2 = generateRandomString();
+        conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON " + tableName + "(k2)");
+        Expression e6 = PhoenixRuntime.getTenantIdExpression(conn, indexName2);
+        HTableInterface htable6 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName2));
         assertTenantIds(e6, htable6, new FirstKeyOnlyFilter(), new String[] {"t1", "t2"} );
         
-        tableName = "BAR_" + (isSalted ? "SALTED" : "UNSALTED");
+        tableName = generateRandomString() + "BAR_" + (isSalted ? "SALTED" : "UNSALTED");
         conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2)) " + (isSalted ? "SALT_BUCKETS=3" : ""));
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t1','x')");
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t2','y')");
         Expression e7 = PhoenixRuntime.getFirstPKColumnExpression(conn, tableName);
         HTableInterface htable7 = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName));
         assertTenantIds(e7, htable7, new FirstKeyOnlyFilter(), new String[] {"t1", "t2"} );
-        
+
+
     }
     
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
index a68ba51..07dd101 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
@@ -89,7 +89,7 @@ public class ProductMetricsIT extends BaseClientManagedTimeIT {
     }
     
     private static void initTable(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(),PRODUCT_METRICS_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(), PRODUCT_METRICS_NAME, PRODUCT_METRICS_NAME,splits, ts-2);
     }
 
     private static void assertNoRows(Connection conn) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index 12066e6..fbcc120 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -99,9 +99,9 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testTableMetadataScan() throws SQLException {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), ATABLE_NAME, null, ts);
-        ensureTableCreated(getUrl(), STABLE_NAME, null, ts);
-        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, null, ts);
+        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, ts);
+        ensureTableCreated(getUrl(), STABLE_NAME, STABLE_NAME, ts);
+        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -274,8 +274,8 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testSchemaMetadataScan() throws SQLException {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, null, ts);
-        ensureTableCreated(getUrl(), PTSDB_NAME, null, ts);
+        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts);
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -303,7 +303,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testColumnMetadataScan() throws SQLException {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, ts);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -477,7 +477,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         assertEquals(SchemaUtil.normalizeIdentifier("col2"), rs.getString("COLUMN_NAME"));
         assertFalse(rs.next());
 
-        ensureTableCreated(getUrl(), TABLE_WITH_SALTING, null, ts);
+        ensureTableCreated(getUrl(), TABLE_WITH_SALTING, TABLE_WITH_SALTING, ts);
         rs = dbmd.getColumns("", "", TABLE_WITH_SALTING, StringUtil.escapeLike("A_INTEGER"));
         assertTrue(rs.next());
         assertEquals(1, rs.getInt("ORDINAL_POSITION"));
@@ -488,8 +488,8 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testPrimaryKeyMetadataScan() throws SQLException {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, null, ts);
-        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, ts);
+        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -591,10 +591,10 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testMultiTableColumnsMetadataScan() throws SQLException {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, null, ts);
-        ensureTableCreated(getUrl(), GROUPBYTEST_NAME, null, ts);
-        ensureTableCreated(getUrl(), PTSDB_NAME, null, ts);
-        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, ts);
+        ensureTableCreated(getUrl(), GROUPBYTEST_NAME, GROUPBYTEST_NAME, ts);
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts);
+        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -652,10 +652,10 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testCreateDropTable() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         
-        ensureTableCreated(getUrl(), BTABLE_NAME, null, ts-2);
-        ensureTableCreated(getUrl(), PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(), BTABLE_NAME, BTABLE_NAME, ts-2);
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-2);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -720,7 +720,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         PhoenixConnection conn1 = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
-        ensureTableCreated(getUrl(), tableName, null, ts);
+        ensureTableCreated(getUrl(), tableName, tableName, ts);
         
         descriptor = admin.getTableDescriptor(htableName);
         assertEquals(3,descriptor.getColumnFamilies().length);
@@ -975,7 +975,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToExistingFamily() throws Throwable {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1006,8 +1006,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToNewFamily() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
-        
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
@@ -1033,7 +1032,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1066,7 +1065,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropKVColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1106,7 +1105,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1124,7 +1123,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropAllKVCols() throws Exception {
         ResultSet rs;
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, null, ts);
         
         Properties props = new Properties();
         
@@ -1170,7 +1169,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testNewerTableDisallowed() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), ATABLE_NAME, null, ts);
+        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, null, ts);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
index 534f77d..6819e57 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryExecWithoutSCNIT.java
@@ -32,13 +32,13 @@ import java.util.Properties;
 import org.junit.Test;
 
 
-public class QueryExecWithoutSCNIT extends BaseHBaseManagedTimeIT {
+public class QueryExecWithoutSCNIT extends BaseHBaseManagedTimeTableReuseIT {
     @Test
     public void testScanNoSCN() throws Exception {
         long ts = System.currentTimeMillis();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
-        String query = "SELECT a_string, b_string FROM aTable WHERE organization_id=? and a_integer = 5";
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        String query = "SELECT a_string, b_string FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
         Properties props = new Properties(); // Test with no CurrentSCN property set
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/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 8711f67..fae7a7c 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
@@ -61,7 +61,7 @@ public class QueryWithLimitIT extends BaseOwnClusterHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
-            ensureTableCreated(getUrl(),KEYONLY_NAME);
+            ensureTableCreated(getUrl(), KEYONLY_NAME, KEYONLY_NAME);
             initTableValues(conn, 100);
             
             String query = "SELECT i1 FROM KEYONLY LIMIT 1";
@@ -85,7 +85,7 @@ public class QueryWithLimitIT extends BaseOwnClusterHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
 
-        ensureTableCreated(getUrl(),KEYONLY_NAME);
+        ensureTableCreated(getUrl(), KEYONLY_NAME, KEYONLY_NAME);
         initTableValues(conn, 100);
         conn.createStatement().execute("UPDATE STATISTICS " + KEYONLY_NAME);
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
index caebcb5..1573533 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
@@ -42,7 +42,7 @@ public class ReadIsolationLevelIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(long ts, byte[][] splits) throws Exception {
         String tenantId = getOrganizationId();
-        ensureTableCreated(getUrl(),ATABLE_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(),ATABLE_NAME, ATABLE_NAME, splits, ts-2);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
index dcc20ff..7670481 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
@@ -33,13 +33,15 @@ import org.junit.Before;
 import org.junit.Test;
 
 
-public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeIT {
+public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private int id;
+    protected  String tableName;
 
     @Before
     public void doBeforeTestSetup() throws Exception {
-        ensureTableCreated(getUrl(), GROUPBYTEST_NAME);
+        this.tableName = generateRandomString();
+        ensureTableCreated(getUrl(), this.tableName, GROUPBYTEST_NAME);
         Connection conn = DriverManager.getConnection(getUrl());
         insertRow(conn, "Report11", 10);
         insertRow(conn, "Report11", 10);
@@ -50,7 +52,8 @@ public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeIT {
     }
 
     private void insertRow(Connection conn, String uri, int appcpu) throws SQLException {
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + GROUPBYTEST_NAME + "(id, uri, appcpu) values (?,?,?)");
+
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + this.tableName + "(id, uri, appcpu) values (?,?,?)");
         statement.setString(1, "id" + id);
         statement.setString(2, uri);
         statement.setInt(3, appcpu);
@@ -62,14 +65,14 @@ public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeIT {
     public void testGroupByScanWithRegexpReplace() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+', '*') suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi");
+        ResultSet rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+', '*') suburi, sum(appcpu) sumcpu from " + this.tableName + " group by suburi");
         assertTrue(rs.next());
         assertEquals(rs.getString("suburi"), "Report*");
         assertEquals(rs.getInt("sumcpu"), 80);
         assertFalse(rs.next());
 
         stmt = conn.createStatement();
-        rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+') suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi");
+        rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+') suburi, sum(appcpu) sumcpu from " + this.tableName + " group by suburi");
         assertTrue(rs.next());
         assertEquals(rs.getString("suburi"), "Report");
         assertEquals(rs.getInt("sumcpu"), 80);
@@ -81,14 +84,14 @@ public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testFilterWithRegexReplace() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        ResultSet rs = conn.createStatement().executeQuery("select id from " + GROUPBYTEST_NAME + " where REGEXP_REPLACE(uri, '[2-3]+', '*') = 'Report*'");
+        ResultSet rs = conn.createStatement().executeQuery("select id from " + this.tableName + " where REGEXP_REPLACE(uri, '[2-3]+', '*') = 'Report*'");
         assertTrue(rs.next());
         assertEquals("id2", rs.getString(1));
         assertTrue(rs.next());
         assertEquals("id3", rs.getString(1));
         assertFalse(rs.next());
 
-        rs = conn.createStatement().executeQuery("select id from " + GROUPBYTEST_NAME + " where REGEXP_REPLACE(uri, '[2-3]+') = 'Report'");
+        rs = conn.createStatement().executeQuery("select id from " + this.tableName + " where REGEXP_REPLACE(uri, '[2-3]+') = 'Report'");
         assertTrue(rs.next());
         assertEquals("id2", rs.getString(1));
         assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
index 938fd5d..6669017 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
@@ -33,13 +33,15 @@ import org.junit.Before;
 import org.junit.Test;
 
 
-public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
+public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private int id;
+    String tableName;
 
     @Before
     public void doBeforeTestSetup() throws Exception {
-        ensureTableCreated(getUrl(), GROUPBYTEST_NAME);
+        tableName = generateRandomString();
+        ensureTableCreated(getUrl(), tableName, GROUPBYTEST_NAME);
         Connection conn = DriverManager.getConnection(getUrl());
         insertRow(conn, "Report1?1", 10);
         insertRow(conn, "Report1?2", 10);
@@ -50,7 +52,7 @@ public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
     }
 
     private void insertRow(Connection conn, String uri, int appcpu) throws SQLException {
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + GROUPBYTEST_NAME + "(id, uri, appcpu) values (?,?,?)");
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(id, uri, appcpu) values (?,?,?)");
         statement.setString(1, "id" + id);
         statement.setString(2, uri);
         statement.setInt(3, appcpu);
@@ -59,7 +61,7 @@ public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
     }
 
     private void testGroupByScanWithRegexpSubstr(Connection conn, Integer offset, String exceptedSubstr) throws Exception {
-        String cmd = "select REGEXP_SUBSTR(uri, '[^\\\\?]+'" + ((offset == null) ? "" : ", " + offset.intValue()) +") suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi";
+        String cmd = "select REGEXP_SUBSTR(uri, '[^\\\\?]+'" + ((offset == null) ? "" : ", " + offset.intValue()) +") suburi, sum(appcpu) sumcpu from " + tableName + " group by suburi";
         Statement stmt = conn.createStatement();
         ResultSet rs = stmt.executeQuery(cmd);
         assertTrue(rs.next());
@@ -87,7 +89,7 @@ public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
     }
 
     private void testFilterWithRegexSubstr(Connection conn, Integer offset, String exceptedSubstr) throws Exception {
-        String cmd = "select id from " + GROUPBYTEST_NAME + " where REGEXP_SUBSTR(uri, '[^\\\\?]+'"+ ((offset == null) ? "" : ", " + offset.intValue()) +") = '" + exceptedSubstr + "1'";
+        String cmd = "select id from " + tableName + " where REGEXP_SUBSTR(uri, '[^\\\\?]+'"+ ((offset == null) ? "" : ", " + offset.intValue()) +") = '" + exceptedSubstr + "1'";
         ResultSet rs = conn.createStatement().executeQuery(cmd);
         assertTrue(rs.next());
         assertEquals("id0", rs.getString(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
index 2722be1..ff29840 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
@@ -52,27 +52,7 @@ import org.junit.Test;
 import com.google.common.collect.Maps;
 
 
-public class ReverseScanIT extends BaseHBaseManagedTimeIT {
-    protected static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
-
-    @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-        // Ensures our split points will be used
-        // TODO: do deletePriorTables before test?
-        Connection conn = DriverManager.getConnection(getUrl());
-        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
-        try {
-            admin.disableTable(TestUtil.ATABLE_NAME);
-            admin.deleteTable(TestUtil.ATABLE_NAME);
-        } catch (TableNotFoundException e) {
-        } finally {
-            admin.close();
-            conn.close();
-        }
-     }
+public class ReverseScanIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static byte[][] getSplitsAtRowKeys(String tenantId) {
         return new byte[][] { 
@@ -85,10 +65,10 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testReverseRangeScan() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String query = "SELECT entity_id FROM aTable WHERE entity_id >= '" + ROW3 + "' ORDER BY organization_id DESC, entity_id DESC";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE entity_id >= '" + ROW3 + "' ORDER BY organization_id DESC, entity_id DESC";
         try {
             Statement stmt = conn.createStatement();
             stmt.setFetchSize(2);
@@ -113,11 +93,11 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             assertEquals(
-                    "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER ATABLE\n" + 
+                    "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" +
                     "    SERVER FILTER BY FIRST KEY ONLY AND ENTITY_ID >= '00A323122312312'",
                     QueryUtil.getExplainPlan(rs));
             
-            PreparedStatement statement = conn.prepareStatement("SELECT entity_id FROM aTable WHERE organization_id = ? AND entity_id >= ? ORDER BY organization_id DESC, entity_id DESC");
+            PreparedStatement statement = conn.prepareStatement("SELECT entity_id FROM " + tableName + " WHERE organization_id = ? AND entity_id >= ? ORDER BY organization_id DESC, entity_id DESC");
             statement.setString(1, tenantId);
             statement.setString(2, ROW7);
             rs = statement.executeQuery();
@@ -138,10 +118,10 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testReverseSkipScan() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String query = "SELECT entity_id FROM aTable WHERE organization_id = ? AND entity_id IN (?,?,?,?,?) ORDER BY organization_id DESC, entity_id DESC";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id = ? AND entity_id IN (?,?,?,?,?) ORDER BY organization_id DESC, entity_id DESC";
         try {
             PreparedStatement statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
@@ -173,15 +153,16 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateRandomString();
         conn.createStatement()
-                .execute("CREATE TABLE T" + " ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) ");
-        conn.createStatement().execute("upsert into T values ('a',1,3)");
-        conn.createStatement().execute("upsert into T values ('b',1,3)");
-        conn.createStatement().execute("upsert into T values ('c',1,3)");
-        conn.createStatement().execute("upsert into T values ('d',1,3)");
-        conn.createStatement().execute("upsert into T values ('e',1,3)");
+                .execute("CREATE TABLE " + tableName + " ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) ");
+        conn.createStatement().execute("upsert into " + tableName + " values ('a',1,3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('b',1,3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('c',1,3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('d',1,3)");
+        conn.createStatement().execute("upsert into " + tableName + " values ('e',1,3)");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT k FROM T where k>'b' and k<'d' order by k desc");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName + " where k>'b' and k<'d' order by k desc");
         assertTrue(rs.next());
         assertEquals("c", rs.getString(1));
         assertTrue(!rs.next());
@@ -190,17 +171,18 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testReverseScanIndex() throws Exception {
+        String indexName = generateRandomString();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getSplitsAtRowKeys(tenantId), getUrl());
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = "CREATE INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer DESC) INCLUDE ("
+        String ddl = "CREATE INDEX " + indexName + " ON " + tableName + " (a_integer DESC) INCLUDE ("
         + "    A_STRING, " + "    B_STRING, " + "    A_DATE)";
         conn.createStatement().execute(ddl);
         
         String query = 
-                "SELECT a_integer FROM atable where a_integer is not null order by a_integer nulls last limit 1";
+                "SELECT a_integer FROM " + tableName + " where a_integer is not null order by a_integer nulls last limit 1";
 
         PreparedStatement statement = conn.prepareStatement(query);
         ResultSet rs=statement.executeQuery();
@@ -210,7 +192,7 @@ public class ReverseScanIT extends BaseHBaseManagedTimeIT {
         
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         assertEquals(
-                "CLIENT SERIAL 1-WAY REVERSE RANGE SCAN OVER ATABLE_IDX [not null]\n" + 
+                "CLIENT SERIAL 1-WAY REVERSE RANGE SCAN OVER " + indexName + " [not null]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER 1 ROW LIMIT\n" + 
                 "CLIENT 1 ROW LIMIT",QueryUtil.getExplainPlan(rs));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index ce38fcf..f35484d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -38,6 +38,7 @@ import static org.apache.phoenix.util.TestUtil.ROW7;
 import static org.apache.phoenix.util.TestUtil.ROW8;
 import static org.apache.phoenix.util.TestUtil.ROW9;
 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;
@@ -65,7 +66,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithEqualsExpression() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) = (7, 5)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -90,7 +91,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithGreaterThanExpression() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= (4, 4)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -116,7 +117,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithUnEqualNumberArgs() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer, y_integer) >= (7, 5)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -142,7 +143,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testBindVarsInRowValueConstructor() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) = (?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -169,7 +170,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSAndLiteralExpressionOnRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= 7";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -193,7 +194,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSLiteralExpressionOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND 7 <= (a_integer, x_integer)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -217,7 +218,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSBuiltInFunctionOperatingOnIntegerLiteralRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= to_number('7')";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -241,7 +242,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSWithBuiltInFunctionOperatingOnIntegerLiteralOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND to_number('7') <= (a_integer, x_integer)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -265,7 +266,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSWithBuiltInFunctionOperatingOnColumnRefOnRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts - 1);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl());
         String upsertQuery = "UPSERT INTO aTable(organization_id, entity_id, a_string) values (?, ?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -331,7 +332,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSWithBuiltInFunctionOperatingOnColumnRefOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts - 1);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl());
         String upsertQuery = "UPSERT INTO aTable(organization_id, entity_id, a_string) values (?, ?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -653,7 +654,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithNonLeadingPkColsOfTypesIntegerAndString() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, a_string FROM aTable WHERE ?=organization_id  AND (a_integer, a_string) <= (5, 'a')";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -679,7 +680,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithNonLeadingPkColsOfTypesTimeStampAndString() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String updateStmt = 
             "upsert into " +
             "ATABLE(" +
@@ -723,7 +724,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testNestedRVCBasic() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         //all the three queries should return the same rows.
         String[] queries = {"SELECT organization_id, entity_id, a_string FROM aTable WHERE ((organization_id, entity_id), a_string) >= ((?, ?), ?)",
                             "SELECT organization_id, entity_id, a_string FROM aTable WHERE (organization_id, entity_id, a_string) >= (?, ?, ?)",
@@ -761,7 +762,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithInListClausePossibleNullValues() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         //we have a row present in aTable where x_integer = 5 and y_integer = NULL which gets translated to 0 when retriving from HBase. 
         String query = "SELECT x_integer, y_integer FROM aTable WHERE ? = organization_id AND (x_integer, y_integer) IN ((5))";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -783,7 +784,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithInListClauseUsingSubsetOfPKColsInOrder() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         // Though we have a row present in aTable where organization_id = tenantId and  x_integer = 5,
         // we'd also need to have an entity_id that is null (which we don't have).
         String query = "SELECT organization_id, entity_id FROM aTable WHERE (organization_id, entity_id) IN (('" + tenantId + "')) AND x_integer = 5";
@@ -813,7 +814,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithCeilAndFloorNeededForDecimal() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) < (8.6, 4.5) AND (a_integer, x_integer) > (6.8, 4)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -840,7 +841,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
         String tenantId = getOrganizationId();
         Date dateUpserted = DateUtil.parseDate("2012-01-01 14:25:28");
         dateUpserted = new Date(dateUpserted.getTime() + 660); // this makes the dateUpserted equivalent to 2012-01-01 14:25:28.660
-        initATableValues(tenantId, getDefaultSplits(tenantId), dateUpserted, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), dateUpserted, ts, getUrl());
         String query = "SELECT a_integer, a_date FROM aTable WHERE ?=organization_id  AND (a_integer, a_date) <= (9, ?) AND (a_integer, a_date) >= (6, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
index 3a847b6..39ba19e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
@@ -43,7 +43,7 @@ import org.junit.Test;
 import com.google.common.collect.Maps;
 
 
-public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
+public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final int BATCH_SIZE = 25;
     private static final int MAX_FILESIZE = 1024 * 10;
@@ -56,13 +56,11 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         }
         PAYLOAD = buf.toString();
     }
-    private static final String TABLE_NAME = "S";
-    private static final byte[] TABLE_NAME_BYTES = Bytes.toBytes(TABLE_NAME);
     private static final int MIN_CHAR = 'a';
     private static final int MAX_CHAR = 'z';
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         // needed for 64 region parallelization due to splitting
@@ -75,13 +73,13 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
-    private static void initTable() throws Exception {
+    private static void initTable(String tableName) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE " + TABLE_NAME + "("
+        conn.createStatement().execute("CREATE TABLE " + tableName + "("
                 + "a VARCHAR PRIMARY KEY, b VARCHAR) " 
                 + HTableDescriptor.MAX_FILESIZE + "=" + MAX_FILESIZE + ","
                 + " SALT_BUCKETS = 4");
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO s VALUES(?,?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?)");
         int rowCount = 0;
         for (int c1 = MIN_CHAR; c1 <= MAX_CHAR; c1++) {
             for (int c2 = MIN_CHAR; c2 <= MAX_CHAR; c2++) {
@@ -99,7 +97,7 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         HBaseAdmin admin = services.getAdmin();
         try {
-            admin.flush(TABLE_NAME);
+            admin.flush(tableName);
         } finally {
             admin.close();
         }
@@ -108,18 +106,20 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testManualSplit() throws Exception {
-        initTable();
+        String tableName = generateRandomString();
+        byte[] tableNameBytes = Bytes.toBytes(tableName);
+        initTable(tableName);
         Connection conn = DriverManager.getConnection(getUrl());
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-        int nRegions = services.getAllTableRegions(TABLE_NAME_BYTES).size();
+        int nRegions = services.getAllTableRegions(tableNameBytes).size();
         int nInitialRegions = nRegions;
         HBaseAdmin admin = services.getAdmin();
         try {
-            admin.split(TABLE_NAME);
+            admin.split(tableName);
             int nTries = 0;
             while (nRegions == nInitialRegions && nTries < 10) {
                 Thread.sleep(1000);
-                nRegions = services.getAllTableRegions(TABLE_NAME_BYTES).size();
+                nRegions = services.getAllTableRegions(tableNameBytes).size();
                 nTries++;
             }
             // Split finished by this time, but cache isn't updated until
@@ -127,10 +127,10 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
             assertEquals(nRegions, nInitialRegions);
             
             int nRows = 2;
-            String query = "SELECT count(*) FROM S WHERE a IN ('tl','jt',' a',' b',' c',' d')";
+            String query = "SELECT count(*) FROM " + tableName + " WHERE a IN ('tl','jt',' a',' b',' c',' d')";
             ResultSet rs1 = conn.createStatement().executeQuery(query);
             assertTrue(rs1.next());
-            nRegions = services.getAllTableRegions(TABLE_NAME_BYTES).size();
+            nRegions = services.getAllTableRegions(tableNameBytes).size();
             // Region cache has been updated, as there are more regions now
             assertNotEquals(nRegions, nInitialRegions);
             /*
@@ -281,8 +281,8 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSkipScanInListOfRVCAfterManualSplit() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE FIELD_HISTORY_ARCHIVE ( "
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " ( "
             + "organization_id CHAR(15) NOT NULL, "
             + "parent_id CHAR(15) NOT NULL, "
             + "created_date DATE NOT NULL, "
@@ -307,7 +307,7 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         ddlStmt.execute();
         conn.commit();
         
-        final String upsertPrefix = "UPSERT INTO FIELD_HISTORY_ARCHIVE VALUES ( '00Dxx0000001gER', ";
+        final String upsertPrefix = "UPSERT INTO " + tableName + " VALUES ( '00Dxx0000001gER', ";
         conn.createStatement().executeUpdate(upsertPrefix + "'001xx000003DGr4', TO_DATE('2014-07-11 20:53:01'), '017xx0000022MmH', '005xx000001Sv21' )");
         conn.createStatement().executeUpdate(upsertPrefix + "'001xx000003DGr5', TO_DATE('2014-07-11 20:53:01'), '017xx0000022Mln', '005xx000001Sv21' )");
         conn.createStatement().executeUpdate(upsertPrefix + "'001xx000003DGsy', TO_DATE('2014-07-11 20:53:01'), '017xx0000022MsO', '005xx000001Sv21' )");
@@ -318,8 +318,8 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         
         String sql = "SELECT "
             + "CREATED_BY_ID, PARENT_ID "
-            + "FROM FIELD_HISTORY_ARCHIVE "
-            + "WHERE ORGANIZATION_ID='00Dxx0000001gER' "
+            + "FROM " + tableName
+            + " WHERE ORGANIZATION_ID='00Dxx0000001gER' "
             + "AND (PARENT_ID,CREATED_DATE,ENTITY_HISTORY_ID)  IN  ("
             + "('001xx000003DGr4',TO_DATE('2014-07-11 20:53:01'),'017xx0000022MmH'),"
             + "('001xx000003DGr5',TO_DATE('2014-07-11 20:53:01'),'017xx0000022Mln'),"
@@ -350,8 +350,8 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMinMaxRangeIntersection() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        
-        PreparedStatement stmt = conn.prepareStatement("create table splits_test "
+        String tableName = generateRandomString();
+        PreparedStatement stmt = conn.prepareStatement("create table " + tableName
             + "(pk1 UNSIGNED_TINYINT NOT NULL, pk2 UNSIGNED_TINYINT NOT NULL, kv VARCHAR "
             + "CONSTRAINT pk PRIMARY KEY (pk1, pk2)) SALT_BUCKETS=4 SPLIT ON (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)");
         // Split each salt bucket into multiple regions
@@ -370,7 +370,7 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         stmt.execute();
         
         // Use a query with a RVC in a non equality expression
-        ResultSet rs = conn.createStatement().executeQuery("select count(kv) from splits_test where pk1 <= 3 and (pk1,PK2) >= (3, 1)");
+        ResultSet rs = conn.createStatement().executeQuery("select count(kv) from " + tableName + " where pk1 <= 3 and (pk1,PK2) >= (3, 1)");
         assertTrue(rs.next());
     }
 }


[09/50] [abbrv] phoenix git commit: Set version to 4.8.0-HBase-1.2 for release

Posted by ma...@apache.org.
Set version to 4.8.0-HBase-1.2 for release


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

Branch: refs/heads/calcite
Commit: 4702080a3d09c091f45a84fe07b6c605b6a906db
Parents: 0b91b45
Author: Ankit Singhal <an...@gmail.com>
Authored: Wed Aug 3 08:58:13 2016 +0000
Committer: Ankit Singhal <an...@gmail.com>
Committed: Wed Aug 3 08:58:13 2016 +0000

----------------------------------------------------------------------
 phoenix-assembly/pom.xml           | 2 +-
 phoenix-client/pom.xml             | 2 +-
 phoenix-core/pom.xml               | 2 +-
 phoenix-flume/pom.xml              | 2 +-
 phoenix-hive/pom.xml               | 2 +-
 phoenix-pherf/pom.xml              | 2 +-
 phoenix-pig/pom.xml                | 2 +-
 phoenix-queryserver-client/pom.xml | 2 +-
 phoenix-queryserver/pom.xml        | 2 +-
 phoenix-server/pom.xml             | 2 +-
 phoenix-spark/pom.xml              | 2 +-
 phoenix-tracing-webapp/pom.xml     | 2 +-
 pom.xml                            | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 07a898a..05ec173 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index e6ce49b..cda26c7 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 039d4f2..17a96fd 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index f6ff004..b6e5ff9 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-hive/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index 2986bea..21ccab7 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 5376b8f..d013003 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.8.0-HBase-1.2-SNAPSHOT</version>
+		<version>4.8.0-HBase-1.2</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 74f0261..709f2f8 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index ecc9edd..a310961 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index 1c186a1..25bf6bf 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 3d94406..af756a4 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 2452e93..3177b09 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+    <version>4.8.0-HBase-1.2</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 00d8b81..9747c3e 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+      <version>4.8.0-HBase-1.2</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4702080a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index da70eeb..79cb870 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.8.0-HBase-1.2-SNAPSHOT</version>
+  <version>4.8.0-HBase-1.2</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>


[45/50] [abbrv] phoenix git commit: PHOENIX-3223 Add `hadoop classpath` to PQS classpath when available

Posted by ma...@apache.org.
PHOENIX-3223 Add `hadoop classpath` to PQS classpath when available

Closes apache/phoenix#205


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

Branch: refs/heads/calcite
Commit: 93a9c9187f63578b9da9864e06c0df3231c9f61c
Parents: ede483c
Author: Josh Elser <el...@apache.org>
Authored: Tue Aug 30 15:58:50 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 30 18:49:08 2016 -0400

----------------------------------------------------------------------
 bin/queryserver.py | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/93a9c918/bin/queryserver.py
----------------------------------------------------------------------
diff --git a/bin/queryserver.py b/bin/queryserver.py
index 1048adb..1ad8b86 100755
--- a/bin/queryserver.py
+++ b/bin/queryserver.py
@@ -73,6 +73,7 @@ else:
 # HBase/Phoenix client side property override
 hbase_config_path = phoenix_utils.hbase_conf_dir
 hadoop_config_path = phoenix_utils.hadoop_conf
+hadoop_classpath = phoenix_utils.hadoop_classpath
 
 # TODO: add windows support
 phoenix_file_basename = '%s-queryserver' % getpass.getuser()
@@ -119,7 +120,8 @@ out_file_path = os.path.join(log_dir, phoenix_out_file)
 
 # The command is run through subprocess so environment variables are automatically inherited
 java_cmd = '%(java)s -cp ' + hbase_config_path + os.pathsep + hadoop_config_path + os.pathsep + \
-    phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.phoenix_queryserver_jar  + \
+    phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.phoenix_queryserver_jar + \
+    os.pathsep + hadoop_classpath + \
     " -Dproc_phoenixserver" + \
     " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " -Dpsql.root.logger=%(root_logger)s" + \


[17/50] [abbrv] phoenix git commit: PHOENIX-2944 DATE Comparison Broken(Saurabh Seth)

Posted by ma...@apache.org.
PHOENIX-2944 DATE Comparison Broken(Saurabh Seth)


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

Branch: refs/heads/calcite
Commit: 657917bfb15ecd5bef4616458f2c8cd3ede0cf6b
Parents: 44cf082
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Aug 16 10:29:18 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Tue Aug 16 10:29:18 2016 +0530

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DateTimeIT.java  | 44 ++++++++++++++++++++
 .../org/apache/phoenix/schema/types/PDate.java  |  2 +-
 .../apache/phoenix/schema/types/PTimestamp.java |  2 +-
 3 files changed, 46 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/657917bf/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index 461816a..7ffc54f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -732,4 +732,48 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         assertTrue(rs.next());
         assertEquals(new java.util.Date().getYear(),rs.getTimestamp(2).getYear());
     }
+    
+    @Test
+    public void testLiteralDateComparison() throws Exception {
+        ResultSet rs =
+                conn.createStatement().executeQuery(
+                    "select DATE '2016-05-10 00:00:00' > DATE '2016-05-11 00:00:00'");
+
+        assertTrue(rs.next());
+        assertEquals(false, rs.getBoolean(1));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testLiteralTimestampComparison() throws Exception {
+        ResultSet rs =
+                conn.createStatement().executeQuery(
+                    "select TIMESTAMP '2016-05-10 00:00:00' > TIMESTAMP '2016-05-11 00:00:00'");
+
+        assertTrue(rs.next());
+        assertEquals(false, rs.getBoolean(1));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testLiteralDateTimestampComparison() throws Exception {
+        ResultSet rs =
+                conn.createStatement().executeQuery(
+                    "select DATE '2016-05-10 00:00:00' > TIMESTAMP '2016-05-11 00:00:00'");
+
+        assertTrue(rs.next());
+        assertEquals(false, rs.getBoolean(1));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testLiteralDateTimestampComparison2() throws Exception {
+        ResultSet rs =
+                conn.createStatement().executeQuery(
+                    "select TIMESTAMP '2016-05-10 00:00:00' > DATE '2016-05-11 00:00:00'");
+
+        assertTrue(rs.next());
+        assertEquals(false, rs.getBoolean(1));
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/657917bf/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
index b10b1ac..c27d0fc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
@@ -129,7 +129,7 @@ public class PDate extends PDataType<Date> {
     if (rhsType == PTimestamp.INSTANCE || rhsType == PUnsignedTimestamp.INSTANCE) {
       return -rhsType.compareTo(rhs, lhs, PTime.INSTANCE);
     }
-    return ((java.util.Date) rhs).compareTo((java.util.Date) lhs);
+    return ((java.util.Date) lhs).compareTo((java.util.Date) rhs);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/657917bf/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index 1f654fe..cdfb533 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -174,7 +174,7 @@ public class PTimestamp extends PDataType<Timestamp> {
         if (equalsAny(rhsType, PTimestamp.INSTANCE, PUnsignedTimestamp.INSTANCE)) {
             return ((java.sql.Timestamp) lhs).compareTo((java.sql.Timestamp) rhs);
         }
-        int c = ((java.util.Date) rhs).compareTo((java.util.Date) lhs);
+        int c = ((java.util.Date) lhs).compareTo((java.util.Date) rhs);
         if (c != 0) return c;
         return ((java.sql.Timestamp) lhs).getNanos();
     }


[24/50] [abbrv] phoenix git commit: PHOENIX-930 duplicated columns cause query exception and drop table exception (Junegunn Choi, Kalyan Hadoop)

Posted by ma...@apache.org.
PHOENIX-930 duplicated columns cause query exception and drop table exception (Junegunn Choi, Kalyan Hadoop)


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

Branch: refs/heads/calcite
Commit: b36bb31fe42cb7bf5b6de5bfab63bcad424998a4
Parents: 5fecde8
Author: James Taylor <ja...@apache.org>
Authored: Fri Aug 19 16:34:28 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Tue Aug 23 10:29:40 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/index/IndexMetadataIT.java  | 40 ++++++++++
 .../apache/phoenix/schema/MetaDataClient.java   | 78 +++++---------------
 .../org/apache/phoenix/schema/PTableImpl.java   | 12 +--
 .../compile/CreateTableCompilerTest.java        | 47 ++++++++++++
 4 files changed, 113 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index 1af15a1..a48cc4b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -43,6 +43,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -401,6 +402,45 @@ public class IndexMetadataIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+
+    @Test
+    public void testTableWithSameColumnNames() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        try {
+            String ddl = "create table test_table (char_pk varchar not null,"
+        		+ " int_col integer, long_col integer, int_col integer"
+        		+ " constraint pk primary key (char_pk))";
+            PreparedStatement stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+            fail("Should have caught exception");
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals(SQLExceptionCode.COLUMN_EXIST_IN_DEF.getErrorCode(), e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testTableWithSameColumnNamesWithFamily() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        try {
+            String ddl = "create table test_table (char_pk varchar not null,"
+        		+ " a.int_col integer, a.long_col integer,"
+        		+ " a.int_col integer, b.long_col integer"
+        		+ " constraint pk primary key (char_pk))";
+            PreparedStatement stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+            fail("Should have caught exception");
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals(SQLExceptionCode.COLUMN_EXIST_IN_DEF.getErrorCode(), e.getErrorCode());
+        } finally {
+            conn.close();
+        }
+    }
     
     @Test
     public void testIndexDefinitionWithSameColumnNamesInTwoFamily() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 1264e32..7f97f4a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.schema;
 
-import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
 import static com.google.common.collect.Sets.newLinkedHashSet;
 import static com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize;
 import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
@@ -111,6 +110,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -120,7 +120,6 @@ import java.util.Set;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -216,7 +215,6 @@ import org.apache.tephra.TxConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
@@ -1886,7 +1884,7 @@ public class MetaDataClient {
             }
 
             List<ColumnDef> colDefs = statement.getColumnDefs();
-            List<PColumn> columns;
+            LinkedHashMap<PColumn,PColumn> columns;
             LinkedHashSet<PColumn> pkColumns;
 
             if (tenantId != null && !sharedTable) {
@@ -1905,7 +1903,7 @@ public class MetaDataClient {
             if (tableType == PTableType.VIEW) {
                 physicalNames = Collections.singletonList(PNameFactory.newName(parent.getPhysicalName().getString()));
                 if (viewType == ViewType.MAPPED) {
-                    columns = newArrayListWithExpectedSize(colDefs.size());
+                    columns = Maps.newLinkedHashMap();
                     pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size());
                 } else {
                     // Propagate property values to VIEW.
@@ -1926,8 +1924,10 @@ public class MetaDataClient {
                     if (saltBucketNum != null) { // Don't include salt column in columns, as it should not have it when created
                         allColumns = allColumns.subList(1, allColumns.size());
                     }
-                    columns = newArrayListWithExpectedSize(allColumns.size() + colDefs.size());
-                    columns.addAll(allColumns);
+                    columns = new LinkedHashMap<PColumn,PColumn>(allColumns.size() + colDefs.size());
+                    for (PColumn column : allColumns) {
+                        columns.put(column, column);
+                    }
                     pkColumns = newLinkedHashSet(parent.getPKColumns());
 
                     // Add row linking from view to its parent table
@@ -1943,7 +1943,7 @@ public class MetaDataClient {
                     linkStatement.execute();
                 }
             } else {
-                columns = newArrayListWithExpectedSize(colDefs.size());
+                columns = new LinkedHashMap<PColumn,PColumn>(colDefs.size());
                 pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size() + 1); // in case salted
             }
 
@@ -2026,11 +2026,9 @@ public class MetaDataClient {
                         throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                     }
                 }
-                if (tableType == PTableType.VIEW && hasColumnWithSameNameAndFamily(columns, column)) {
-                    // we only need to check for dup columns for views because they inherit columns from parent
+                if (columns.put(column, column) != null) {
                     throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
                 }
-                columns.add(column);
                 if ((colDef.getDataType() == PVarbinary.INSTANCE || colDef.getDataType().isArrayType())
                         && SchemaUtil.isPKColumn(column)
                         && pkColumnsIterator.hasNext()) {
@@ -2129,7 +2127,7 @@ public class MetaDataClient {
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
                         null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
-                        PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns, null, null,
+                        PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.values(), null, null,
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
@@ -2163,14 +2161,14 @@ public class MetaDataClient {
             
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
-                for (int i = 0; i < columns.size(); i++) {
-                    PColumn column = columns.get(i);
+                for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
+                    PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
                     // set the autoPartition column attributes   
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
-                        columns.set(i, column = new DelegateColumn(column) {
+                        entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
                                 // set to non-null value so that we will generate a Put that 
@@ -2186,7 +2184,7 @@ public class MetaDataClient {
                     }
                     else if (isViewColumnReferenced != null) {
                         if (viewColumnConstants != null && columnPosition < viewColumnConstants.length) {
-                            columns.set(i, column = new DelegateColumn(column) {
+                            entry.setValue(column = new DelegateColumn(column) {
                                 @Override
                                 public byte[] getViewConstant() {
                                     return viewColumnConstants[columnPosition];
@@ -2197,7 +2195,7 @@ public class MetaDataClient {
                                 }
                             });
                         } else {
-                            columns.set(i, column = new DelegateColumn(column) {
+                            entry.setValue(column = new DelegateColumn(column) {
                                 @Override
                                 public boolean isViewReferenced() {
                                     return isViewColumnReferenced.get(columnPosition);
@@ -2337,19 +2335,12 @@ public class MetaDataClient {
                 // If the parent table of the view has the auto partition sequence name attribute,
                 // set the view statement and relevant partition column attributes correctly
                 if (parent!=null && parent.getAutoPartitionSeqName()!=null) {
-                    int autoPartitionColIndex = -1;
-                    PColumn autoPartitionCol = parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent));
-                    for (int i=0; i<columns.size(); ++i) {
-                        if (autoPartitionCol.getName().equals(columns.get(i).getName())) {
-                            autoPartitionColIndex = i;
-                        }
-                    }
+                    final PColumn autoPartitionCol = parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent));
                     final Long autoPartitionNum = Long.valueOf(result.getAutoPartitionNum());
-                    final PColumn column = columns.get(autoPartitionColIndex);
-                    columns.set(autoPartitionColIndex, new DelegateColumn(column) {
+                    columns.put(autoPartitionCol, new DelegateColumn(autoPartitionCol) {
                         @Override
                         public byte[] getViewConstant() {
-                            PDataType dataType = column.getDataType();
+                            PDataType dataType = autoPartitionCol.getDataType();
                             Object val = dataType.toObject(autoPartitionNum, PLong.INSTANCE);
                             byte[] bytes = new byte [dataType.getByteSize() + 1];
                             dataType.toBytes(val, bytes, 0);
@@ -2371,7 +2362,7 @@ public class MetaDataClient {
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 PTable table =  PTableImpl.makePTable(
                         tenantId, newSchemaName, PNameFactory.newName(tableName), tableType, indexState, timestamp!=null ? timestamp : result.getMutationTime(),
-                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns,
+                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         dataTableName == null ? null : newSchemaName, dataTableName == null ? null : PNameFactory.newName(dataTableName), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
                         indexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
@@ -2384,29 +2375,6 @@ public class MetaDataClient {
         }
     }
 
-    private byte[][] getSplitKeys(List<HRegionLocation> allTableRegions) {
-        if(allTableRegions.size() == 1) return null;
-        byte[][] splitKeys = new byte[allTableRegions.size()-1][];
-        int i = 0;
-        for (HRegionLocation region : allTableRegions) {
-            if (region.getRegionInfo().getStartKey().length != 0) {
-                splitKeys[i] = region.getRegionInfo().getStartKey();
-                i++;
-            }
-        }
-        return splitKeys;
-    }
-
-    private static boolean hasColumnWithSameNameAndFamily(Collection<PColumn> columns, PColumn column) {
-        for (PColumn currColumn : columns) {
-           if (Objects.equal(currColumn.getFamilyName(), column.getFamilyName()) &&
-               Objects.equal(currColumn.getName(), column.getName())) {
-               return true;
-           }
-        }
-        return false;
-    }
-
     /**
      * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
      * <p>
@@ -2517,7 +2485,6 @@ public class MetaDataClient {
             Delete tableDelete = new Delete(key, clientTimeStamp);
             tableMetaData.add(tableDelete);
             boolean hasViewIndexTable = false;
-            boolean hasLocalIndexTable = false;
             if (parentTableName != null) {
                 byte[] linkKey = MetaDataUtil.getParentLinkKey(tenantIdStr, schemaName, parentTableName, tableName);
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
@@ -2558,11 +2525,6 @@ public class MetaDataClient {
                         // PName name, PTableType type, long timeStamp, long sequenceNumber, List<PColumn> columns
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
-                            for (PTable index : table.getIndexes()) {
-                                if (index.getIndexType() == IndexType.LOCAL) {
-                                    hasLocalIndexTable = true;
-                                } 
-                            }
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
                                                      // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
@@ -3716,7 +3678,7 @@ public class MetaDataClient {
                 || useSchemaStatement.getSchemaName().toUpperCase().equals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE)) {
             connection.setSchema(null);
         } else {
-            PSchema schema = FromCompiler.getResolverForSchema(useSchemaStatement, connection)
+            FromCompiler.getResolverForSchema(useSchemaStatement, connection)
                     .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 847979a..92c49f9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -299,7 +300,7 @@ public class PTableImpl implements PTable {
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
+            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
@@ -313,7 +314,7 @@ public class PTableImpl implements PTable {
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
-            List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
+            Collection<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
@@ -328,7 +329,7 @@ public class PTableImpl implements PTable {
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
-            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, List<PColumn> columns,
+            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, Collection<PColumn> columns,
             PName parentSchemaName, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
             List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
@@ -367,7 +368,7 @@ public class PTableImpl implements PTable {
     }
 
     private void init(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
-            PName pkName, Integer bucketNum, List<PColumn> columns, PName parentSchemaName, PName parentTableName,
+            PName pkName, Integer bucketNum, Collection<PColumn> columns, PName parentSchemaName, PName parentTableName,
             List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
@@ -423,8 +424,7 @@ public class PTableImpl implements PTable {
             allColumns = new PColumn[columns.size()];
             pkColumns = Lists.newArrayListWithExpectedSize(columns.size());
         }
-        for (int i = 0; i < columns.size(); i++) {
-            PColumn column = columns.get(i);
+        for (PColumn column : columns) {
             allColumns[column.getPosition()] = column;
             PName familyName = column.getFamilyName();
             if (familyName == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b36bb31f/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
new file mode 100644
index 0000000..ed907c3
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/CreateTableCompilerTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.compile;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.schema.ColumnAlreadyExistsException;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+public class CreateTableCompilerTest extends BaseConnectionlessQueryTest {
+    @Test
+    public void testCreateTableWithDuplicateColumns() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        PhoenixConnection conn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class);
+        String ddl = "CREATE TABLE T (ID INTEGER PRIMARY KEY, DUPE INTEGER, DUPE INTEGER)";
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (ColumnAlreadyExistsException e) {
+            assertEquals("DUPE", e.getColumnName());
+        }
+    }
+}


[18/50] [abbrv] phoenix git commit: PHOENIX-2995 Write performance severely degrades with large number of views

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index 0e044b9..04882e0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -64,9 +64,9 @@ public class TransactionUtil {
             .build().buildException();
     }
     
-    public static TransactionAwareHTable getTransactionAwareHTable(HTableInterface htable, PTable table) {
+    public static TransactionAwareHTable getTransactionAwareHTable(HTableInterface htable, boolean isImmutableRows) {
     	// Conflict detection is not needed for tables with write-once/append-only data
-    	return new TransactionAwareHTable(htable, table.isImmutableRows() ? TxConstants.ConflictDetection.NONE : TxConstants.ConflictDetection.ROW);
+    	return new TransactionAwareHTable(htable, isImmutableRows ? TxConstants.ConflictDetection.NONE : TxConstants.ConflictDetection.ROW);
     }
     
     // we resolve transactional tables at the txn read pointer

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3130fa99/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
index 9c92679..ef88c8c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
@@ -31,17 +31,15 @@ import com.google.common.collect.Sets;
 
 public class PMetaDataImplTest {
     
-    private static PMetaData addToTable(PMetaData metaData, String name, int size, TestTimeKeeper timeKeeper) throws SQLException {
+    private static void addToTable(PMetaData metaData, String name, int size, TestTimeKeeper timeKeeper) throws SQLException {
         PTable table = new PSizedTable(new PTableKey(null,name), size);
-        PMetaData newMetaData = metaData.addTable(table, System.currentTimeMillis());
+        metaData.addTable(table, System.currentTimeMillis());
         timeKeeper.incrementTime();
-        return newMetaData;
     }
     
-    private static PMetaData removeFromTable(PMetaData metaData, String name, TestTimeKeeper timeKeeper) throws SQLException {
-        PMetaData newMetaData =  metaData.removeTable(null, name, null, HConstants.LATEST_TIMESTAMP);
+    private static void removeFromTable(PMetaData metaData, String name, TestTimeKeeper timeKeeper) throws SQLException {
+        metaData.removeTable(null, name, null, HConstants.LATEST_TIMESTAMP);
         timeKeeper.incrementTime();
-        return newMetaData;
     }
     
     private static PTable getFromTable(PMetaData metaData, String name, TestTimeKeeper timeKeeper) throws TableNotFoundException {
@@ -77,41 +75,41 @@ public class PMetaDataImplTest {
         long maxSize = 10;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
         PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
-        metaData = addToTable(metaData, "a", 5, timeKeeper);
+        addToTable(metaData, "a", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 4, timeKeeper);
+        addToTable(metaData, "b", 4, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 3, timeKeeper);
+        addToTable(metaData, "c", 3, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "b","c");
 
-        metaData = addToTable(metaData, "b", 8, timeKeeper);
+        addToTable(metaData, "b", 8, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "b");
 
-        metaData = addToTable(metaData, "d", 11, timeKeeper);
+        addToTable(metaData, "d", 11, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
         
-        metaData = removeFromTable(metaData, "d", timeKeeper);
+        removeFromTable(metaData, "d", timeKeeper);
         assertNames(metaData);
         
-        metaData = addToTable(metaData, "a", 4, timeKeeper);
+        addToTable(metaData, "a", 4, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 3, timeKeeper);
+        addToTable(metaData, "b", 3, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 2, timeKeeper);
+        addToTable(metaData, "c", 2, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b","c");
         
         getFromTable(metaData, "a", timeKeeper);
-        metaData = addToTable(metaData, "d", 3, timeKeeper);
+        addToTable(metaData, "d", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "c", "a","d");
         
         // Clone maintains insert order
         metaData = metaData.clone();
-        metaData = addToTable(metaData, "e", 6, timeKeeper);
+        addToTable(metaData, "e", 6, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "d","e");
     }
@@ -121,17 +119,17 @@ public class PMetaDataImplTest {
         long maxSize = 5;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
         PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
-        metaData = addToTable(metaData, "a", 1, timeKeeper);
+        addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1, timeKeeper);
+        addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "a", "b");
-        metaData = addToTable(metaData, "c", 3, timeKeeper);
+        addToTable(metaData, "c", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b", "c");
         getFromTable(metaData, "a", timeKeeper);
         getFromTable(metaData, "b", timeKeeper);
-        metaData = addToTable(metaData, "d", 3, timeKeeper);
+        addToTable(metaData, "d", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b", "d");
     }
@@ -141,18 +139,18 @@ public class PMetaDataImplTest {
         long maxSize = 5;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
         PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
-        metaData = addToTable(metaData, "a", 1, timeKeeper);
+        addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1, timeKeeper);
+        addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 5, timeKeeper);
+        addToTable(metaData, "c", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "d", 20, timeKeeper);
+        addToTable(metaData, "d", 20, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
-        metaData = addToTable(metaData, "e", 1, timeKeeper);
+        addToTable(metaData, "e", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "f", 2, timeKeeper);
+        addToTable(metaData, "f", 2, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "e", "f");
     }
@@ -162,18 +160,18 @@ public class PMetaDataImplTest {
         long maxSize = 0;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
         PMetaData metaData = new PMetaDataImpl(0, maxSize, timeKeeper);
-        metaData = addToTable(metaData, "a", 1, timeKeeper);
+        addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1, timeKeeper);
+        addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "c", 5, timeKeeper);
+        addToTable(metaData, "c", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "d", 20, timeKeeper);
+        addToTable(metaData, "d", 20, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
-        metaData = addToTable(metaData, "e", 1, timeKeeper);
+        addToTable(metaData, "e", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "f", 2, timeKeeper);
+        addToTable(metaData, "f", 2, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "f");
     }
@@ -184,12 +182,12 @@ public class PMetaDataImplTest {
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
         PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
         String tableName = "a";
-        metaData = addToTable(metaData, tableName, 1, timeKeeper);
+        addToTable(metaData, tableName, 1, timeKeeper);
         PTableRef aTableRef = metaData.getTableRef(new PTableKey(null,tableName));
         assertNotNull(aTableRef);
         assertEquals(1, metaData.getAge(aTableRef));
         tableName = "b";
-        metaData = addToTable(metaData, tableName, 1, timeKeeper);
+        addToTable(metaData, tableName, 1, timeKeeper);
         PTableRef bTableRef = metaData.getTableRef(new PTableKey(null,tableName));
         assertNotNull(bTableRef);
         assertEquals(1, metaData.getAge(bTableRef));


[29/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index d3f54fe..0377a37 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -75,17 +75,11 @@ import org.apache.tephra.hbase.TransactionAwareHTable;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-public class TransactionIT extends BaseHBaseManagedTimeIT {
-    
-    private static final String FULL_TABLE_NAME = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + TRANSACTIONAL_DATA_TABLE;
-    
-    @Before
-    public void setUp() throws SQLException {
-        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
-    }
+public class TransactionIT extends BaseHBaseManagedTimeTableReuseIT {
     
+
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
@@ -94,13 +88,16 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         
     @Test
     public void testReadOwnWrites() throws Exception {
-        String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        String selectSql = "SELECT * FROM "+ fullTableName;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(false);
             ResultSet rs = conn.createStatement().executeQuery(selectSql);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -126,13 +123,16 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testTxnClosedCorrecty() throws Exception {
-        String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        String selectSql = "SELECT * FROM "+fullTableName;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(false);
             ResultSet rs = conn.createStatement().executeQuery(selectSql);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -155,14 +155,17 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDelete() throws Exception {
-        String selectSQL = "SELECT * FROM " + FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        String selectSQL = "SELECT * FROM " + fullTableName;
         try (Connection conn1 = DriverManager.getConnection(getUrl()); 
                 Connection conn2 = DriverManager.getConnection(getUrl())) {
             conn1.setAutoCommit(false);
             ResultSet rs = conn1.createStatement().executeQuery(selectSQL);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -173,11 +176,11 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             
             // verify rows can be read even though commit has not been called
-            int rowsDeleted = conn1.createStatement().executeUpdate("DELETE FROM " + FULL_TABLE_NAME);
+            int rowsDeleted = conn1.createStatement().executeUpdate("DELETE FROM " + fullTableName);
             assertEquals(2, rowsDeleted);
             
             // Delete and second upsert not committed yet, so there should be one row.
-            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME);
+            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             
@@ -191,36 +194,45 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testAutoCommitQuerySingleTable() throws Exception {
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(true);
             // verify no rows returned
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + FULL_TABLE_NAME);
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName);
             assertFalse(rs.next());
         }
     }
     
     @Test
     public void testAutoCommitQueryMultiTables() throws Exception {
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(true);
             // verify no rows returned
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + FULL_TABLE_NAME + " a JOIN " + FULL_TABLE_NAME + " b ON (a.long_pk = b.int_pk)");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName + " a JOIN " + fullTableName + " b ON (a.long_pk = b.int_pk)");
             assertFalse(rs.next());
         } 
     }
     
     @Test
     public void testColConflicts() throws Exception {
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
         try (Connection conn1 = DriverManager.getConnection(getUrl()); 
                 Connection conn2 = DriverManager.getConnection(getUrl())) {
             conn1.setAutoCommit(false);
             conn2.setAutoCommit(false);
-            String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
+            String selectSql = "SELECT * FROM "+fullTableName;
             conn1.setAutoCommit(false);
             ResultSet rs = conn1.createStatement().executeQuery(selectSql);
             assertFalse(rs.next());
             // upsert row using conn1
-            String upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsertSql);
             TestUtil.setRowKeyColumns(stmt, 1);
             stmt.setInt(7, 10);
@@ -243,24 +255,24 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         }
     }
     
-    private void testRowConflicts() throws Exception {
-        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+    private void testRowConflicts(String fullTableName) throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl());
                 Connection conn2 = DriverManager.getConnection(getUrl())) {
             conn1.setAutoCommit(false);
             conn2.setAutoCommit(false);
-            String selectSql = "SELECT * FROM "+FULL_TABLE_NAME;
+            String selectSql = "SELECT * FROM "+fullTableName;
             conn1.setAutoCommit(false);
             ResultSet rs = conn1.createStatement().executeQuery(selectSql);
-            boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, FULL_TABLE_NAME)).isImmutableRows();
+            boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, fullTableName)).isImmutableRows();
             assertFalse(rs.next());
             // upsert row using conn1
-            String upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsertSql);
             TestUtil.setRowKeyColumns(stmt, 1);
             stmt.setInt(7, 10);
             stmt.execute();
             // upsert row using conn2
-            upsertSql = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, b.int_col2) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, b.int_col2) VALUES(?, ?, ?, ?, ?, ?, ?)";
             stmt = conn2.prepareStatement(upsertSql);
             TestUtil.setRowKeyColumns(stmt, 1);
             stmt.setInt(7, 11);
@@ -281,51 +293,59 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testRowConflictDetected() throws Exception {
-        testRowConflicts();
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        testRowConflicts(fullTableName);
     }
     
     @Test
     public void testNoConflictDetectionForImmutableRows() throws Exception {
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("ALTER TABLE " + FULL_TABLE_NAME + " SET IMMUTABLE_ROWS=true");
-        testRowConflicts();
+        conn.createStatement().execute("ALTER TABLE " + fullTableName + " SET IMMUTABLE_ROWS=true");
+        testRowConflicts(fullTableName);
     }
     
     @Test
     public void testNonTxToTxTable() throws Exception {
+        String nonTxTableName = generateRandomString();
+
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE NON_TX_TABLE(k INTEGER PRIMARY KEY, v VARCHAR)");
-        conn.createStatement().execute("UPSERT INTO NON_TX_TABLE VALUES (1)");
-        conn.createStatement().execute("UPSERT INTO NON_TX_TABLE VALUES (2, 'a')");
-        conn.createStatement().execute("UPSERT INTO NON_TX_TABLE VALUES (3, 'b')");
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (1)");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (2, 'a')");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (3, 'b')");
         conn.commit();
         
-        conn.createStatement().execute("CREATE INDEX IDX ON NON_TX_TABLE(v)");
+        conn.createStatement().execute("CREATE INDEX IDX ON " + nonTxTableName + "(v)");
         // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("NON_TX_TABLE"));
+        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
         List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
         for (Put put : puts) {
             put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
         }
         htable.put(puts);
         
-        conn.createStatement().execute("ALTER TABLE NON_TX_TABLE SET TRANSACTIONAL=true");
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + " SET TRANSACTIONAL=true");
         
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("NON_TX_TABLE"));
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("IDX"));
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
 
-        conn.createStatement().execute("UPSERT INTO NON_TX_TABLE VALUES (4, 'c')");
-        ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ k FROM NON_TX_TABLE WHERE v IS NULL");
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "NON_TX_TABLE")).isTransactional());
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ k FROM " + nonTxTableName + " WHERE v IS NULL");
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  nonTxTableName)).isTransactional());
         assertTrue(rs.next());
         assertEquals(1,rs.getInt(1));
         assertFalse(rs.next());
         conn.commit();
         
-        conn.createStatement().execute("UPSERT INTO NON_TX_TABLE VALUES (5, 'd')");
-        rs = conn.createStatement().executeQuery("SELECT k FROM NON_TX_TABLE");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (5, 'd')");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
         assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "IDX")).isTransactional());
         assertTrue(rs.next());
         assertEquals(1,rs.getInt(1));
@@ -340,7 +360,7 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         conn.rollback();
         
-        rs = conn.createStatement().executeQuery("SELECT k FROM NON_TX_TABLE");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
         assertTrue(rs.next());
         assertEquals(1,rs.getInt(1));
         assertTrue(rs.next());
@@ -355,13 +375,15 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     @Ignore
     @Test
     public void testNonTxToTxTableFailure() throws Exception {
+        String nonTxTableName = generateRandomString();
+
         Connection conn = DriverManager.getConnection(getUrl());
         // Put table in SYSTEM schema to prevent attempts to update the cache after we disable SYSTEM.CATALOG
-        conn.createStatement().execute("CREATE TABLE SYSTEM.NON_TX_TABLE(k INTEGER PRIMARY KEY, v VARCHAR)");
-        conn.createStatement().execute("UPSERT INTO SYSTEM.NON_TX_TABLE VALUES (1)");
+        conn.createStatement().execute("CREATE TABLE SYSTEM." + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
+        conn.createStatement().execute("UPSERT INTO SYSTEM." + nonTxTableName + " VALUES (1)");
         conn.commit();
         // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM.NON_TX_TABLE"));
+        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
         Put put = new Put(PInteger.INSTANCE.toBytes(1));
         put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
         htable.put(put);
@@ -372,7 +394,7 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             // This will succeed initially in updating the HBase metadata, but then will fail when
             // the SYSTEM.CATALOG table is attempted to be updated, exercising the code to restore
             // the coprocessors back to the non transactional ones.
-            conn.createStatement().execute("ALTER TABLE SYSTEM.NON_TX_TABLE SET TRANSACTIONAL=true");
+            conn.createStatement().execute("ALTER TABLE SYSTEM." + nonTxTableName + " SET TRANSACTIONAL=true");
             fail();
         } catch (SQLException e) {
             assertTrue(e.getMessage().contains(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " is disabled"));
@@ -381,28 +403,30 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             admin.close();
         }
         
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM SYSTEM.NON_TX_TABLE WHERE v IS NULL");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM SYSTEM." + nonTxTableName + " WHERE v IS NULL");
         assertTrue(rs.next());
         assertEquals(1,rs.getInt(1));
         assertFalse(rs.next());
         
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM.NON_TX_TABLE"));
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
         assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
         assertEquals(1,conn.unwrap(PhoenixConnection.class).getQueryServices().
-                getTableDescriptor(Bytes.toBytes("SYSTEM.NON_TX_TABLE")).
+                getTableDescriptor(Bytes.toBytes("SYSTEM." + nonTxTableName)).
                 getFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions());
     }
     
     @Test
     public void testProperties() throws Exception {
+        String nonTxTableName = generateRandomString();
+
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE NON_TX_TABLE1(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR) TTL=1000");
-        conn.createStatement().execute("CREATE INDEX idx1 ON NON_TX_TABLE1(a.v, b.v) TTL=1000");
-        conn.createStatement().execute("CREATE INDEX idx2 ON NON_TX_TABLE1(c.v) INCLUDE (a.v, b.v) TTL=1000");
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "1(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR) TTL=1000");
+        conn.createStatement().execute("CREATE INDEX idx1 ON " + nonTxTableName + "1(a.v, b.v) TTL=1000");
+        conn.createStatement().execute("CREATE INDEX idx2 ON " + nonTxTableName + "1(c.v) INCLUDE (a.v, b.v) TTL=1000");
 
-        conn.createStatement().execute("ALTER TABLE NON_TX_TABLE1 SET TRANSACTIONAL=true");
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "1 SET TRANSACTIONAL=true");
 
-        HTableDescriptor desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes("NON_TX_TABLE1"));
+        HTableDescriptor desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes(nonTxTableName + "1"));
         for (HColumnDescriptor colDesc : desc.getFamilies()) {
             assertEquals(QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL, colDesc.getMaxVersions());
             assertEquals(1000, colDesc.getTimeToLive());
@@ -423,39 +447,39 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             assertEquals(1000, Integer.parseInt(colDesc.getValue(TxConstants.PROPERTY_TTL)));
         }
         
-        conn.createStatement().execute("CREATE TABLE NON_TX_TABLE2(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
-        conn.createStatement().execute("ALTER TABLE NON_TX_TABLE2 SET TRANSACTIONAL=true, VERSIONS=10");
-        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes("NON_TX_TABLE2"));
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "2(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "2 SET TRANSACTIONAL=true, VERSIONS=10");
+        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes( nonTxTableName + "2"));
         for (HColumnDescriptor colDesc : desc.getFamilies()) {
             assertEquals(10, colDesc.getMaxVersions());
             assertEquals(HColumnDescriptor.DEFAULT_TTL, colDesc.getTimeToLive());
             assertEquals(null, colDesc.getValue(TxConstants.PROPERTY_TTL));
         }
-        conn.createStatement().execute("ALTER TABLE NON_TX_TABLE2 SET TTL=1000");
-        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes("NON_TX_TABLE2"));
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "2 SET TTL=1000");
+        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes( nonTxTableName + "2"));
         for (HColumnDescriptor colDesc : desc.getFamilies()) {
             assertEquals(10, colDesc.getMaxVersions());
             assertEquals(1000, colDesc.getTimeToLive());
             assertEquals(1000, Integer.parseInt(colDesc.getValue(TxConstants.PROPERTY_TTL)));
         }
 
-        conn.createStatement().execute("CREATE TABLE NON_TX_TABLE3(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
-        conn.createStatement().execute("ALTER TABLE NON_TX_TABLE3 SET TRANSACTIONAL=true, b.VERSIONS=10, c.VERSIONS=20");
-        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes("NON_TX_TABLE3"));
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "3(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "3 SET TRANSACTIONAL=true, b.VERSIONS=10, c.VERSIONS=20");
+        desc = conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(Bytes.toBytes( nonTxTableName + "3"));
         assertEquals(QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL, desc.getFamily(Bytes.toBytes("A")).getMaxVersions());
         assertEquals(10, desc.getFamily(Bytes.toBytes("B")).getMaxVersions());
         assertEquals(20, desc.getFamily(Bytes.toBytes("C")).getMaxVersions());
 
-        conn.createStatement().execute("CREATE TABLE NON_TX_TABLE4(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "4(k INTEGER PRIMARY KEY, a.v VARCHAR, b.v VARCHAR, c.v VARCHAR)");
         try {
-            conn.createStatement().execute("ALTER TABLE NON_TX_TABLE4 SET TRANSACTIONAL=true, VERSIONS=1");
+            conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "4 SET TRANSACTIONAL=true, VERSIONS=1");
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.TX_MAX_VERSIONS_MUST_BE_GREATER_THAN_ONE.getErrorCode(), e.getErrorCode());
         }
 
         try {
-            conn.createStatement().execute("ALTER TABLE NON_TX_TABLE4 SET TRANSACTIONAL=true, b.VERSIONS=1");
+            conn.createStatement().execute("ALTER TABLE " + nonTxTableName + "4 SET TRANSACTIONAL=true, b.VERSIONS=1");
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.TX_MAX_VERSIONS_MUST_BE_GREATER_THAN_ONE.getErrorCode(), e.getErrorCode());
@@ -472,6 +496,10 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateTableToBeTransactional() throws Exception {
+
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String ddl = "CREATE TABLE TEST_TRANSACTIONAL_TABLE (k varchar primary key) transactional=true";
@@ -515,13 +543,15 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     }
 
     public void testCurrentDate() throws Exception {
-        String selectSql = "SELECT current_date() FROM "+FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        String selectSql = "SELECT current_date() FROM "+fullTableName;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(false);
             ResultSet rs = conn.createStatement().executeQuery(selectSql);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -545,34 +575,36 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testReCreateTxnTableAfterDroppingExistingNonTxnTable() throws SQLException {
+        String tableName = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         Statement stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE DEMO(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-        stmt.execute("DROP TABLE DEMO");
-        stmt.execute("CREATE TABLE DEMO(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) TRANSACTIONAL=true");
-        stmt.execute("CREATE INDEX DEMO_IDX ON DEMO (v1) INCLUDE(v2)");
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "DEMO")).isTransactional());
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "DEMO_IDX")).isTransactional());
+        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+        stmt.execute("DROP TABLE " + tableName);
+        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) TRANSACTIONAL=true");
+        stmt.execute("CREATE INDEX " + tableName + "_IDX ON " + tableName + " (v1) INCLUDE(v2)");
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName)).isTransactional());
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  tableName + "_IDX")).isTransactional());
     }
     
     @Test
     public void testRowTimestampDisabled() throws SQLException {
+        String tableName = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
             Statement stmt = conn.createStatement();
             try {
-                stmt.execute("CREATE TABLE DEMO(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP)) TRANSACTIONAL=true");
+                stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP)) TRANSACTIONAL=true");
                 fail();
             }
             catch(SQLException e) {
                 assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP.getErrorCode(), e.getErrorCode());
             }
-            stmt.execute("CREATE TABLE DEMO(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP))");
+            stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP))");
             try {
-                stmt.execute("ALTER TABLE DEMO SET TRANSACTIONAL=true");
+                stmt.execute("ALTER TABLE " + tableName + " SET TRANSACTIONAL=true");
                 fail();
             }
             catch(SQLException e) {
@@ -587,11 +619,11 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String fullTableName = generateRandomString();
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
         
         TransactionSystemClient txServiceClient = pconn.getQueryServices().getTransactionSystemClient();
 
-        String fullTableName = "T";
         Statement stmt = conn.createStatement();
         stmt.execute("CREATE TABLE " + fullTableName + "(K VARCHAR PRIMARY KEY, V1 VARCHAR, V2 VARCHAR) TRANSACTIONAL=true");
         HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(fullTableName));
@@ -706,9 +738,9 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     public void testCheckpointAndRollback() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        String fullTableName = generateRandomString();
         conn.setAutoCommit(false);
         try {
-            String fullTableName = "T";
             Statement stmt = conn.createStatement();
             stmt.execute("CREATE TABLE " + fullTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) TRANSACTIONAL=true");
             stmt.executeUpdate("upsert into " + fullTableName + " values('x', 'a', 'a')");
@@ -748,7 +780,10 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testInflightUpdateNotSeen() throws Exception {
-        String selectSQL = "SELECT * FROM " + FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        String selectSQL = "SELECT * FROM " + fullTableName;
         try (Connection conn1 = DriverManager.getConnection(getUrl()); 
                 Connection conn2 = DriverManager.getConnection(getUrl())) {
             conn1.setAutoCommit(false);
@@ -756,7 +791,7 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             ResultSet rs = conn1.createStatement().executeQuery(selectSQL);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -766,32 +801,32 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             TestUtil.setRowKeyColumns(stmt, 2);
             stmt.execute();
             
-            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME + " WHERE int_col1 IS NULL");
+            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName + " WHERE int_col1 IS NULL");
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             
-            upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, int_col1) VALUES(?, ?, ?, ?, ?, ?, 1)";
+            upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, int_col1) VALUES(?, ?, ?, ?, ?, ?, 1)";
             stmt = conn1.prepareStatement(upsert);
             TestUtil.setRowKeyColumns(stmt, 1);
             stmt.execute();
             
-            rs = conn1.createStatement().executeQuery("SELECT int_col1 FROM " + FULL_TABLE_NAME + " WHERE int_col1 = 1");
+            rs = conn1.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " WHERE int_col1 = 1");
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertFalse(rs.next());
             
-            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME + " WHERE int_col1 = 1");
+            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName + " WHERE int_col1 = 1");
             assertTrue(rs.next());
             assertEquals(0, rs.getInt(1));
-            rs = conn2.createStatement().executeQuery("SELECT * FROM " + FULL_TABLE_NAME + " WHERE int_col1 = 1");
+            rs = conn2.createStatement().executeQuery("SELECT * FROM " + fullTableName + " WHERE int_col1 = 1");
             assertFalse(rs.next());
             
             conn1.commit();
             
-            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME + " WHERE int_col1 = 1");
+            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName + " WHERE int_col1 = 1");
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
-            rs = conn2.createStatement().executeQuery("SELECT * FROM " + FULL_TABLE_NAME + " WHERE int_col1 = 1");
+            rs = conn2.createStatement().executeQuery("SELECT * FROM " + fullTableName + " WHERE int_col1 = 1");
             assertTrue(rs.next());
             assertFalse(rs.next());
         }
@@ -799,7 +834,10 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testInflightDeleteNotSeen() throws Exception {
-        String selectSQL = "SELECT * FROM " + FULL_TABLE_NAME;
+        String transTableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        ensureTableCreated(getUrl(), transTableName, TRANSACTIONAL_DATA_TABLE);
+        String selectSQL = "SELECT * FROM " + fullTableName;
         try (Connection conn1 = DriverManager.getConnection(getUrl()); 
                 Connection conn2 = DriverManager.getConnection(getUrl())) {
             conn1.setAutoCommit(false);
@@ -807,7 +845,7 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             ResultSet rs = conn1.createStatement().executeQuery(selectSQL);
             assertFalse(rs.next());
             
-            String upsert = "UPSERT INTO " + FULL_TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn1.prepareStatement(upsert);
             // upsert two rows
             TestUtil.setRowKeyColumns(stmt, 1);
@@ -817,28 +855,28 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
             
             conn1.commit();
             
-            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME);
+            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             
-            String delete = "DELETE FROM " + FULL_TABLE_NAME + " WHERE varchar_pk = 'varchar1'";
+            String delete = "DELETE FROM " + fullTableName + " WHERE varchar_pk = 'varchar1'";
             stmt = conn1.prepareStatement(delete);
             int count = stmt.executeUpdate();
             assertEquals(1,count);
             
-            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME);
+            rs = conn1.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertFalse(rs.next());
             
-            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME);
+            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             assertFalse(rs.next());
             
             conn1.commit();
             
-            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + FULL_TABLE_NAME);
+            rs = conn2.createStatement().executeQuery("SELECT count(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertFalse(rs.next());
@@ -899,23 +937,23 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     public void testInflightPartialEval() throws SQLException {
 
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String transactTableName = "TR";
+            String transactTableName = "TR" + generateRandomString();
             Statement stmt = conn.createStatement();
             stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " +
                 "TRANSACTIONAL=true");
             
             try (Connection conn1 = DriverManager.getConnection(getUrl()); Connection conn2 = DriverManager.getConnection(getUrl())) {
-                conn1.createStatement().execute("UPSERT INTO tr VALUES ('a','b','x')");
+                conn1.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','b','x')");
                 // Select to force uncommitted data to be written
-                ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM tr");
+                ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + transactTableName);
                 assertTrue(rs.next());
                 assertEquals("a", rs.getString(1));
                 assertEquals("b", rs.getString(2));
                 assertFalse(rs.next());
                 
-                conn2.createStatement().execute("UPSERT INTO tr VALUES ('a','c','x')");
+                conn2.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','c','x')");
                 // Select to force uncommitted data to be written
-                rs = conn2.createStatement().executeQuery("SELECT * FROM tr");
+                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName );
                 assertTrue(rs.next());
                 assertEquals("a", rs.getString(1));
                 assertEquals("c", rs.getString(2));
@@ -924,14 +962,14 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
                 // If the AndExpression were to see the uncommitted row from conn2, the filter would
                 // filter the row out early and no longer continue to evaluate other cells due to
                 // the way partial evaluation holds state.
-                rs = conn1.createStatement().executeQuery("SELECT * FROM tr WHERE v1 != 'c' AND v2 = 'x'");
+                rs = conn1.createStatement().executeQuery("SELECT * FROM " +  transactTableName + " WHERE v1 != 'c' AND v2 = 'x'");
                 assertTrue(rs.next());
                 assertEquals("a", rs.getString(1));
                 assertEquals("b", rs.getString(2));
                 assertFalse(rs.next());
                 
                 // Same as above for conn1 data
-                rs = conn2.createStatement().executeQuery("SELECT * FROM tr WHERE v1 != 'b' AND v2 = 'x'");
+                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName + " WHERE v1 != 'b' AND v2 = 'x'");
                 assertTrue(rs.next());
                 assertEquals("a", rs.getString(1));
                 assertEquals("c", rs.getString(2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index df9a195..9f47a1e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -33,7 +33,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -41,7 +41,6 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -53,26 +52,19 @@ import org.apache.tephra.Transaction.VisibilityLevel;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
+public class TxCheckpointIT extends BaseHBaseManagedTimeTableReuseIT {
 	
 	private final boolean localIndex;
 	private final boolean mutable;
-	private String tableName;
-    private String indexName;
-    private String seqName;
-    private String fullTableName;
 
 	public TxCheckpointIT(boolean localIndex, boolean mutable) {
 		this.localIndex = localIndex;
 		this.mutable = mutable;
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME;
-        this.indexName = "IDX_" + System.currentTimeMillis();
-        this.seqName = "SEQ_" + System.currentTimeMillis();
-        this.fullTableName = SchemaUtil.getTableName(tableName, tableName);
+
 	}
 	
 	@BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
@@ -89,6 +81,10 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpsertSelectDoesntSeeUpsertedData() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String seqName = "SEQ_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(tableName, tableName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
@@ -110,17 +106,23 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testRollbackOfUncommittedDeleteSingleCol() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(tableName, tableName);
         String indexDDL = "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE(v2)";
-        testRollbackOfUncommittedDelete(indexDDL);
+        testRollbackOfUncommittedDelete(indexDDL, fullTableName);
     }
 
     @Test
     public void testRollbackOfUncommittedDeleteMultiCol() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(tableName, tableName);
         String indexDDL = "CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + " (v1, v2)";
-        testRollbackOfUncommittedDelete(indexDDL);
+        testRollbackOfUncommittedDelete(indexDDL, fullTableName);
     }
     
-    private void testRollbackOfUncommittedDelete(String indexDDL) throws Exception {
+    private void testRollbackOfUncommittedDelete(String indexDDL, String fullTableName) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
@@ -207,6 +209,9 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
     
 	@Test
 	public void testCheckpointForUpsertSelect() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(tableName, tableName);
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
 			conn.setAutoCommit(false);
@@ -222,17 +227,17 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
             stmt.executeUpdate("upsert into " + fullTableName + " values(3, 'a3', 'b3')");
 			conn.commit();
 
-			upsertRows(conn);
+			upsertRows(conn, fullTableName);
 			conn.rollback();
-			verifyRows(conn, 3);
+			verifyRows(conn, fullTableName, 3);
 
-			upsertRows(conn);
+			upsertRows(conn, fullTableName);
 			conn.commit();
-			verifyRows(conn, 6);
+			verifyRows(conn, fullTableName, 6);
 		}
 	}
 
-	private void verifyRows(Connection conn, int expectedMaxId) throws SQLException {
+	private void verifyRows(Connection conn, String fullTableName, int expectedMaxId) throws SQLException {
 		ResultSet rs;
 		//query the data table
 		rs = conn.createStatement().executeQuery("select /*+ NO_INDEX */ max(id) from " + fullTableName + "");
@@ -247,7 +252,7 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
 		assertFalse(rs.next());
 	}
 
-	private void upsertRows(Connection conn) throws SQLException {
+	private void upsertRows(Connection conn, String fullTableName) throws SQLException {
 		ResultSet rs;
 		MutationState state = conn.unwrap(PhoenixConnection.class)
 				.getMutationState();
@@ -294,6 +299,9 @@ public class TxCheckpointIT extends BaseHBaseManagedTimeIT {
 	
 	@Test
     public void testCheckpointForDeleteAndUpsert() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(tableName, tableName);
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		ResultSet rs;
 		try (Connection conn = DriverManager.getConnection(getUrl(), props);) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index d7c7c62..dbd0a99 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -105,7 +105,7 @@ public interface QueryServices extends SQLCloseable {
 
     public static final String MASTER_INFO_PORT_ATTRIB = "hbase.master.info.port";
     public static final String REGIONSERVER_INFO_PORT_ATTRIB = "hbase.regionserver.info.port";
-    public static final String REGIONSERVER_LEASE_PERIOD_ATTRIB = "hbase.regionserver.lease.period";
+    public static final String HBASE_CLIENT_SCANNER_TIMEOUT_ATTRIB = "hbase.client.scanner.timeout.period";
     public static final String RPC_TIMEOUT_ATTRIB = "hbase.rpc.timeout";
     public static final String DYNAMIC_JARS_DIR_KEY = "hbase.dynamic.jars.dir";
     public static final String ZOOKEEPER_QUORUM_ATTRIB = "hbase.zookeeper.quorum";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index d874860..475c141 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -58,7 +58,7 @@ import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK;
 import static org.apache.phoenix.query.QueryServices.QUEUE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.REGIONSERVER_INFO_PORT_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.REGIONSERVER_LEASE_PERIOD_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.HBASE_CLIENT_SCANNER_TIMEOUT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.RENEW_LEASE_ENABLED;
 import static org.apache.phoenix.query.QueryServices.RENEW_LEASE_THREAD_POOL_SIZE;
 import static org.apache.phoenix.query.QueryServices.RENEW_LEASE_THRESHOLD_MILLISECONDS;
@@ -555,7 +555,7 @@ public class QueryServicesOptions {
     }
 
     public QueryServicesOptions setRegionServerLeasePeriodMs(int period) {
-        return set(REGIONSERVER_LEASE_PERIOD_ATTRIB, period);
+        return set(HBASE_CLIENT_SCANNER_TIMEOUT_ATTRIB, period);
     }
 
     public QueryServicesOptions setRpcTimeoutMs(int timeout) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index 33c03ab..a116a2c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -122,7 +122,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
 
     @Test
     public void testGetByteBitExpression() throws SQLException {
-        ensureTableCreated(getUrl(), TestUtil.BINARY_NAME);
+        ensureTableCreated(getUrl(), TestUtil.BINARY_NAME, TestUtil.BINARY_NAME);
         int result = 1;
         String query = "select * from " + BINARY_NAME + " where GET_BYTE(a_binary, 0)=" + result;
         Scan scan = compileStatement(query).getScan();
@@ -1044,7 +1044,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
      */
     @Test
     public void testValueComparisonInt() throws SQLException {
-        ensureTableCreated(getUrl(),"PKIntValueTest");
+        ensureTableCreated(getUrl(),"PKIntValueTest", "PKIntValueTest");
         String query;
         // int <-> long
         // Case 1: int = long, comparison always false, key is degenerated.
@@ -1070,7 +1070,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
 
     @Test
     public void testValueComparisonUnsignedInt() throws SQLException {
-        ensureTableCreated(getUrl(), "PKUnsignedIntValueTest");
+        ensureTableCreated(getUrl(), "PKUnsignedIntValueTest", "PKUnsignedIntValueTest");
         String query;
         // unsigned_int <-> negative int/long
         // Case 1: unsigned_int = negative int, always false;
@@ -1102,7 +1102,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
 
     @Test
     public void testValueComparisonUnsignedLong() throws SQLException {
-        ensureTableCreated(getUrl(), "PKUnsignedLongValueTest");
+        ensureTableCreated(getUrl(), "PKUnsignedLongValueTest", "PKUnsignedLongValueTest");
         String query;
         // unsigned_long <-> positive int/long
         // Case 1: unsigned_long = negative int/long, always false;
@@ -1256,7 +1256,7 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     
     @Test
     public void testForceRangeScanKeepsFilters() throws SQLException {
-        ensureTableCreated(getUrl(), TestUtil.ENTITY_HISTORY_TABLE_NAME);
+        ensureTableCreated(getUrl(), TestUtil.ENTITY_HISTORY_TABLE_NAME, TestUtil.ENTITY_HISTORY_TABLE_NAME);
         String tenantId = "000000000000001";
         String keyPrefix = "002";
         String query = "select /*+ RANGE_SCAN */ ORGANIZATION_ID, PARENT_ID, CREATED_DATE, ENTITY_HISTORY_ID from " + TestUtil.ENTITY_HISTORY_TABLE_NAME + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index fdb4002..daef367 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -83,9 +83,11 @@ import static org.apache.phoenix.util.TestUtil.TABLE_WITH_ARRAY;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_SALTING;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.SUM_DOUBLE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -193,6 +195,7 @@ import org.apache.tephra.persist.InMemoryTransactionStateStorage;
  * make sure to shutdown the mini cluster in a method annotated by @AfterClass.  
  *
  */
+
 public abstract class BaseTest {
     protected static final String TEST_TABLE_SCHEMA = "(" +
             "   varchar_pk VARCHAR NOT NULL, " +
@@ -227,7 +230,7 @@ public abstract class BaseTest {
             .setNameFormat("DROP-TABLE-BASETEST" + "-thread-%s").build();
     private static final ExecutorService dropHTableService = Executors
             .newSingleThreadExecutor(factory);
-    
+
     static {
         ImmutableMap.Builder<String,String> builder = ImmutableMap.builder();
         builder.put(ENTITY_HISTORY_TABLE_NAME,"create table " + ENTITY_HISTORY_TABLE_NAME +
@@ -442,7 +445,7 @@ public abstract class BaseTest {
         builder.put(INDEX_DATA_TABLE, "create table " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + INDEX_DATA_TABLE + TEST_TABLE_SCHEMA + "IMMUTABLE_ROWS=true");
         builder.put(MUTABLE_INDEX_DATA_TABLE, "create table " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + TEST_TABLE_SCHEMA);
         builder.put(TRANSACTIONAL_DATA_TABLE, "create table " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + TRANSACTIONAL_DATA_TABLE + TEST_TABLE_SCHEMA + "TRANSACTIONAL=true");
-        builder.put("SumDoubleTest","create table SumDoubleTest" +
+        builder.put(SUM_DOUBLE_NAME,"create table SumDoubleTest" +
                 "   (id varchar not null primary key, d DOUBLE, f FLOAT, ud UNSIGNED_DOUBLE, uf UNSIGNED_FLOAT, i integer, de decimal)");
         builder.put(JOIN_ORDER_TABLE_FULL_NAME, "create table " + JOIN_ORDER_TABLE_FULL_NAME +
                 "   (\"order_id\" varchar(15) not null primary key, " +
@@ -803,24 +806,31 @@ public abstract class BaseTest {
     }
 
     protected static void ensureTableCreated(String url, String tableName) throws SQLException {
-        ensureTableCreated(url, tableName, null, null);
+        ensureTableCreated(url, tableName, tableName, null, null);
+    }
+
+    protected static void ensureTableCreated(String url, String tableName, String tableDDLType) throws SQLException {
+        ensureTableCreated(url, tableName, tableDDLType, null, null);
     }
 
-    public static void ensureTableCreated(String url, String tableName, byte[][] splits) throws SQLException {
-        ensureTableCreated(url, tableName, splits, null);
+    public static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits) throws SQLException {
+        ensureTableCreated(url, tableName, tableDDLType, splits, null);
     }
 
-    protected static void ensureTableCreated(String url, String tableName, Long ts) throws SQLException {
-        ensureTableCreated(url, tableName, null, ts);
+    protected static void ensureTableCreated(String url, String tableName, String tableDDLType, Long ts) throws SQLException {
+        ensureTableCreated(url, tableName, tableDDLType, null, ts);
     }
 
-    protected static void ensureTableCreated(String url, String tableName, byte[][] splits, Long ts) throws SQLException {
-        String ddl = tableDDLMap.get(tableName);
+    protected static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits, Long ts) throws SQLException {
+        String ddl = tableDDLMap.get(tableDDLType);
+        if(!tableDDLType.equals(tableName)) {
+           ddl =  ddl.replace(tableDDLType, tableName);
+        }
         createSchema(url,tableName, ts);
         createTestTable(url, ddl, splits, ts);
     }
 
-    protected static String generateRandomString() {
+    public static String generateRandomString() {
       return RandomStringUtils.randomAlphabetic(20).toUpperCase();
     }
 
@@ -1052,16 +1062,20 @@ public abstract class BaseTest {
         }
         rs.close();
     }
-    
+
     protected static void initSumDoubleValues(byte[][] splits, String url) throws Exception {
-        ensureTableCreated(url, "SumDoubleTest", splits);
+        initSumDoubleValues(SUM_DOUBLE_NAME, splits, url);
+    }
+
+    protected static void initSumDoubleValues(String tableName, byte[][] splits, String url) throws Exception {
+        ensureTableCreated(url, tableName, SUM_DOUBLE_NAME, splits);
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(url, props);
         try {
             // Insert all rows at ts
             PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    "SumDoubleTest(" +
+                    "upsert into " + tableName +
+                    "(" +
                     "    id, " +
                     "    d, " +
                     "    f, " +
@@ -1108,22 +1122,32 @@ public abstract class BaseTest {
             conn.close();
         }
     }
-    
-    protected static void initATableValues(String tenantId, byte[][] splits, String url) throws Exception {
-        initATableValues(tenantId, splits, null, url);
+
+    protected static String initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
+        return initATableValues(tenantId, splits, date, ts, getUrl());
     }
     
-    protected static void initATableValues(String tenantId, byte[][] splits, Date date, String url) throws Exception {
-        initATableValues(tenantId, splits, date, null, url);
+    protected static String initATableValues(String tenantId, byte[][] splits, String url) throws Exception {
+        return initATableValues(tenantId, splits, null, url);
     }
     
+    protected static String initATableValues(String tenantId, byte[][] splits, Date date, String url) throws Exception {
+        return initATableValues(tenantId, splits, date, null, url);
+    }
+
+    protected static String initATableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+        return initATableValues(null, tenantId, splits, date, ts, url);
+    }
     
-    
-    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+    protected static String initATableValues(String tableName, String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+        if(tableName == null) {
+            tableName = generateRandomString();
+        }
+        String tableDDLType = ATABLE_NAME;
         if (ts == null) {
-            ensureTableCreated(url, ATABLE_NAME, splits);
+            ensureTableCreated(url, tableName, tableDDLType, splits);
         } else {
-            ensureTableCreated(url, ATABLE_NAME, splits, ts-5);
+            ensureTableCreated(url, tableName, tableDDLType, splits, ts-5);
         }
         
         Properties props = new Properties();
@@ -1134,8 +1158,8 @@ public abstract class BaseTest {
         try {
             // Insert all rows at ts
             PreparedStatement stmt = conn.prepareStatement(
-                    "upsert into " +
-                    "ATABLE(" +
+                    "upsert into " + tableName +
+                    "(" +
                     "    ORGANIZATION_ID, " +
                     "    ENTITY_ID, " +
                     "    A_STRING, " +
@@ -1322,12 +1346,10 @@ public abstract class BaseTest {
             conn.commit();
         } finally {
             conn.close();
+            return tableName;
         }
     }
-    
-    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-        initATableValues(tenantId, splits, date, ts, getUrl());
-    }
+
     
     protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
         initEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
@@ -1347,9 +1369,9 @@ public abstract class BaseTest {
     
     private static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
         if (ts == null) {
-            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, splits);
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits);
         } else {
-            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, splits, ts-2);
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits, ts-2);
         }
         
         Properties props = new Properties();
@@ -1451,9 +1473,9 @@ public abstract class BaseTest {
     
     protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
         if (ts == null) {
-            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, splits);
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits);
         } else {
-            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2);
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2);
         }
         
         Properties props = new Properties();
@@ -1555,15 +1577,15 @@ public abstract class BaseTest {
     
     protected static void initJoinTableValues(String url, byte[][] splits, Long ts) throws Exception {
         if (ts == null) {
-            ensureTableCreated(url, JOIN_CUSTOMER_TABLE_FULL_NAME, splits);
-            ensureTableCreated(url, JOIN_ITEM_TABLE_FULL_NAME, splits);
-            ensureTableCreated(url, JOIN_SUPPLIER_TABLE_FULL_NAME, splits);
-            ensureTableCreated(url, JOIN_ORDER_TABLE_FULL_NAME, splits);
+            ensureTableCreated(url, JOIN_CUSTOMER_TABLE_FULL_NAME, JOIN_CUSTOMER_TABLE_FULL_NAME, splits);
+            ensureTableCreated(url, JOIN_ITEM_TABLE_FULL_NAME, JOIN_ITEM_TABLE_FULL_NAME, splits);
+            ensureTableCreated(url, JOIN_SUPPLIER_TABLE_FULL_NAME, JOIN_SUPPLIER_TABLE_FULL_NAME, splits);
+            ensureTableCreated(url, JOIN_ORDER_TABLE_FULL_NAME, JOIN_ORDER_TABLE_FULL_NAME, splits);
         } else {
-            ensureTableCreated(url, JOIN_CUSTOMER_TABLE_FULL_NAME, splits, ts - 2);
-            ensureTableCreated(url, JOIN_ITEM_TABLE_FULL_NAME, splits, ts - 2);
-            ensureTableCreated(url, JOIN_SUPPLIER_TABLE_FULL_NAME, splits, ts - 2);
-            ensureTableCreated(url, JOIN_ORDER_TABLE_FULL_NAME, splits, ts - 2);
+            ensureTableCreated(url, JOIN_CUSTOMER_TABLE_FULL_NAME, JOIN_CUSTOMER_TABLE_FULL_NAME, splits, ts - 2);
+            ensureTableCreated(url, JOIN_ITEM_TABLE_FULL_NAME, JOIN_ITEM_TABLE_FULL_NAME, splits, ts - 2);
+            ensureTableCreated(url, JOIN_SUPPLIER_TABLE_FULL_NAME, JOIN_SUPPLIER_TABLE_FULL_NAME, splits, ts - 2);
+            ensureTableCreated(url, JOIN_ORDER_TABLE_FULL_NAME, JOIN_ORDER_TABLE_FULL_NAME, splits, ts - 2);
         }
         
         SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
@@ -2063,5 +2085,29 @@ public abstract class BaseTest {
         } finally {
             conn.close();
         }
-    }  
+    }
+
+    protected static void verifySequence(String tenantID, String sequenceName, String sequenceSchemaName, boolean exists) throws SQLException {
+
+        PhoenixConnection phxConn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
+        String ddl = "SELECT "
+                + PhoenixDatabaseMetaData.TENANT_ID + ","
+                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
+                + PhoenixDatabaseMetaData.SEQUENCE_NAME
+                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE
+                + " WHERE ";
+
+        ddl += " TENANT_ID  " + ((tenantID == null ) ? "IS NULL " : " = '" + tenantID + "'");
+        ddl += " AND SEQUENCE_NAME " + ((sequenceName == null) ? "IS NULL " : " = '" +  sequenceName + "'");
+        ddl += " AND SEQUENCE_SCHEMA " + ((sequenceSchemaName == null) ? "IS NULL " : " = '" + sequenceSchemaName + "'" );
+
+        ResultSet rs = phxConn.createStatement().executeQuery(ddl);
+
+        if(exists) {
+            assertTrue(rs.next());
+        } else {
+            assertFalse(rs.next());
+        }
+        phxConn.close();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 0a11977..68e7b9b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.util;
 
+import static org.apache.phoenix.query.BaseTest.generateRandomString;
 import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY_NAME;
 import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_NAME;
@@ -449,8 +450,8 @@ public class TestUtil {
      * @param input
      *            input to be inserted
      */
-    public static void upsertRow(Connection conn, String sortOrder, int id, Object input) throws SQLException {
-        String dml = String.format("UPSERT INTO TEST_TABLE_%s VALUES(?,?)", sortOrder);
+    public static void upsertRow(Connection conn, String tableName, String sortOrder, int id, Object input) throws SQLException {
+        String dml = String.format("UPSERT INTO " + tableName + "_%s VALUES(?,?)", sortOrder);
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setInt(1, id);
         if (input instanceof String)
@@ -471,11 +472,11 @@ public class TestUtil {
         conn.commit();
     }
 
-    private static void createTable(Connection conn, String inputSqlType, String sortOrder) throws SQLException {
+    private static void createTable(Connection conn, String inputSqlType, String tableName, String sortOrder) throws SQLException {
         String dmlFormat =
-            "CREATE TABLE TEST_TABLE_%s" + "(id INTEGER NOT NULL, pk %s NOT NULL, " + "kv %s "
+            "CREATE TABLE " + tableName + "_%s (id INTEGER NOT NULL, pk %s NOT NULL, " + "kv %s "
                 + "CONSTRAINT PK_CONSTRAINT PRIMARY KEY (id, pk %s))";
-        String ddl = String.format(dmlFormat, sortOrder, inputSqlType, inputSqlType, sortOrder);
+        String ddl = String.format(dmlFormat,sortOrder, inputSqlType, inputSqlType, sortOrder);
         conn.createStatement().execute(ddl);
         conn.commit();
     }
@@ -491,13 +492,15 @@ public class TestUtil {
      * @param inputList
      *            list of values to be inserted into the pk column
      */
-    public static void initTables(Connection conn, String inputSqlType, List<Object> inputList) throws Exception {
-        createTable(conn, inputSqlType, "ASC");
-        createTable(conn, inputSqlType, "DESC");
+    public static String initTables(Connection conn, String inputSqlType, List<Object> inputList) throws Exception {
+        String tableName = generateRandomString();
+        createTable(conn, inputSqlType, tableName, "ASC");
+        createTable(conn, inputSqlType, tableName, "DESC");
         for (int i = 0; i < inputList.size(); ++i) {
-            upsertRow(conn, "ASC", i, inputList.get(i));
-            upsertRow(conn, "DESC", i, inputList.get(i));
+            upsertRow(conn, tableName, "ASC", i, inputList.get(i));
+            upsertRow(conn, tableName, "DESC", i, inputList.get(i));
         }
+        return tableName;
     }
     
     public static List<KeyRange> getAllSplits(Connection conn, String tableName) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
----------------------------------------------------------------------
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
index a59a356..22dfa36 100644
--- a/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/PhoenixSinkIT.java
@@ -29,7 +29,7 @@ import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.flume.sink.DefaultSinkFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.flume.serializer.EventSerializers;
 import org.apache.phoenix.flume.serializer.CustomSerializer;
 import org.apache.phoenix.flume.sink.NullPhoenixSink;
@@ -48,7 +48,7 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 
 
-public class PhoenixSinkIT extends BaseHBaseManagedTimeIT {
+public class PhoenixSinkIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private Context sinkContext;
     private PhoenixSink sink;
@@ -128,13 +128,14 @@ public class PhoenixSinkIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSinkLifecycle () {
-        
-        String ddl = "CREATE TABLE flume_test " +
+        String tableName = generateRandomString();
+
+        String ddl = "CREATE TABLE " + tableName +
                 "  (flume_time timestamp not null, col1 varchar , col2 varchar" +
                 "  CONSTRAINT pk PRIMARY KEY (flume_time))\n";
         
         sinkContext = new Context ();
-        sinkContext.put(FlumeConstants.CONFIG_TABLE, "flume_test");
+        sinkContext.put(FlumeConstants.CONFIG_TABLE,  tableName);
         sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());
         sinkContext.put(FlumeConstants.CONFIG_SERIALIZER,EventSerializers.REGEX.name());
         sinkContext.put(FlumeConstants.CONFIG_TABLE_DDL, ddl);
@@ -158,12 +159,12 @@ public class PhoenixSinkIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateTable () throws Exception {
-        
-        String ddl = "CREATE TABLE flume_test " +
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " " +
                 "  (flume_time timestamp not null, col1 varchar , col2 varchar" +
                 "  CONSTRAINT pk PRIMARY KEY (flume_time))\n";
 
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName =  tableName;
         sinkContext = new Context ();
         sinkContext.put(FlumeConstants.CONFIG_TABLE, fullTableName);
         sinkContext.put(FlumeConstants.CONFIG_JDBC_URL, getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
index 1c0a836..6863401 100644
--- a/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
+++ b/phoenix-flume/src/it/java/org/apache/phoenix/flume/RegexEventSerializerIT.java
@@ -40,7 +40,7 @@ import org.apache.flume.conf.Configurables;
 import org.apache.flume.event.EventBuilder;
 import org.apache.flume.lifecycle.LifecycleState;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.flume.serializer.EventSerializers;
 import org.apache.phoenix.flume.sink.PhoenixSink;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -51,7 +51,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 
-public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
+public class RegexEventSerializerIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private Context sinkContext;
     private PhoenixSink sink;
@@ -59,7 +59,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testKeyGenerator() throws EventDeliveryException, SQLException {
         
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName = generateRandomString();
         initSinkContextWithDefaults(fullTableName);
         
         sink = new PhoenixSink();
@@ -94,7 +94,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMismatchKeyGenerator() throws EventDeliveryException, SQLException {
         
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName = generateRandomString();
         initSinkContextWithDefaults(fullTableName);
         setConfig(FlumeConstants.CONFIG_SERIALIZER_PREFIX + FlumeConstants.CONFIG_ROWKEY_TYPE_GENERATOR,DefaultKeyGenerator.UUID.name());
      
@@ -127,7 +127,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMissingColumnsInEvent() throws EventDeliveryException, SQLException {
         
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName = generateRandomString();
         initSinkContextWithDefaults(fullTableName);
       
         sink = new PhoenixSink();
@@ -160,7 +160,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testBatchEvents() throws EventDeliveryException, SQLException {
         
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName = generateRandomString();
         initSinkContextWithDefaults(fullTableName);
       
         sink = new PhoenixSink();
@@ -205,7 +205,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     public void testApacheLogRegex() throws Exception {
         
         sinkContext = new Context ();
-        final String fullTableName = "s1.apachelogs";
+        final String fullTableName = generateRandomString();
         final String logRegex = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) \"([^ ]+) ([^ ]+)" +
                                 " ([^\"]+)\" (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\")" +
                                 " ([^ \"]*|\"[^\"]*\"))?";
@@ -280,7 +280,7 @@ public class RegexEventSerializerIT extends BaseHBaseManagedTimeIT {
     public void testEventsWithHeaders() throws Exception {
         
         sinkContext = new Context ();
-        final String fullTableName = "FLUME_TEST";
+        final String fullTableName = generateRandomString();
         final String ddl = "CREATE TABLE " + fullTableName +
                 "  (rowkey VARCHAR not null, col1 varchar , cf1.col2 varchar , host varchar , source varchar \n" +
                 "  CONSTRAINT pk PRIMARY KEY (rowkey))\n";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
index 16d2d20..07618a7 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
@@ -18,7 +18,7 @@
 
 package org.apache.phoenix.pherf;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.pherf.configuration.XMLConfigParser;
 import org.apache.phoenix.pherf.result.ResultUtil;
 import org.apache.phoenix.pherf.schema.SchemaReader;
@@ -30,7 +30,7 @@ import java.nio.file.Path;
 import java.util.List;
 import java.util.Properties;
 
-public class ResultBaseTestIT extends BaseHBaseManagedTimeIT {
+public class ResultBaseTestIT extends BaseHBaseManagedTimeTableReuseIT {
     protected static final String matcherScenario = ".*scenario/.*test.*xml";
     protected static final String matcherSchema = ".*datamodel/.*test.*sql";
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
index 4ff1fb5..4f6d3c1 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/SchemaReaderIT.java
@@ -30,7 +30,7 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataModel;
 import org.apache.phoenix.pherf.configuration.Scenario;
@@ -39,7 +39,8 @@ import org.apache.phoenix.pherf.schema.SchemaReader;
 import org.apache.phoenix.pherf.util.PhoenixUtil;
 import org.junit.Test;
 
-public class SchemaReaderIT extends BaseHBaseManagedTimeIT {
+
+public class SchemaReaderIT extends BaseHBaseManagedTimeTableReuseIT {
     protected static PhoenixUtil util = PhoenixUtil.create(true);
 
     @Test 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
index 94ccc25..4ff818f 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -42,7 +42,7 @@ import org.junit.BeforeClass;
 
 import com.google.common.collect.Maps;
 
-public class BasePigIT extends BaseHBaseManagedTimeIT {
+public class BasePigIT extends BaseHBaseManagedTimeTableReuseIT {
     protected TupleFactory tupleFactory;
     protected String zkQuorum;
     protected Connection conn;
@@ -50,7 +50,7 @@ public class BasePigIT extends BaseHBaseManagedTimeIT {
     protected PigServer pigServer;
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
index ba49bab..b7db592 100644
--- a/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
+++ b/phoenix-queryserver/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
@@ -50,7 +50,7 @@ import org.junit.Test;
 /**
  * Smoke test for query server.
  */
-public class QueryServerBasicsIT extends BaseHBaseManagedTimeIT {
+public class QueryServerBasicsIT extends BaseHBaseManagedTimeTableReuseIT {
 
   private static final Log LOG = LogFactory.getLog(QueryServerBasicsIT.class);
 


[42/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)


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

Branch: refs/heads/calcite
Commit: 14dab2f40df0d09f48f9cabbaea897009f635914
Parents: 64cb9b0
Author: Samarth <sa...@salesforce.com>
Authored: Fri Aug 26 16:15:57 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Aug 26 16:15:57 2016 -0700

----------------------------------------------------------------------
 .../end2end/AlterMultiTenantTableWithViews.java | 701 ------------------
 .../AlterMultiTenantTableWithViewsIT.java       | 704 +++++++++++++++++++
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 193 ++---
 .../phoenix/end2end/AppendOnlySchemaIT.java     | 115 ++-
 .../phoenix/end2end/ArithmeticQueryIT.java      |  20 +-
 .../phoenix/end2end/ArrayAppendFunctionIT.java  | 186 ++---
 .../phoenix/end2end/ArrayConcatFunctionIT.java  | 160 +++--
 .../phoenix/end2end/ArrayPrependFunctionIT.java | 220 +++---
 .../phoenix/end2end/AutoPartitionViewsIT.java   | 131 ++--
 .../BaseHBaseManagedTimeTableReuseIT.java       |   5 +
 .../org/apache/phoenix/end2end/BaseQueryIT.java |   6 +-
 .../end2end/BaseTenantSpecificViewIndexIT.java  |  80 ++-
 .../apache/phoenix/end2end/BinaryRowKeyIT.java  |  33 +-
 .../phoenix/end2end/CSVCommonsLoaderIT.java     |  80 ++-
 .../phoenix/end2end/CoalesceFunctionIT.java     | 104 +--
 .../end2end/ColumnProjectionOptimizationIT.java |  17 +-
 .../end2end/CountDistinctCompressionIT.java     |   3 +-
 .../phoenix/end2end/CustomEntityDataIT.java     |   2 +-
 .../org/apache/phoenix/end2end/DateTimeIT.java  | 146 ++--
 .../org/apache/phoenix/end2end/DeleteIT.java    | 138 ++--
 .../apache/phoenix/end2end/DerivedTableIT.java  |   3 +-
 .../phoenix/end2end/DisableLocalIndexIT.java    |  18 +-
 .../apache/phoenix/end2end/DistinctCountIT.java |   4 +-
 .../apache/phoenix/end2end/DynamicColumnIT.java |  39 +-
 .../phoenix/end2end/EncodeFunctionIT.java       |  42 +-
 .../phoenix/end2end/EvaluationOfORIT.java       |  13 +-
 .../phoenix/end2end/ExecuteStatementsIT.java    |  42 +-
 .../phoenix/end2end/ExpFunctionEnd2EndIT.java   |  20 +-
 .../phoenix/end2end/ExtendedQueryExecIT.java    |  12 +-
 .../apache/phoenix/end2end/FunkyNamesIT.java    |   2 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   | 217 +++---
 .../apache/phoenix/end2end/HashJoinMoreIT.java  |  51 +-
 .../org/apache/phoenix/end2end/InListIT.java    |  41 +-
 .../phoenix/end2end/InMemoryOrderByIT.java      |   2 +-
 .../apache/phoenix/end2end/InstrFunctionIT.java |  48 +-
 .../org/apache/phoenix/end2end/IsNullIT.java    |  37 +-
 .../org/apache/phoenix/end2end/KeyOnlyIT.java   |   6 +-
 .../phoenix/end2end/LastValueFunctionIT.java    | 194 ++---
 .../phoenix/end2end/LnLogFunctionEnd2EndIT.java |  15 +-
 .../org/apache/phoenix/end2end/MapReduceIT.java |  61 +-
 .../phoenix/end2end/MappingTableDataTypeIT.java |   2 +-
 .../phoenix/end2end/ModulusExpressionIT.java    |  45 +-
 .../phoenix/end2end/MultiCfQueryExecIT.java     |   2 +-
 .../end2end/NamespaceSchemaMappingIT.java       |   4 +-
 .../phoenix/end2end/NativeHBaseTypesIT.java     |   2 +-
 .../org/apache/phoenix/end2end/OrderByIT.java   |  74 +-
 .../phoenix/end2end/ParallelIteratorsIT.java    |   6 +-
 .../apache/phoenix/end2end/PercentileIT.java    |  93 +--
 .../phoenix/end2end/PhoenixRuntimeIT.java       |  42 +-
 .../phoenix/end2end/ProductMetricsIT.java       |   2 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |  49 +-
 .../phoenix/end2end/QueryExecWithoutSCNIT.java  |   6 +-
 .../phoenix/end2end/QueryWithLimitIT.java       |   4 +-
 .../phoenix/end2end/ReadIsolationLevelIT.java   |   2 +-
 .../end2end/RegexpReplaceFunctionIT.java        |  17 +-
 .../phoenix/end2end/RegexpSubstrFunctionIT.java |  12 +-
 .../apache/phoenix/end2end/ReverseScanIT.java   |  58 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |  35 +-
 .../end2end/SkipScanAfterManualSplitIT.java     |  44 +-
 .../apache/phoenix/end2end/SkipScanQueryIT.java | 156 ++--
 .../phoenix/end2end/SortMergeJoinMoreIT.java    |  66 +-
 .../phoenix/end2end/SpillableGroupByIT.java     |   2 +-
 .../phoenix/end2end/SpooledTmpFileDeleteIT.java |  20 +-
 .../phoenix/end2end/SqrtFunctionEnd2EndIT.java  |  25 +-
 .../org/apache/phoenix/end2end/StddevIT.java    |  18 +-
 .../org/apache/phoenix/end2end/StringIT.java    |  66 +-
 .../org/apache/phoenix/end2end/SubqueryIT.java  |   5 +-
 .../apache/phoenix/end2end/TenantIdTypeIT.java  |   4 +-
 .../end2end/TenantSpecificViewIndexIT.java      |  48 +-
 .../phoenix/end2end/ToCharFunctionIT.java       |   7 +-
 .../phoenix/end2end/ToDateFunctionIT.java       |   9 +-
 .../java/org/apache/phoenix/end2end/TopNIT.java |   8 +-
 .../phoenix/end2end/TruncateFunctionIT.java     |   2 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   | 130 ++--
 .../phoenix/end2end/UpsertBigValuesIT.java      |  78 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |  18 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   6 +-
 .../org/apache/phoenix/end2end/UseSchemaIT.java |  42 +-
 .../phoenix/end2end/VariableLengthPKIT.java     |  38 +-
 .../phoenix/end2end/index/DropMetadataIT.java   | 113 ++-
 .../index/GlobalIndexOptimizationIT.java        | 175 ++---
 .../phoenix/end2end/index/ImmutableIndexIT.java |  36 +-
 .../end2end/index/IndexExpressionIT.java        | 327 +++++----
 .../apache/phoenix/end2end/index/IndexIT.java   |  85 ++-
 .../phoenix/end2end/index/IndexMetadataIT.java  | 253 +++----
 .../phoenix/end2end/index/LocalIndexIT.java     | 125 +++-
 .../phoenix/end2end/index/MutableIndexIT.java   |  94 ++-
 .../phoenix/end2end/index/SaltedIndexIT.java    |  95 +--
 .../phoenix/end2end/index/ViewIndexIT.java      |  54 +-
 .../end2end/index/txn/MutableRollbackIT.java    |  58 +-
 .../phoenix/end2end/index/txn/RollbackIT.java   |  18 +-
 .../phoenix/end2end/salted/SaltedTableIT.java   |   2 +-
 .../salted/SaltedTableUpsertSelectIT.java       |   1 -
 .../salted/SaltedTableVarLengthRowKeyIT.java    |   1 -
 .../phoenix/iterate/PhoenixQueryTimeoutIT.java  |   3 -
 .../iterate/RoundRobinResultIteratorIT.java     |  26 +-
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |  26 +-
 .../phoenix/rpc/UpdateCacheWithScnIT.java       |   2 +-
 .../apache/phoenix/trace/BaseTracingTestIT.java |   4 +-
 .../org/apache/phoenix/tx/TransactionIT.java    | 246 ++++---
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |  50 +-
 .../org/apache/phoenix/query/QueryServices.java |   2 +-
 .../phoenix/query/QueryServicesOptions.java     |   4 +-
 .../phoenix/compile/WhereOptimizerTest.java     |  10 +-
 .../java/org/apache/phoenix/query/BaseTest.java | 130 ++--
 .../java/org/apache/phoenix/util/TestUtil.java  |  23 +-
 .../org/apache/phoenix/flume/PhoenixSinkIT.java |  17 +-
 .../phoenix/flume/RegexEventSerializerIT.java   |  16 +-
 .../apache/phoenix/pherf/ResultBaseTestIT.java  |   4 +-
 .../apache/phoenix/pherf/SchemaReaderIT.java    |   5 +-
 .../java/org/apache/phoenix/pig/BasePigIT.java  |   6 +-
 .../phoenix/end2end/QueryServerBasicsIT.java    |   2 +-
 pom.xml                                         | 134 ++--
 113 files changed, 3974 insertions(+), 3331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViews.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViews.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViews.java
deleted file mode 100644
index 2cdfe4a..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViews.java
+++ /dev/null
@@ -1,701 +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.end2end;
-
-import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.compile.QueryPlan;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.jdbc.PhoenixStatement;
-import org.apache.phoenix.schema.ColumnNotFoundException;
-import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.SchemaUtil;
-import org.junit.Test;
-
-import com.google.common.base.Objects;
-
-public class AlterMultiTenantTableWithViews extends BaseHBaseManagedTimeIT {
-
-    private Connection getTenantConnection(String tenantId) throws Exception {
-        Properties tenantProps = new Properties();
-        tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-        return DriverManager.getConnection(getUrl(), tenantProps);
-    }
-    
-    private static long getTableSequenceNumber(PhoenixConnection conn, String tableName) throws SQLException {
-        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), SchemaUtil.normalizeIdentifier(tableName)));
-        return table.getSequenceNumber();
-    }
-    
-    private static short getMaxKeySequenceNumber(PhoenixConnection conn, String tableName) throws SQLException {
-        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), SchemaUtil.normalizeIdentifier(tableName)));
-        return SchemaUtil.getMaxKeySeq(table);
-    }
-    
-    private static void verifyNewColumns(ResultSet rs, String ... values) throws SQLException {
-        assertTrue(rs.next());
-        int i = 1;
-        for (String value : values) {
-            assertEquals(value, rs.getString(i++));
-        }
-        assertFalse(rs.next());
-        assertEquals(values.length, i - 1);
-    }
-    
-    @Test
-    public void testAddDropColumnToBaseTablePropagatesToEntireViewHierarchy() throws Exception {
-        String baseTable = "testViewHierarchy";
-        String view1 = "view1";
-        String view2 = "view2";
-        String view3 = "view3";
-        String view4 = "view4";
-        /*                                     baseTable
-                                 /                  |               \ 
-                         view1(tenant1)    view3(tenant2)          view4(global)
-                          /
-                        view2(tenant1)  
-        */
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            conn.createStatement().execute(baseTableDDL);
-            
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-                
-                String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1;
-                tenant1Conn.createStatement().execute(view2DDL);
-            }
-            
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
-                tenant2Conn.createStatement().execute(view3DDL);
-            }
-            
-            String view4DDL = "CREATE VIEW " + view4 + " AS SELECT * FROM " + baseTable;
-            conn.createStatement().execute(view4DDL);
-            
-            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD V3 VARCHAR";
-            conn.createStatement().execute(alterBaseTable);
-            
-            // verify that the column is visible to view4
-            conn.createStatement().execute("SELECT V3 FROM " + view4);
-            
-            // verify that the column is visible to view1 and view2
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                tenant1Conn.createStatement().execute("SELECT V3 from " + view1);
-                tenant1Conn.createStatement().execute("SELECT V3 from " + view2);
-            }
-            
-            // verify that the column is visible to view3
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                tenant2Conn.createStatement().execute("SELECT V3 from " + view3);
-            }
-
-            alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V1";
-            conn.createStatement().execute(alterBaseTable);
-
-            // verify that the column is not visible to view4
-            try {
-                conn.createStatement().execute("SELECT V1 FROM " + view4);
-                fail();
-            } catch (ColumnNotFoundException e) {
-            }
-            // verify that the column is not visible to view1 and view2
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                try {
-                    tenant1Conn.createStatement().execute("SELECT V1 from " + view1);
-                    fail();
-                } catch (ColumnNotFoundException e) {
-                }
-                try {
-                    tenant1Conn.createStatement().execute("SELECT V1 from " + view2);
-                    fail();
-                } catch (ColumnNotFoundException e) {
-                }
-            }
-
-            // verify that the column is not visible to view3
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                try {
-                    tenant2Conn.createStatement().execute("SELECT V1 from " + view3);
-                    fail();
-                } catch (ColumnNotFoundException e) {
-                }
-            }
-        }
-           
-    }
-    
-    @Test
-    public void testChangingPKOfBaseTableChangesPKForAllViews() throws Exception {
-        String baseTable = "testChangePKOfBaseTable";
-        String view1 = "view1";
-        String view2 = "view2";
-        String view3 = "view3";
-        String view4 = "view4";
-        /*                                     baseTable
-                                 /                  |               \ 
-                         view1(tenant1)    view3(tenant2)          view4(global)
-                          /
-                        view2(tenant1)  
-         */
-        Connection tenant1Conn = null, tenant2Conn = null;
-        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE "
-                    + baseTable
-                    + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            globalConn.createStatement().execute(baseTableDDL);
-
-            tenant1Conn = getTenantConnection("tenant1");
-            String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
-            tenant1Conn.createStatement().execute(view1DDL);
-
-            String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1;
-            tenant1Conn.createStatement().execute(view2DDL);
-
-            tenant2Conn = getTenantConnection("tenant2");
-            String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
-            tenant2Conn.createStatement().execute(view3DDL);
-
-            String view4DDL = "CREATE VIEW " + view4 + " AS SELECT * FROM " + baseTable;
-            globalConn.createStatement().execute(view4DDL);
-
-            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD NEW_PK varchar primary key ";
-            globalConn.createStatement().execute(alterBaseTable);
-            
-            // verify that the new column new_pk is now part of the primary key for the entire hierarchy
-            
-            globalConn.createStatement().execute("SELECT * FROM " + baseTable);
-            assertTrue(checkColumnPartOfPk(globalConn.unwrap(PhoenixConnection.class), "NEW_PK", baseTable));
-            
-            tenant1Conn.createStatement().execute("SELECT * FROM " + view1);
-            assertTrue(checkColumnPartOfPk(tenant1Conn.unwrap(PhoenixConnection.class), "NEW_PK", view1));
-            
-            tenant1Conn.createStatement().execute("SELECT * FROM " + view2);
-            assertTrue(checkColumnPartOfPk(tenant1Conn.unwrap(PhoenixConnection.class), "NEW_PK", view2));
-            
-            tenant2Conn.createStatement().execute("SELECT * FROM " + view3);
-            assertTrue(checkColumnPartOfPk(tenant2Conn.unwrap(PhoenixConnection.class), "NEW_PK", view3));
-            
-            globalConn.createStatement().execute("SELECT * FROM " + view4);
-            assertTrue(checkColumnPartOfPk(globalConn.unwrap(PhoenixConnection.class), "NEW_PK", view4));
-
-        } finally {
-            if (tenant1Conn != null) {
-                try {
-                    tenant1Conn.close();
-                } catch (Throwable ignore) {}
-            }
-            if (tenant2Conn != null) {
-                try {
-                    tenant2Conn.close();
-                } catch (Throwable ignore) {}
-            }
-        }
-
-    }
-    
-    private boolean checkColumnPartOfPk(PhoenixConnection conn, String columnName, String tableName) throws SQLException {
-        String normalizedTableName = SchemaUtil.normalizeIdentifier(tableName);
-        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), normalizedTableName));
-        List<PColumn> pkCols = table.getPKColumns();
-        String normalizedColumnName = SchemaUtil.normalizeIdentifier(columnName);
-        for (PColumn pkCol : pkCols) {
-            if (pkCol.getName().getString().equals(normalizedColumnName)) {
-                return true;
-            }
-        }
-        return false;
-    }
-    
-    private int getIndexOfPkColumn(PhoenixConnection conn, String columnName, String tableName) throws SQLException {
-        String normalizedTableName = SchemaUtil.normalizeIdentifier(tableName);
-        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), normalizedTableName));
-        List<PColumn> pkCols = table.getPKColumns();
-        String normalizedColumnName = SchemaUtil.normalizeIdentifier(columnName);
-        int i = 0;
-        for (PColumn pkCol : pkCols) {
-            if (pkCol.getName().getString().equals(normalizedColumnName)) {
-                return i;
-            }
-            i++;
-        }
-        return -1;
-    }
-    
-    @Test
-    public void testAddPKColumnToBaseTableWhoseViewsHaveIndices() throws Exception {
-        String baseTable = "testAddPKColumnToBaseTableWhoseViewsHaveIndices";
-        String view1 = "view1";
-        String view2 = "view2";
-        String view3 = "view3";
-        String tenant1 = "tenant1";
-        String tenant2 = "tenant2";
-        String view2Index = view2 + "_idx";
-        String view3Index = view3 + "_idx";
-        /*                          baseTable(mutli-tenant)
-                                 /                           \                
-                         view1(tenant1)                  view3(tenant2, index) 
-                          /
-                        view2(tenant1, index)  
-         */
-        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-            // make sure that the tables are empty, but reachable
-            globalConn
-            .createStatement()
-            .execute(
-                    "CREATE TABLE "
-                            + baseTable
-                            + " (TENANT_ID VARCHAR NOT NULL, K1 varchar not null, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, K1)) MULTI_TENANT = true ");
-
-        }
-        try (Connection viewConn = getTenantConnection(tenant1)) {
-            // create tenant specific view for tenant1 - view1
-            viewConn.createStatement().execute("CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable);
-            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
-            assertEquals(0, getTableSequenceNumber(phxConn, view1));
-            assertEquals(2, getMaxKeySequenceNumber(phxConn, view1));
-
-            // create a view - view2 on view - view1
-            viewConn.createStatement().execute("CREATE VIEW " + view2 + " AS SELECT * FROM " + view1);
-            assertEquals(0, getTableSequenceNumber(phxConn, view2));
-            assertEquals(2, getMaxKeySequenceNumber(phxConn, view2));
-
-
-            // create an index on view2
-            viewConn.createStatement().execute("CREATE INDEX " + view2Index + " ON " + view2 + " (v1) include (v2)");
-            assertEquals(0, getTableSequenceNumber(phxConn, view2Index));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view2Index));
-        }
-        try (Connection viewConn = getTenantConnection(tenant2)) {
-            // create tenant specific view for tenant2 - view3
-            viewConn.createStatement().execute("CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable);
-            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
-            assertEquals(0, getTableSequenceNumber(phxConn, view3));
-            assertEquals(2, getMaxKeySequenceNumber(phxConn, view3));
-
-
-            // create an index on view3
-            viewConn.createStatement().execute("CREATE INDEX " + view3Index + " ON " + view3 + " (v1) include (v2)");
-            assertEquals(0, getTableSequenceNumber(phxConn, view3Index));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view3Index));
-
-
-        }
-
-        // alter the base table by adding 1 non-pk and 2 pk columns
-        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-            globalConn.createStatement().execute("ALTER TABLE " + baseTable + " ADD v3 VARCHAR, k2 VARCHAR PRIMARY KEY, k3 VARCHAR PRIMARY KEY");
-            assertEquals(4, getMaxKeySequenceNumber(globalConn.unwrap(PhoenixConnection.class), baseTable));
-
-            // Upsert records in the base table
-            String upsert = "UPSERT INTO " + baseTable + " (TENANT_ID, K1, K2, K3, V1, V2, V3) VALUES (?, ?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = globalConn.prepareStatement(upsert);
-            stmt.setString(1, tenant1);
-            stmt.setString(2, "K1");
-            stmt.setString(3, "K2");
-            stmt.setString(4, "K3");
-            stmt.setString(5, "V1");
-            stmt.setString(6, "V2");
-            stmt.setString(7, "V3");
-            stmt.executeUpdate();
-            stmt.setString(1, tenant2);
-            stmt.setString(2, "K11");
-            stmt.setString(3, "K22");
-            stmt.setString(4, "K33");
-            stmt.setString(5, "V11");
-            stmt.setString(6, "V22");
-            stmt.setString(7, "V33");
-            stmt.executeUpdate();
-            globalConn.commit();
-        }
-
-        // Verify now that the sequence number of data table, indexes and views have changed.
-        // Also verify that the newly added pk columns show up as pk columns of data table, indexes and views.
-        try (Connection viewConn = getTenantConnection(tenant1)) {
-
-            ResultSet rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view1);
-            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
-            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view1));
-            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view1));
-            assertEquals(1, getTableSequenceNumber(phxConn, view1));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view1));
-            verifyNewColumns(rs, "K2", "K3", "V3");
-
-
-            rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view2);
-            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view2));
-            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view2));
-            assertEquals(1, getTableSequenceNumber(phxConn, view2));
-            assertEquals(4, getMaxKeySequenceNumber(phxConn, view2));
-            verifyNewColumns(rs, "K2", "K3", "V3");
-
-            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view2Index));
-            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view2Index));
-            assertEquals(1, getTableSequenceNumber(phxConn, view2Index));
-            assertEquals(6, getMaxKeySequenceNumber(phxConn, view2Index));
-        }
-        try (Connection viewConn = getTenantConnection(tenant2)) {
-            ResultSet rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view3);
-            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
-            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view3));
-            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view3));
-            assertEquals(1, getTableSequenceNumber(phxConn, view3));
-            verifyNewColumns(rs, "K22", "K33", "V33");
-
-            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view3Index));
-            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view3Index));
-            assertEquals(1, getTableSequenceNumber(phxConn, view3Index));
-            assertEquals(6, getMaxKeySequenceNumber(phxConn, view3Index));
-        }
-        // Verify that the index is actually being used when using newly added pk col
-        try (Connection viewConn = getTenantConnection(tenant1)) {
-            String upsert = "UPSERT INTO " + view2 + " (K1, K2, K3, V1, V2, V3) VALUES ('key1', 'key2', 'key3', 'value1', 'value2', 'value3')";
-            viewConn.createStatement().executeUpdate(upsert);
-            viewConn.commit();
-            Statement stmt = viewConn.createStatement();
-            String sql = "SELECT V2 FROM " + view2 + " WHERE V1 = 'value1' AND K3 = 'key3'";
-            QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
-            assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(view2Index)));
-            ResultSet rs = viewConn.createStatement().executeQuery(sql);
-            verifyNewColumns(rs, "value2");
-        }
-
-    }
-    
-    @Test
-    public void testAddingPkAndKeyValueColumnsToBaseTableWithDivergedView() throws Exception {
-        String baseTable = "testAlteringPkOfBaseTableWithDivergedView".toUpperCase();
-        String view1 = "view1".toUpperCase();
-        String divergedView = "divergedView".toUpperCase();
-        String divergedViewIndex = divergedView + "_IDX";
-        /*                                     baseTable
-                                 /                  |                
-                         view1(tenant1)         divergedView(tenant2)    
-                            
-        */
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            conn.createStatement().execute(baseTableDDL);
-            
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
-            
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                String divergedViewDDL = "CREATE VIEW " + divergedView + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant2Conn.createStatement().execute(divergedViewDDL);
-                // Drop column V2 from the view to have it diverge from the base table
-                tenant2Conn.createStatement().execute("ALTER VIEW " + divergedView + " DROP COLUMN V2");
-                
-                // create an index on the diverged view
-                String indexDDL = "CREATE INDEX " + divergedViewIndex + " ON " + divergedView + " (V1) include (V3)";
-                tenant2Conn.createStatement().execute(indexDDL);
-            }
-            
-            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
-            conn.createStatement().execute(alterBaseTable);
-            
-            
-            // verify that the both columns were added to view1
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-            }
-            
-            // verify that only the primary key column PK2 was added to diverged view
-            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
-                tenant2Conn.createStatement().execute("SELECT PK2 from " + divergedView);
-                try {
-                    tenant2Conn.createStatement().execute("SELECT KV FROM " + divergedView);
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
-            }
-            
-            // Upsert records in diverged view. Verify that the PK column was added to the index on it.
-            String upsert = "UPSERT INTO " + divergedView + " (PK1, PK2, V1, V3) VALUES ('PK1', 'PK2', 'V1', 'V3')";
-            try (Connection viewConn = getTenantConnection("tenant2")) {
-                viewConn.createStatement().executeUpdate(upsert);
-                viewConn.commit();
-                Statement stmt = viewConn.createStatement();
-                String sql = "SELECT V3 FROM " + divergedView + " WHERE V1 = 'V1' AND PK2 = 'PK2'";
-                QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
-                assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(divergedViewIndex)));
-                ResultSet rs = viewConn.createStatement().executeQuery(sql);
-                verifyNewColumns(rs, "V3");
-            }
-            
-            // For non-diverged view, base table columns will be added at the same position as base table
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
-            // For a diverged view, only base table's pk column will be added and that too at the end.
-            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
-            
-            // Add existing column VIEW_COL2 to the base table
-            alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
-            conn.createStatement().execute(alterBaseTable);
-            
-            // For the non-diverged view, adding the column VIEW_COL2 will end up changing its ordinal position in the view.
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", "VIEW_COL1");
-            // For the diverged view, adding the column VIEW_COL2 will not change its ordinal position in the view. It also won't change the base column count or the sequence number
-            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
-        }
-    }
-    
-    @Test
-    public void testAddColumnsToSaltedBaseTableWithViews() throws Exception {
-        String baseTable = "testAddColumnsToSaltedBaseTableWithViews".toUpperCase();
-        String view1 = "view1".toUpperCase();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            conn.createStatement().execute(baseTableDDL);
-
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
-
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
-
-            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
-            conn.createStatement().execute(alterBaseTable);
-
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
-
-            // verify that the both columns were added to view1
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-            }
-        }
-    }
-    
-    @Test
-    public void testDropColumnsFromSaltedBaseTableWithViews() throws Exception {
-        String baseTable = "testDropColumnsFromSaltedBaseTableWithViews".toUpperCase();
-        String view1 = "view1".toUpperCase();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            conn.createStatement().execute(baseTableDDL);
-
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-            }
-
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
-
-            String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
-            conn.createStatement().execute(alterBaseTable);
-
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
-
-            // verify that the dropped columns aren't visible
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                try {
-                    tenant1Conn.createStatement().execute("SELECT KV from " + view1);
-                    fail();
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
-                try {
-                    tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
-                    fail();
-                } catch (SQLException e) {
-                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
-                }
-            }
-        }
-    }
-    
-    @Test
-    public void testAlteringViewConditionallyModifiesHTableMetadata() throws Exception {
-        String baseTable = "testAlteringViewConditionallyModifiesBaseTable".toUpperCase();
-        String view1 = "view1".toUpperCase();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
-            conn.createStatement().execute(baseTableDDL);
-            HTableDescriptor tableDesc1 = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable)); 
-            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
-                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
-                tenant1Conn.createStatement().execute(view1DDL);
-                // This should not modify the base table
-                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR";
-                tenant1Conn.createStatement().execute(alterView);
-                HTableDescriptor tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
-                assertEquals(tableDesc1, tableDesc2);
-                
-                // Add a new column family that doesn't already exist in the base table
-                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL2 VARCHAR";
-                tenant1Conn.createStatement().execute(alterView);
-                
-                // Verify that the column family now shows up in the base table descriptor
-                tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
-                assertFalse(tableDesc2.equals(tableDesc1));
-                assertNotNull(tableDesc2.getFamily(Bytes.toBytes("CF")));
-                
-                // Add a column with an existing column family. This shouldn't modify the base table.
-                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL3 VARCHAR";
-                tenant1Conn.createStatement().execute(alterView);
-                HTableDescriptor tableDesc3 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
-                assertTrue(tableDesc3.equals(tableDesc2));
-                assertNotNull(tableDesc3.getFamily(Bytes.toBytes("CF")));
-            }
-        }
-    }
-    
-    @Test
-    public void testCacheInvalidatedAfterAddingColumnToBaseTableWithViews() throws Exception {
-        String baseTable = "testCacheInvalidatedAfterAddingColumnToBaseTableWithViews";
-        String viewName = baseTable + "_view";
-        String tenantId = "tenantId";
-        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-            String tableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true" ;
-            globalConn.createStatement().execute(tableDDL);
-            Properties tenantProps = new Properties();
-            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-            // create a tenant specific view
-            try (Connection tenantConn =  DriverManager.getConnection(getUrl(), tenantProps)) {
-                String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTable;
-                tenantConn.createStatement().execute(viewDDL);
-                
-                // Add a column to the base table using global connection
-                globalConn.createStatement().execute("ALTER TABLE " + baseTable + " ADD NEW_COL VARCHAR");
-
-                // Check now whether the tenant connection can see the column that was added
-                tenantConn.createStatement().execute("SELECT NEW_COL FROM " + viewName);
-                tenantConn.createStatement().execute("SELECT NEW_COL FROM " + baseTable);
-            }
-        }
-    }
-    
-    @Test
-    public void testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews() throws Exception {
-        String baseTable = "testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews";
-        String viewName = baseTable + "_view";
-        String tenantId = "tenantId";
-        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
-            String tableDDL =
-                    "CREATE TABLE "
-                            + baseTable
-                            + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true" ;
-            globalConn.createStatement().execute(tableDDL);
-            Properties tenantProps = new Properties();
-            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-            // create a tenant specific view
-            try (Connection tenantConn =  DriverManager.getConnection(getUrl(), tenantProps)) {
-                String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTable;
-                tenantConn.createStatement().execute(viewDDL);
-    
-                // Add a column to the base table using global connection
-                globalConn.createStatement()
-                        .execute("ALTER TABLE " + baseTable + " DROP COLUMN V1");
-    
-                // Check now whether the tenant connection can see the column that was dropped
-                try {
-                    tenantConn.createStatement().execute("SELECT V1 FROM " + viewName);
-                    fail();
-                } catch (ColumnNotFoundException e) {
-                }
-                try {
-                    tenantConn.createStatement().execute("SELECT V1 FROM " + baseTable);
-                    fail();
-                } catch (ColumnNotFoundException e) {
-                }
-            }
-        }
-    }
-
-    public static void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
-        PreparedStatement p = conn.prepareStatement("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME=? AND TABLE_TYPE=?");
-        p.setString(1, tableName);
-        p.setString(2, tableType.getSerializedValue());
-        ResultSet rs = p.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in BaseColumnCount"), baseColumnCount, rs.getInt("BASE_COLUMN_COUNT"));
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnCount"), columnCount, rs.getInt("COLUMN_COUNT"));
-        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in sequenceNumber"), sequenceNumber, rs.getInt("TABLE_SEQ_NUM"));
-        rs.close();
-    
-        ResultSet parentTableColumnsRs = null; 
-        if (parentTableName != null) {
-            parentTableColumnsRs = conn.getMetaData().getColumns(null, null, parentTableName, null);
-            parentTableColumnsRs.next();
-        }
-        
-        ResultSet viewColumnsRs = conn.getMetaData().getColumns(null, null, tableName, null);
-        for (int i = 0; i < columnName.length; i++) {
-            if (columnName[i] != null) {
-                assertTrue(viewColumnsRs.next());
-                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnName: i=" + i), columnName[i], viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME));
-                int viewColOrdinalPos = viewColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
-                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition: i=" + i), i+1, viewColOrdinalPos);
-                // validate that all the columns in the base table are present in the view   
-                if (parentTableColumnsRs != null && !parentTableColumnsRs.isAfterLast()) {
-                    ResultSetMetaData parentTableColumnsMetadata = parentTableColumnsRs.getMetaData();
-                    assertEquals(parentTableColumnsMetadata.getColumnCount(), viewColumnsRs.getMetaData().getColumnCount());
-                    int parentTableColOrdinalRs = parentTableColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
-                    assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition of view and base table for i=" + i), parentTableColOrdinalRs, viewColOrdinalPos);
-                    for (int columnIndex = 1; columnIndex < parentTableColumnsMetadata.getColumnCount(); columnIndex++) {
-                        String viewColumnValue = viewColumnsRs.getString(columnIndex);
-                        String parentTableColumnValue = parentTableColumnsRs.getString(columnIndex);
-                        if (!Objects.equal(viewColumnValue, parentTableColumnValue)) {
-                            if (parentTableColumnsMetadata.getColumnName(columnIndex).equals(PhoenixDatabaseMetaData.TABLE_NAME)) {
-                                assertEquals(parentTableName, parentTableColumnValue);
-                                assertEquals(tableName, viewColumnValue);
-                            } 
-                        }
-                    }
-                    parentTableColumnsRs.next();
-                }
-            }
-        }
-        assertFalse(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, ""), viewColumnsRs.next());
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
new file mode 100644
index 0000000..8ec93bc
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -0,0 +1,704 @@
+/*
+ * 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 static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.IndexUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+import com.google.common.base.Objects;
+
+public class AlterMultiTenantTableWithViewsIT extends BaseHBaseManagedTimeTableReuseIT {
+
+    private Connection getTenantConnection(String tenantId) throws Exception {
+        Properties tenantProps = new Properties();
+        tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        return DriverManager.getConnection(getUrl(), tenantProps);
+    }
+    
+    private static long getTableSequenceNumber(PhoenixConnection conn, String tableName) throws SQLException {
+        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), SchemaUtil.normalizeIdentifier(tableName)));
+        return table.getSequenceNumber();
+    }
+    
+    private static short getMaxKeySequenceNumber(PhoenixConnection conn, String tableName) throws SQLException {
+        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), SchemaUtil.normalizeIdentifier(tableName)));
+        return SchemaUtil.getMaxKeySeq(table);
+    }
+    
+    private static void verifyNewColumns(ResultSet rs, String ... values) throws SQLException {
+        assertTrue(rs.next());
+        int i = 1;
+        for (String value : values) {
+            assertEquals(value, rs.getString(i++));
+        }
+        assertFalse(rs.next());
+        assertEquals(values.length, i - 1);
+    }
+    
+    @Test
+    public void testAddDropColumnToBaseTablePropagatesToEntireViewHierarchy() throws Exception {
+        String baseTable = "testViewHierarchy";
+        String baseViewName = generateRandomString();
+        String view1 = baseViewName + "_VIEW1";
+        String view2 = baseViewName + "_VIEW2";
+        String view3 = baseViewName + "_VIEW3";
+        String view4 = baseViewName + "_VIEW4";
+        /*                                     baseTable
+                                 /                  |               \ 
+                         view1(tenant1)    view3(tenant2)          view4(global)
+                          /
+                        view2(tenant1)  
+        */
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+            
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+                
+                String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1;
+                tenant1Conn.createStatement().execute(view2DDL);
+            }
+            
+            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+                String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
+                tenant2Conn.createStatement().execute(view3DDL);
+            }
+            
+            String view4DDL = "CREATE VIEW " + view4 + " AS SELECT * FROM " + baseTable;
+            conn.createStatement().execute(view4DDL);
+            
+            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD V3 VARCHAR";
+            conn.createStatement().execute(alterBaseTable);
+            
+            // verify that the column is visible to view4
+            conn.createStatement().execute("SELECT V3 FROM " + view4);
+            
+            // verify that the column is visible to view1 and view2
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                tenant1Conn.createStatement().execute("SELECT V3 from " + view1);
+                tenant1Conn.createStatement().execute("SELECT V3 from " + view2);
+            }
+            
+            // verify that the column is visible to view3
+            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+                tenant2Conn.createStatement().execute("SELECT V3 from " + view3);
+            }
+
+            alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V1";
+            conn.createStatement().execute(alterBaseTable);
+
+            // verify that the column is not visible to view4
+            try {
+                conn.createStatement().execute("SELECT V1 FROM " + view4);
+                fail();
+            } catch (ColumnNotFoundException e) {
+            }
+            // verify that the column is not visible to view1 and view2
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                try {
+                    tenant1Conn.createStatement().execute("SELECT V1 from " + view1);
+                    fail();
+                } catch (ColumnNotFoundException e) {
+                }
+                try {
+                    tenant1Conn.createStatement().execute("SELECT V1 from " + view2);
+                    fail();
+                } catch (ColumnNotFoundException e) {
+                }
+            }
+
+            // verify that the column is not visible to view3
+            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+                try {
+                    tenant2Conn.createStatement().execute("SELECT V1 from " + view3);
+                    fail();
+                } catch (ColumnNotFoundException e) {
+                }
+            }
+        }
+           
+    }
+    
+    @Test
+    public void testChangingPKOfBaseTableChangesPKForAllViews() throws Exception {
+        String baseTable = "testChangePKOfBaseTable";
+        String baseViewName = generateRandomString();
+        String view1 = baseViewName + "_VIEW1";
+        String view2 = baseViewName + "_VIEW2";
+        String view3 = baseViewName + "_VIEW3";
+        String view4 = baseViewName + "_VIEW4";
+        /*                                     baseTable
+                                 /                  |               \ 
+                         view1(tenant1)    view3(tenant2)          view4(global)
+                          /
+                        view2(tenant1)  
+         */
+        Connection tenant1Conn = null, tenant2Conn = null;
+        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE "
+                    + baseTable
+                    + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            globalConn.createStatement().execute(baseTableDDL);
+
+            tenant1Conn = getTenantConnection("tenant1");
+            String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable;
+            tenant1Conn.createStatement().execute(view1DDL);
+
+            String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1;
+            tenant1Conn.createStatement().execute(view2DDL);
+
+            tenant2Conn = getTenantConnection("tenant2");
+            String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable;
+            tenant2Conn.createStatement().execute(view3DDL);
+
+            String view4DDL = "CREATE VIEW " + view4 + " AS SELECT * FROM " + baseTable;
+            globalConn.createStatement().execute(view4DDL);
+
+            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD NEW_PK varchar primary key ";
+            globalConn.createStatement().execute(alterBaseTable);
+            
+            // verify that the new column new_pk is now part of the primary key for the entire hierarchy
+            
+            globalConn.createStatement().execute("SELECT * FROM " + baseTable);
+            assertTrue(checkColumnPartOfPk(globalConn.unwrap(PhoenixConnection.class), "NEW_PK", baseTable));
+            
+            tenant1Conn.createStatement().execute("SELECT * FROM " + view1);
+            assertTrue(checkColumnPartOfPk(tenant1Conn.unwrap(PhoenixConnection.class), "NEW_PK", view1));
+            
+            tenant1Conn.createStatement().execute("SELECT * FROM " + view2);
+            assertTrue(checkColumnPartOfPk(tenant1Conn.unwrap(PhoenixConnection.class), "NEW_PK", view2));
+            
+            tenant2Conn.createStatement().execute("SELECT * FROM " + view3);
+            assertTrue(checkColumnPartOfPk(tenant2Conn.unwrap(PhoenixConnection.class), "NEW_PK", view3));
+            
+            globalConn.createStatement().execute("SELECT * FROM " + view4);
+            assertTrue(checkColumnPartOfPk(globalConn.unwrap(PhoenixConnection.class), "NEW_PK", view4));
+
+        } finally {
+            if (tenant1Conn != null) {
+                try {
+                    tenant1Conn.close();
+                } catch (Throwable ignore) {}
+            }
+            if (tenant2Conn != null) {
+                try {
+                    tenant2Conn.close();
+                } catch (Throwable ignore) {}
+            }
+        }
+
+    }
+    
+    private boolean checkColumnPartOfPk(PhoenixConnection conn, String columnName, String tableName) throws SQLException {
+        String normalizedTableName = SchemaUtil.normalizeIdentifier(tableName);
+        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), normalizedTableName));
+        List<PColumn> pkCols = table.getPKColumns();
+        String normalizedColumnName = SchemaUtil.normalizeIdentifier(columnName);
+        for (PColumn pkCol : pkCols) {
+            if (pkCol.getName().getString().equals(normalizedColumnName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+    
+    private int getIndexOfPkColumn(PhoenixConnection conn, String columnName, String tableName) throws SQLException {
+        String normalizedTableName = SchemaUtil.normalizeIdentifier(tableName);
+        PTable table = conn.getTable(new PTableKey(conn.getTenantId(), normalizedTableName));
+        List<PColumn> pkCols = table.getPKColumns();
+        String normalizedColumnName = SchemaUtil.normalizeIdentifier(columnName);
+        int i = 0;
+        for (PColumn pkCol : pkCols) {
+            if (pkCol.getName().getString().equals(normalizedColumnName)) {
+                return i;
+            }
+            i++;
+        }
+        return -1;
+    }
+    
+    @Test
+    public void testAddPKColumnToBaseTableWhoseViewsHaveIndices() throws Exception {
+        String baseTable = "testAddPKColumnToBaseTableWhoseViewsHaveIndices";
+        String baseViewName = generateRandomString();
+        String view1 = baseViewName + "_VIEW1";
+        String view2 = baseViewName + "_VIEW2";
+        String view3 = baseViewName + "_VIEW3";
+        String tenant1 = baseViewName + "_T1";
+        String tenant2 = baseViewName + "_T2";
+        String view2Index = view2 + "_IDX";
+        String view3Index = view3 + "_IDX";
+        /*                          baseTable(mutli-tenant)
+                                 /                           \                
+                         view1(tenant1)                  view3(tenant2, index) 
+                          /
+                        view2(tenant1, index)  
+         */
+        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
+            // make sure that the tables are empty, but reachable
+            globalConn
+            .createStatement()
+            .execute(
+                    "CREATE TABLE "
+                            + baseTable
+                            + " (TENANT_ID VARCHAR NOT NULL, K1 varchar not null, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, K1)) MULTI_TENANT = true ");
+
+        }
+        try (Connection viewConn = getTenantConnection(tenant1)) {
+            // create tenant specific view for tenant1 - view1
+            viewConn.createStatement().execute("CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable);
+            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
+            assertEquals(0, getTableSequenceNumber(phxConn, view1));
+            assertEquals(2, getMaxKeySequenceNumber(phxConn, view1));
+
+            // create a view - view2 on view - view1
+            viewConn.createStatement().execute("CREATE VIEW " + view2 + " AS SELECT * FROM " + view1);
+            assertEquals(0, getTableSequenceNumber(phxConn, view2));
+            assertEquals(2, getMaxKeySequenceNumber(phxConn, view2));
+
+
+            // create an index on view2
+            viewConn.createStatement().execute("CREATE INDEX " + view2Index + " ON " + view2 + " (v1) include (v2)");
+            assertEquals(0, getTableSequenceNumber(phxConn, view2Index));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, view2Index));
+        }
+        try (Connection viewConn = getTenantConnection(tenant2)) {
+            // create tenant specific view for tenant2 - view3
+            viewConn.createStatement().execute("CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable);
+            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
+            assertEquals(0, getTableSequenceNumber(phxConn, view3));
+            assertEquals(2, getMaxKeySequenceNumber(phxConn, view3));
+
+
+            // create an index on view3
+            viewConn.createStatement().execute("CREATE INDEX " + view3Index + " ON " + view3 + " (v1) include (v2)");
+            assertEquals(0, getTableSequenceNumber(phxConn, view3Index));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, view3Index));
+
+
+        }
+
+        // alter the base table by adding 1 non-pk and 2 pk columns
+        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
+            globalConn.createStatement().execute("ALTER TABLE " + baseTable + " ADD v3 VARCHAR, k2 VARCHAR PRIMARY KEY, k3 VARCHAR PRIMARY KEY");
+            assertEquals(4, getMaxKeySequenceNumber(globalConn.unwrap(PhoenixConnection.class), baseTable));
+
+            // Upsert records in the base table
+            String upsert = "UPSERT INTO " + baseTable + " (TENANT_ID, K1, K2, K3, V1, V2, V3) VALUES (?, ?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = globalConn.prepareStatement(upsert);
+            stmt.setString(1, tenant1);
+            stmt.setString(2, "K1");
+            stmt.setString(3, "K2");
+            stmt.setString(4, "K3");
+            stmt.setString(5, "V1");
+            stmt.setString(6, "V2");
+            stmt.setString(7, "V3");
+            stmt.executeUpdate();
+            stmt.setString(1, tenant2);
+            stmt.setString(2, "K11");
+            stmt.setString(3, "K22");
+            stmt.setString(4, "K33");
+            stmt.setString(5, "V11");
+            stmt.setString(6, "V22");
+            stmt.setString(7, "V33");
+            stmt.executeUpdate();
+            globalConn.commit();
+        }
+
+        // Verify now that the sequence number of data table, indexes and views have changed.
+        // Also verify that the newly added pk columns show up as pk columns of data table, indexes and views.
+        try (Connection viewConn = getTenantConnection(tenant1)) {
+
+            ResultSet rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view1);
+            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view1));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view1));
+            assertEquals(1, getTableSequenceNumber(phxConn, view1));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, view1));
+            verifyNewColumns(rs, "K2", "K3", "V3");
+
+
+            rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view2);
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view2));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view2));
+            assertEquals(1, getTableSequenceNumber(phxConn, view2));
+            assertEquals(4, getMaxKeySequenceNumber(phxConn, view2));
+            verifyNewColumns(rs, "K2", "K3", "V3");
+
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view2Index));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view2Index));
+            assertEquals(1, getTableSequenceNumber(phxConn, view2Index));
+            assertEquals(6, getMaxKeySequenceNumber(phxConn, view2Index));
+        }
+        try (Connection viewConn = getTenantConnection(tenant2)) {
+            ResultSet rs = viewConn.createStatement().executeQuery("SELECT K2, K3, V3 FROM " + view3);
+            PhoenixConnection phxConn = viewConn.unwrap(PhoenixConnection.class);
+            assertEquals(2, getIndexOfPkColumn(phxConn, "k2", view3));
+            assertEquals(3, getIndexOfPkColumn(phxConn, "k3", view3));
+            assertEquals(1, getTableSequenceNumber(phxConn, view3));
+            verifyNewColumns(rs, "K22", "K33", "V33");
+
+            assertEquals(4, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k2"), view3Index));
+            assertEquals(5, getIndexOfPkColumn(phxConn, IndexUtil.getIndexColumnName(null, "k3"), view3Index));
+            assertEquals(1, getTableSequenceNumber(phxConn, view3Index));
+            assertEquals(6, getMaxKeySequenceNumber(phxConn, view3Index));
+        }
+        // Verify that the index is actually being used when using newly added pk col
+        try (Connection viewConn = getTenantConnection(tenant1)) {
+            String upsert = "UPSERT INTO " + view2 + " (K1, K2, K3, V1, V2, V3) VALUES ('key1', 'key2', 'key3', 'value1', 'value2', 'value3')";
+            viewConn.createStatement().executeUpdate(upsert);
+            viewConn.commit();
+            Statement stmt = viewConn.createStatement();
+            String sql = "SELECT V2 FROM " + view2 + " WHERE V1 = 'value1' AND K3 = 'key3'";
+            QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
+            assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(view2Index)));
+            ResultSet rs = viewConn.createStatement().executeQuery(sql);
+            verifyNewColumns(rs, "value2");
+        }
+
+    }
+    
+    @Test
+    public void testAddingPkAndKeyValueColumnsToBaseTableWithDivergedView() throws Exception {
+        String baseTable = "testAlteringPkOfBaseTableWithDivergedView".toUpperCase();
+        String view1 = generateRandomString();
+        String divergedView = generateRandomString();
+        String divergedViewIndex = divergedView + "_IDX";
+        /*                                     baseTable
+                                 /                  |                
+                         view1(tenant1)         divergedView(tenant2)    
+                            
+        */
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+            
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+            }
+            
+            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+                String divergedViewDDL = "CREATE VIEW " + divergedView + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant2Conn.createStatement().execute(divergedViewDDL);
+                // Drop column V2 from the view to have it diverge from the base table
+                tenant2Conn.createStatement().execute("ALTER VIEW " + divergedView + " DROP COLUMN V2");
+                
+                // create an index on the diverged view
+                String indexDDL = "CREATE INDEX " + divergedViewIndex + " ON " + divergedView + " (V1) include (V3)";
+                tenant2Conn.createStatement().execute(indexDDL);
+            }
+            
+            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
+            conn.createStatement().execute(alterBaseTable);
+            
+            
+            // verify that the both columns were added to view1
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
+                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
+            }
+            
+            // verify that only the primary key column PK2 was added to diverged view
+            try (Connection tenant2Conn = getTenantConnection("tenant2")) {
+                tenant2Conn.createStatement().execute("SELECT PK2 from " + divergedView);
+                try {
+                    tenant2Conn.createStatement().execute("SELECT KV FROM " + divergedView);
+                } catch (SQLException e) {
+                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+                }
+            }
+            
+            // Upsert records in diverged view. Verify that the PK column was added to the index on it.
+            String upsert = "UPSERT INTO " + divergedView + " (PK1, PK2, V1, V3) VALUES ('PK1', 'PK2', 'V1', 'V3')";
+            try (Connection viewConn = getTenantConnection("tenant2")) {
+                viewConn.createStatement().executeUpdate(upsert);
+                viewConn.commit();
+                Statement stmt = viewConn.createStatement();
+                String sql = "SELECT V3 FROM " + divergedView + " WHERE V1 = 'V1' AND PK2 = 'PK2'";
+                QueryPlan plan = stmt.unwrap(PhoenixStatement.class).optimizeQuery(sql);
+                assertTrue(plan.getTableRef().getTable().getName().getString().equals(SchemaUtil.normalizeIdentifier(divergedViewIndex)));
+                ResultSet rs = viewConn.createStatement().executeQuery(sql);
+                verifyNewColumns(rs, "V3");
+            }
+            
+            // For non-diverged view, base table columns will be added at the same position as base table
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
+            // For a diverged view, only base table's pk column will be added and that too at the end.
+            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
+            
+            // Add existing column VIEW_COL2 to the base table
+            alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
+            conn.createStatement().execute(alterBaseTable);
+            
+            // For the non-diverged view, adding the column VIEW_COL2 will end up changing its ordinal position in the view.
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", "VIEW_COL1");
+            // For the diverged view, adding the column VIEW_COL2 will not change its ordinal position in the view. It also won't change the base column count or the sequence number
+            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
+        }
+    }
+    
+    @Test
+    public void testAddColumnsToSaltedBaseTableWithViews() throws Exception {
+        String baseTable = "testAddColumnsToSaltedBaseTableWithViews".toUpperCase();
+        String view1 = generateRandomString();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+            }
+
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
+
+            String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
+            conn.createStatement().execute(alterBaseTable);
+
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
+
+            // verify that the both columns were added to view1
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                tenant1Conn.createStatement().execute("SELECT KV from " + view1);
+                tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
+            }
+        }
+    }
+    
+    @Test
+    public void testDropColumnsFromSaltedBaseTableWithViews() throws Exception {
+        String baseTable = "testDropColumnsFromSaltedBaseTableWithViews".toUpperCase();
+        String view1 = generateRandomString();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+            }
+
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
+
+            String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
+            conn.createStatement().execute(alterBaseTable);
+
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
+
+            // verify that the dropped columns aren't visible
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                try {
+                    tenant1Conn.createStatement().execute("SELECT KV from " + view1);
+                    fail();
+                } catch (SQLException e) {
+                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+                }
+                try {
+                    tenant1Conn.createStatement().execute("SELECT PK2 from " + view1);
+                    fail();
+                } catch (SQLException e) {
+                    assertEquals(SQLExceptionCode.COLUMN_NOT_FOUND.getErrorCode(), e.getErrorCode());
+                }
+            }
+        }
+    }
+    
+    @Test
+    public void testAlteringViewConditionallyModifiesHTableMetadata() throws Exception {
+        String baseTable = "testAlteringViewConditionallyModifiesBaseTable".toUpperCase();
+        String view1 = generateRandomString();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+            HTableDescriptor tableDesc1 = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable)); 
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+                // This should not modify the base table
+                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                HTableDescriptor tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertEquals(tableDesc1, tableDesc2);
+                
+                // Add a new column family that doesn't already exist in the base table
+                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL2 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                
+                // Verify that the column family now shows up in the base table descriptor
+                tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertFalse(tableDesc2.equals(tableDesc1));
+                assertNotNull(tableDesc2.getFamily(Bytes.toBytes("CF")));
+                
+                // Add a column with an existing column family. This shouldn't modify the base table.
+                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL3 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                HTableDescriptor tableDesc3 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertTrue(tableDesc3.equals(tableDesc2));
+                assertNotNull(tableDesc3.getFamily(Bytes.toBytes("CF")));
+            }
+        }
+    }
+    
+    @Test
+    public void testCacheInvalidatedAfterAddingColumnToBaseTableWithViews() throws Exception {
+        String baseTable = "testCacheInvalidatedAfterAddingColumnToBaseTableWithViews";
+        String viewName = baseTable + "_view";
+        String tenantId = "tenantId";
+        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
+            String tableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true" ;
+            globalConn.createStatement().execute(tableDDL);
+            Properties tenantProps = new Properties();
+            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+            // create a tenant specific view
+            try (Connection tenantConn =  DriverManager.getConnection(getUrl(), tenantProps)) {
+                String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTable;
+                tenantConn.createStatement().execute(viewDDL);
+                
+                // Add a column to the base table using global connection
+                globalConn.createStatement().execute("ALTER TABLE " + baseTable + " ADD NEW_COL VARCHAR");
+
+                // Check now whether the tenant connection can see the column that was added
+                tenantConn.createStatement().execute("SELECT NEW_COL FROM " + viewName);
+                tenantConn.createStatement().execute("SELECT NEW_COL FROM " + baseTable);
+            }
+        }
+    }
+    
+    @Test
+    public void testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews() throws Exception {
+        String baseTable = "testCacheInvalidatedAfterDroppingColumnFromBaseTableWithViews";
+        String viewName = baseTable + "_view";
+        String tenantId = "tenantId";
+        try (Connection globalConn = DriverManager.getConnection(getUrl())) {
+            String tableDDL =
+                    "CREATE TABLE "
+                            + baseTable
+                            + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true" ;
+            globalConn.createStatement().execute(tableDDL);
+            Properties tenantProps = new Properties();
+            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+            // create a tenant specific view
+            try (Connection tenantConn =  DriverManager.getConnection(getUrl(), tenantProps)) {
+                String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTable;
+                tenantConn.createStatement().execute(viewDDL);
+    
+                // Add a column to the base table using global connection
+                globalConn.createStatement()
+                        .execute("ALTER TABLE " + baseTable + " DROP COLUMN V1");
+    
+                // Check now whether the tenant connection can see the column that was dropped
+                try {
+                    tenantConn.createStatement().execute("SELECT V1 FROM " + viewName);
+                    fail();
+                } catch (ColumnNotFoundException e) {
+                }
+                try {
+                    tenantConn.createStatement().execute("SELECT V1 FROM " + baseTable);
+                    fail();
+                } catch (ColumnNotFoundException e) {
+                }
+            }
+        }
+    }
+
+    public static void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
+        PreparedStatement p = conn.prepareStatement("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME=? AND TABLE_TYPE=?");
+        p.setString(1, tableName);
+        p.setString(2, tableType.getSerializedValue());
+        ResultSet rs = p.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in BaseColumnCount"), baseColumnCount, rs.getInt("BASE_COLUMN_COUNT"));
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnCount"), columnCount, rs.getInt("COLUMN_COUNT"));
+        assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in sequenceNumber"), sequenceNumber, rs.getInt("TABLE_SEQ_NUM"));
+        rs.close();
+    
+        ResultSet parentTableColumnsRs = null; 
+        if (parentTableName != null) {
+            parentTableColumnsRs = conn.getMetaData().getColumns(null, null, parentTableName, null);
+            parentTableColumnsRs.next();
+        }
+        
+        ResultSet viewColumnsRs = conn.getMetaData().getColumns(null, null, tableName, null);
+        for (int i = 0; i < columnName.length; i++) {
+            if (columnName[i] != null) {
+                assertTrue(viewColumnsRs.next());
+                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in columnName: i=" + i), columnName[i], viewColumnsRs.getString(PhoenixDatabaseMetaData.COLUMN_NAME));
+                int viewColOrdinalPos = viewColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
+                assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition: i=" + i), i+1, viewColOrdinalPos);
+                // validate that all the columns in the base table are present in the view   
+                if (parentTableColumnsRs != null && !parentTableColumnsRs.isAfterLast()) {
+                    ResultSetMetaData parentTableColumnsMetadata = parentTableColumnsRs.getMetaData();
+                    assertEquals(parentTableColumnsMetadata.getColumnCount(), viewColumnsRs.getMetaData().getColumnCount());
+                    int parentTableColOrdinalRs = parentTableColumnsRs.getInt(PhoenixDatabaseMetaData.ORDINAL_POSITION);
+                    assertEquals(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, "Mismatch in ordinalPosition of view and base table for i=" + i), parentTableColOrdinalRs, viewColOrdinalPos);
+                    for (int columnIndex = 1; columnIndex < parentTableColumnsMetadata.getColumnCount(); columnIndex++) {
+                        String viewColumnValue = viewColumnsRs.getString(columnIndex);
+                        String parentTableColumnValue = parentTableColumnsRs.getString(columnIndex);
+                        if (!Objects.equal(viewColumnValue, parentTableColumnValue)) {
+                            if (parentTableColumnsMetadata.getColumnName(columnIndex).equals(PhoenixDatabaseMetaData.TABLE_NAME)) {
+                                assertEquals(parentTableName, parentTableColumnValue);
+                                assertEquals(tableName, viewColumnValue);
+                            } 
+                        }
+                    }
+                    parentTableColumnsRs.next();
+                }
+            }
+        }
+        assertFalse(AlterTableWithViewsIT.getSystemCatalogEntriesForTable(conn, tableName, ""), viewColumnsRs.next());
+    }
+}


[43/50] [abbrv] phoenix git commit: PHOENIX-1647 Correctly return that Phoenix supports schema name references in DatabaseMetaData (kliewkliew)

Posted by ma...@apache.org.
PHOENIX-1647 Correctly return that Phoenix supports schema name references in DatabaseMetaData (kliewkliew)


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

Branch: refs/heads/calcite
Commit: d873c2ffd1e539ecd56858c82f0ba2d23e877cf9
Parents: 14dab2f
Author: Mujtaba <mu...@apache.org>
Authored: Mon Aug 29 15:09:22 2016 -0700
Committer: Mujtaba <mu...@apache.org>
Committed: Mon Aug 29 15:09:22 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d873c2ff/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 c10bc9f..2fed40c 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
@@ -1534,7 +1534,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
 
     @Override
     public boolean supportsSchemasInDataManipulation() throws SQLException {
-        return false;
+        return true;
     }
 
     @Override


[21/50] [abbrv] phoenix git commit: PHOENIX-808 Create snapshot of SYSTEM.CATALOG prior to upgrade and restore on any failure

Posted by ma...@apache.org.
PHOENIX-808 Create snapshot of SYSTEM.CATALOG prior to upgrade and restore on any failure


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

Branch: refs/heads/calcite
Commit: bea9e03aebe0c3ef657f4a5b762f47cb7d8c81bb
Parents: 386cbbb
Author: Samarth <sa...@salesforce.com>
Authored: Mon Aug 22 10:35:38 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Aug 22 10:35:38 2016 -0700

----------------------------------------------------------------------
 .../phoenix/coprocessor/MetaDataProtocol.java   |  19 +
 .../query/ConnectionQueryServicesImpl.java      | 365 ++++++++++++-------
 .../org/apache/phoenix/util/UpgradeUtil.java    |  11 +
 3 files changed, 271 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bea9e03a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 8982fe7..dce89bd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -19,7 +19,9 @@ package org.apache.phoenix.coprocessor;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
@@ -27,6 +29,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.PFunctionProtos;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.schema.PColumn;
@@ -83,6 +86,22 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 = MIN_TABLE_TIMESTAMP + 18;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0;
+    
+    // ALWAYS update this map whenever rolling out a new release (major, minor or patch release). 
+    // Key is the SYSTEM.CATALOG timestamp for the version and value is the version string.
+    public static final Map<Long, String> TIMESTAMP_VERSION_MAP = new HashMap<>(10);
+    static {
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0, "4.1.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_2_0, "4.2.0");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1, "4.2.1");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, "4.3.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, "4.5.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, "4.6.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, "4.7.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0, "4.8.x");
+    }
+    public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER; 
+    
     // TODO: pare this down to minimum, as we don't need duplicates for both table and column errors, nor should we need
     // a different code for every type of error.
     // ENTITY_ALREADY_EXISTS, ENTITY_NOT_FOUND, NEWER_ENTITY_FOUND, ENTITY_NOT_IN_REGION, CONCURRENT_MODIFICATION

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bea9e03a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index a5dcca8..7a57103 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -18,10 +18,15 @@
 package org.apache.phoenix.query;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.hadoop.hbase.HColumnDescriptor.TTL;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MAJOR_VERSION;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MINOR_VERSION;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_PATCH_NUMBER;
+import static org.apache.phoenix.util.UpgradeUtil.getUpgradeSnapshotName;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_ENABLED;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE;
@@ -83,6 +88,8 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -2302,6 +2309,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                         checkClosed();
                         PhoenixConnection metaConnection = null;
+                        boolean success = false;
+                        String snapshotName = null;
+                        String sysCatalogTableName = null;
                         try {
                             openConnection();
                             String noUpgradeProp = props.getProperty(PhoenixRuntime.NO_UPGRADE_ATTRIB);
@@ -2332,148 +2342,150 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + " is found but client does not have "
                                             + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
                                             .build().buildException(); }
-                            }
-
+                            }   
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
-
                             } catch (NewerTableAlreadyExistsException ignore) {
                                 // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed timestamp.
                                 // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
                             } catch (TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
-                                    // This will occur if we have an older SYSTEM.CATALOG and we need to update it to include
-                                    // any new columns we've added.
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
-
-                                    String columnsToAdd = "";
-                                    if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
-                                        // We know that we always need to add the STORE_NULLS column for 4.3 release
-                                        columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.STORE_NULLS + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        try (HBaseAdmin admin = getAdmin()) {
-                                            HTableDescriptor[] localIndexTables = admin.listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
-                                            for (HTableDescriptor table : localIndexTables) {
-                                                if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
-                                                        && table.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME) != null) {
-                                                    table.setValue(MetaDataUtil.PARENT_TABLE_KEY,
-                                                        MetaDataUtil.getUserTableName(table
-                                                            .getNameAsString()));
-                                                    // Explicitly disable, modify and enable the table to ensure co-location of data
-                                                    // and index regions. If we just modify the table descriptor when online schema
-                                                    // change enabled may reopen the region in same region server instead of following data region.
-                                                    admin.disableTable(table.getTableName());
-                                                    admin.modifyTable(table.getTableName(), table);
-                                                    admin.enableTable(table.getTableName());
+                                    sysCatalogTableName = e.getTable().getPhysicalName().getString();
+                                    if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
+                                        snapshotName = getUpgradeSnapshotName(sysCatalogTableName, currentServerSideTableTimeStamp);
+                                        createSnapshot(snapshotName, sysCatalogTableName);
+                                    }
+                                        String columnsToAdd = "";
+                                        // This will occur if we have an older SYSTEM.CATALOG and we need to update it to include
+                                        // any new columns we've added.
+                                        if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
+                                            // We know that we always need to add the STORE_NULLS column for 4.3 release
+                                            columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.STORE_NULLS + " " + PBoolean.INSTANCE.getSqlTypeName());
+                                            try (HBaseAdmin admin = getAdmin()) {
+                                                HTableDescriptor[] localIndexTables = admin.listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
+                                                for (HTableDescriptor table : localIndexTables) {
+                                                    if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
+                                                            && table.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME) != null) {
+                                                        table.setValue(MetaDataUtil.PARENT_TABLE_KEY,
+                                                            MetaDataUtil.getUserTableName(table
+                                                                .getNameAsString()));
+                                                        // Explicitly disable, modify and enable the table to ensure co-location of data
+                                                        // and index regions. If we just modify the table descriptor when online schema
+                                                        // change enabled may reopen the region in same region server instead of following data region.
+                                                        admin.disableTable(table.getTableName());
+                                                        admin.modifyTable(table.getTableName(), table);
+                                                        admin.enableTable(table.getTableName());
+                                                    }
                                                 }
                                             }
                                         }
-                                    }
 
-                                    // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
-                                    if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
-                                        columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
-                                            + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
-                                    }
-
-                                    // If we have some new columns from 4.1-4.3 to add, add them now.
-                                    if (!columnsToAdd.isEmpty()) {
-                                        // Ugh..need to assign to another local variable to keep eclipse happy.
-                                        PhoenixConnection newMetaConnection = addColumnsIfNotExists(metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, columnsToAdd);
-                                        metaConnection = newMetaConnection;
-                                    }
+                                        // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
+                                        // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
+                                        // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
+                                        // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
+                                        // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
+                                            columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
+                                                + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
+                                        }
 
-                                    if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0) {
-                                        columnsToAdd = PhoenixDatabaseMetaData.BASE_COLUMN_COUNT + " "
-                                                + PInteger.INSTANCE.getSqlTypeName();
-                                        try {
-                                            metaConnection = addColumn(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
-                                            upgradeTo4_5_0(metaConnection);
-                                        } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
-                                             * Upgrade to 4.5 is a slightly special case. We use the fact that the column
-                                             * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
-                                             * the server side upgrade has finished or is in progress.
-                                             */
-                                            logger.debug("No need to run 4.5 upgrade");
+                                        // If we have some new columns from 4.1-4.3 to add, add them now.
+                                        if (!columnsToAdd.isEmpty()) {
+                                            // Ugh..need to assign to another local variable to keep eclipse happy.
+                                            PhoenixConnection newMetaConnection = addColumnsIfNotExists(metaConnection,
+                                                PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, columnsToAdd);
+                                            metaConnection = newMetaConnection;
                                         }
-                                        Properties props = PropertiesUtil.deepCopy(metaConnection.getClientInfo());
-                                        props.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
-                                        props.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
-                                        PhoenixConnection conn = new PhoenixConnection(ConnectionQueryServicesImpl.this, metaConnection.getURL(), props, metaConnection.getMetaDataCache());
-                                        try {
-                                            List<String> tablesNeedingUpgrade = UpgradeUtil.getPhysicalTablesWithDescRowKey(conn);
-                                            if (!tablesNeedingUpgrade.isEmpty()) {
-                                                logger.warn("The following tables require upgrade due to a bug causing the row key to be incorrect for descending columns and ascending BINARY columns (PHOENIX-2067 and PHOENIX-2120):\n" + Joiner.on(' ').join(tablesNeedingUpgrade) + "\nTo upgrade issue the \"bin/psql.py -u\" command.");
+
+                                        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0) {
+                                            columnsToAdd = PhoenixDatabaseMetaData.BASE_COLUMN_COUNT + " "
+                                                    + PInteger.INSTANCE.getSqlTypeName();
+                                            try {
+                                                metaConnection = addColumn(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                    MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
+                                                upgradeTo4_5_0(metaConnection);
+                                            } catch (ColumnAlreadyExistsException ignored) {
+                                                /* 
+                                                 * Upgrade to 4.5 is a slightly special case. We use the fact that the column
+                                                 * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
+                                                 * the server side upgrade has finished or is in progress.
+                                                 */
+                                                logger.debug("No need to run 4.5 upgrade");
                                             }
-                                            List<String> unsupportedTables = UpgradeUtil.getPhysicalTablesWithDescVarbinaryRowKey(conn);
-                                            if (!unsupportedTables.isEmpty()) {
-                                                logger.warn("The following tables use an unsupported VARBINARY DESC construct and need to be changed:\n" + Joiner.on(' ').join(unsupportedTables));
+                                            Properties props = PropertiesUtil.deepCopy(metaConnection.getClientInfo());
+                                            props.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
+                                            props.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
+                                            PhoenixConnection conn = new PhoenixConnection(ConnectionQueryServicesImpl.this, metaConnection.getURL(), props, metaConnection.getMetaDataCache());
+                                            try {
+                                                List<String> tablesNeedingUpgrade = UpgradeUtil.getPhysicalTablesWithDescRowKey(conn);
+                                                if (!tablesNeedingUpgrade.isEmpty()) {
+                                                    logger.warn("The following tables require upgrade due to a bug causing the row key to be incorrect for descending columns and ascending BINARY columns (PHOENIX-2067 and PHOENIX-2120):\n" + Joiner.on(' ').join(tablesNeedingUpgrade) + "\nTo upgrade issue the \"bin/psql.py -u\" command.");
+                                                }
+                                                List<String> unsupportedTables = UpgradeUtil.getPhysicalTablesWithDescVarbinaryRowKey(conn);
+                                                if (!unsupportedTables.isEmpty()) {
+                                                    logger.warn("The following tables use an unsupported VARBINARY DESC construct and need to be changed:\n" + Joiner.on(' ').join(unsupportedTables));
+                                                }
+                                            } catch (Exception ex) {
+                                                logger.error("Unable to determine tables requiring upgrade due to PHOENIX-2067", ex);
+                                            } finally {
+                                                conn.close();
                                             }
-                                        } catch (Exception ex) {
-                                            logger.error("Unable to determine tables requiring upgrade due to PHOENIX-2067", ex);
-                                        } finally {
-                                            conn.close();
                                         }
-                                    }
-                                    // Add these columns one at a time, each with different timestamps so that if folks have
-                                    // run the upgrade code already for a snapshot, we'll still enter this block (and do the
-                                    // parts we haven't yet done).
-                                    if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
-                                        columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " " + PBoolean.INSTANCE.getSqlTypeName();
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd);
-                                    }
-                                    if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
-                                        // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
-                                            PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
-                                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
-                                        ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
-                                        clearCache();
-                                    }
+                                        // Add these columns one at a time, each with different timestamps so that if folks have
+                                        // run the upgrade code already for a snapshot, we'll still enter this block (and do the
+                                        // parts we haven't yet done).
+                                        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
+                                            columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " " + PBoolean.INSTANCE.getSqlTypeName();
+                                            metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd);
+                                        }
+                                        if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
+                                            // Drop old stats table so that new stats table is created
+                                            metaConnection = dropStatsTable(metaConnection, 
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
+                                            metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
+                                                PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
+                                            metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
+                                                PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
+                                            metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+                                            metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+                                            ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+                                            clearCache();
+                                        }
 
-                                    if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
-                                        metaConnection = addColumnsIfNotExists(metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
-                                            PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED + " "
-                                                    + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 1,
-                                            PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ + " "
-                                                    + PVarchar.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0,
-                                            PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA + " "
-                                                    + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = UpgradeUtil.disableViewIndexes(metaConnection);
-                                        if(getProps().getBoolean(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)) {
-                                            metaConnection = UpgradeUtil.upgradeLocalIndexes(metaConnection);
+                                        if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
+                                            metaConnection = addColumnsIfNotExists(metaConnection,
+                                                PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
+                                                PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED + " "
+                                                        + PBoolean.INSTANCE.getSqlTypeName());
+                                            metaConnection = addColumnsIfNotExists(metaConnection,
+                                                PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 1,
+                                                PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ + " "
+                                                        + PVarchar.INSTANCE.getSqlTypeName());
+                                            metaConnection = addColumnsIfNotExists(metaConnection,
+                                                PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0,
+                                                PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA + " "
+                                                        + PBoolean.INSTANCE.getSqlTypeName());
+                                            metaConnection = UpgradeUtil.disableViewIndexes(metaConnection);
+                                            if(getProps().getBoolean(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB,
+                                                QueryServicesOptions.DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)) {
+                                                metaConnection = UpgradeUtil.upgradeLocalIndexes(metaConnection);
+                                            }
+                                            ConnectionQueryServicesImpl.this.removeTable(null,
+                                                PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
+                                            clearCache();
                                         }
-                                        ConnectionQueryServicesImpl.this.removeTable(null,
-                                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
-                                        clearCache();
-                                    }
                                 }
                             }
 
@@ -2531,7 +2543,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
                                             metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_STATS_NAME,
+                                            SYSTEM_STATS_NAME,
                                             MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
                                             PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
                                                     + PLong.INSTANCE.getSqlTypeName());
@@ -2551,6 +2563,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
                                 } catch (NewerSchemaAlreadyExistsException e) {}
                             }
+                            success = true;
                             scheduleRenewLeaseTasks();
                         } catch (Exception e) {
                             if (e instanceof SQLException) {
@@ -2570,6 +2583,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } finally {
                                 try {
+                                    restoreFromSnapshot(sysCatalogTableName, snapshotName, success);
+                                } catch (SQLException e) {
+                                    if (initializationException != null) {
+                                        initializationException.setNextException(e);
+                                    } else {
+                                        initializationException = e;
+                                    }
+                                }
+                                try {
                                     if (initializationException != null) {
                                         throw initializationException;
                                     }
@@ -2582,6 +2604,101 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return null;
                 }
 
+                private void createSnapshot(String snapshotName, String tableName)
+                        throws SQLException {
+                    HBaseAdmin admin = null;
+                    SQLException sqlE = null;
+                    try {
+                        admin = getAdmin();
+                        admin.snapshot(snapshotName, tableName);
+                        logger.info("Successfully created snapshot " + snapshotName + " for "
+                                + tableName);
+                    } catch (Exception e) {
+                        sqlE = new SQLException(e);
+                    } finally {
+                        try {
+                            if (admin != null) {
+                                admin.close();
+                            }
+                        } catch (Exception e) {
+                            SQLException adminCloseEx = new SQLException(e);
+                            if (sqlE == null) {
+                                sqlE = adminCloseEx;
+                            } else {
+                                sqlE.setNextException(adminCloseEx);
+                            }
+                        } finally {
+                            if (sqlE != null) {
+                                throw sqlE;
+                            }
+                        }
+                    }
+                }
+
+                private void restoreFromSnapshot(String tableName, String snapshotName,
+                        boolean success) throws SQLException {
+                    boolean snapshotRestored = false;
+                    boolean tableDisabled = false;
+                    if (!success && snapshotName != null) {
+                        SQLException sqlE = null;
+                        HBaseAdmin admin = null;
+                        try {
+                            logger.warn("Starting restore of " + tableName + " using snapshot "
+                                    + snapshotName + " because upgrade failed");
+                            admin = getAdmin();
+                            admin.disableTable(tableName);
+                            tableDisabled = true;
+                            admin.restoreSnapshot(snapshotName);
+                            snapshotRestored = true;
+                            logger.warn("Successfully restored " + tableName + " using snapshot "
+                                    + snapshotName);
+                        } catch (Exception e) {
+                            sqlE = new SQLException(e);
+                        } finally {
+                            if (admin != null && tableDisabled) {
+                                try {
+                                    admin.enableTable(tableName);
+                                    if (snapshotRestored) {
+                                        logger.warn("Successfully restored and enabled " + tableName + " using snapshot "
+                                                + snapshotName);
+                                    } else {
+                                        logger.warn("Successfully enabled " + tableName + " after restoring using snapshot "
+                                                + snapshotName + " failed. ");
+                                    }
+                                } catch (Exception e1) {
+                                    SQLException enableTableEx = new SQLException(e1);
+                                    if (sqlE == null) {
+                                        sqlE = enableTableEx;
+                                    } else {
+                                        sqlE.setNextException(enableTableEx);
+                                    }
+                                    logger.error("Failure in enabling "
+                                            + tableName
+                                            + (snapshotRestored ? " after successfully restoring using snapshot"
+                                                    + snapshotName
+                                                    : " after restoring using snapshot "
+                                                            + snapshotName + " failed. "));
+                                } finally {
+                                    try {
+                                        admin.close();
+                                    } catch (Exception e2) {
+                                        SQLException adminCloseEx = new SQLException(e2);
+                                        if (sqlE == null) {
+                                            sqlE = adminCloseEx;
+                                        } else {
+                                            sqlE.setNextException(adminCloseEx);
+                                        }
+                                    } finally {
+                                        if (sqlE != null) {
+                                            throw sqlE;
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+                
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bea9e03a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index 7ed9f70..7c34f4a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -18,6 +18,8 @@
 package org.apache.phoenix.util;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.CURRENT_CLIENT_VERSION;
+import static org.apache.phoenix.coprocessor.MetaDataProtocol.TIMESTAMP_VERSION_MAP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
@@ -51,10 +53,13 @@ import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_
 
 import java.io.IOException;
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.text.Format;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -1888,4 +1893,10 @@ public class UpgradeUtil {
         }
     }
 
+    public static final String getUpgradeSnapshotName(String tableString, long currentSystemTableTimestamp) {
+        Format formatter = new SimpleDateFormat("yyyyMMddHHmmssZ");
+        String date = formatter.format(new Date(System.currentTimeMillis()));
+        String upgradingFrom = TIMESTAMP_VERSION_MAP.get(currentSystemTableTimestamp);
+        return "SNAPSHOT_" + tableString + "_" + upgradingFrom + "_TO_" + CURRENT_CLIENT_VERSION + "_" + date;
+    }
 }
\ No newline at end of file


[08/50] [abbrv] phoenix git commit: PHOENIX-3143 Update KEYS file for 4.8 release

Posted by ma...@apache.org.
PHOENIX-3143 Update KEYS file for 4.8 release


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

Branch: refs/heads/calcite
Commit: 0b91b45aa2b4885a47c3641e55b612b8a351e1b2
Parents: 27c4027
Author: Ankit Singhal <an...@gmail.com>
Authored: Wed Aug 3 14:17:25 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Wed Aug 3 14:17:25 2016 +0530

----------------------------------------------------------------------
 KEYS | 58 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 58 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b91b45a/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index ea1f187..dca2b38 100644
--- a/KEYS
+++ b/KEYS
@@ -71,3 +71,61 @@ fMFOWYmlNK8cL+Z7zaEOFYF5ug/i0hZhhzKZc03ryhEaIEVB9Un62rLnbmgIQxF6
 GR2wPntvTGNAWEcuwD1/D1gws5A=
 =nO9e
 -----END PGP PUBLIC KEY BLOCK-----
+pub   4096R/79FA8AEB 2016-06-22
+uid                  Ankit Singhal (CODE SIGNING KEY) <an...@apache.org>
+sig 3        79FA8AEB 2016-06-22  Ankit Singhal (CODE SIGNING KEY) <an...@apache.org>
+sub   4096R/95770845 2016-06-22
+sig          79FA8AEB 2016-06-22  Ankit Singhal (CODE SIGNING KEY) <an...@apache.org>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v1
+
+mQINBFdrI3YBEACwSDbkxMnW9E8BIUFIh9abjG7o3BIIYSK4k888fOwg/Pr5rvmM
+S9zmf/sgvLa0nTbzq6Va0w0rF0XTdS8jKSHUhuOlWtEs4V9h3J9nzi10u0vU6oVy
+gVfUr1lXPEnKa5pbrrNliWTdAl3Mlzq9eRECdNzY4K5qMmymGx5+ZVHFPi3sfAmX
+lp0b56IOiBjzyLipqcHYc2y1GTvbLEKmd4GQZEUozh+h5SWNBHO2SxdZA0pCad13
+616o16d6nUisifznyvJBW0XEimUt7AaSsyEcZFBF2G9HGHf4C5AyK2Pm/YYiiLD4
+we0Yr0ip67MS3ck9X13HyDY3RV6ak/h+YcW/haEX1nL/nsz2aetvtG5ThHGgbgxt
+YzdCbJFWGKhSVKosyfZcpAzcxbww9BGr950SilSoQ31puovVjG1zLBYrTeSuZmIW
+EiP6dJxT2jwgVmGRfpro6O5PDyta2Kxnch5Q/L8DlIHJzwEAcFEgVt4tAEBvS8xY
+m3oSq2KZdVxVuJ0dAKAFDdxxWHxy8JxvQFjPVlcfvIqfvNqgmHGCv0YbrV1fvlU9
+JD9YKz1erk9sjUVkh1T2PPGnXaT7ljY2EngYLW69N4HTw9XDpBAtHKSteGvUNetj
+QLrCkW+dmAktBUBY6cUwM68Rj9oRgfNW6F/IV8oi87wKIpNpWDYiCgiOkwARAQAB
+tDNBbmtpdCBTaW5naGFsIChDT0RFIFNJR05JTkcgS0VZKSA8YW5raXRAYXBhY2hl
+Lm9yZz6JAjkEEwECACMCGwMCHgECF4AFAldrJegGCwkIBwMCBhUKCQgLAgUWAgMB
+AAAKCRANSO8XefqK6/D3EACpZ0rsprDZXruZk72OtXPYkxDcgrU55QSI1p5LFu0e
+MEmdWPQ5RCzuZfaPvkG0ZKM7SEZxJ146HBTy795cgGhpoYRb9ipM9js721KawbDB
+Warr3RWxIbrJQjcEITaKh8hpNJLxY8hsW6vREgiMb+iSxD76+WmedcuTEWEHyv7x
+ryedl+RbT8RFp/8UBCJy/D9XOiGye3pBFbYHZP0kfv7IOxnlYEb8EVgutAVB6xNF
+92rrsjEuHupevQbssOEpOB8s6AaOC4CSVAY+BlFzMDR6RYu1Mf80WNGR7dB8pwrj
+g8JOrcHEFhwZRvLwcfr/v77hQ3zw7QZ12ur4UemRxjItArSaKIDlXh7gs8pilVfH
+ysIx4FXnhG8sGopSWcZEXoIowE+jtc7z94LDlRKLhRIuaQZgiM7pSrpcYAW8SQEb
+bjICPTNTME7wSZyrPTanSU9wX5A08T2o667CELl98eTDeRhj16URB3ve16jK97OY
+E/vERxKtMFvR7FVe6HgPO70y5L4M4qbAgDivC+UlILTw6dKAkduPx0rXj9IRVaFp
+Q1efBhkglkh2W5zR/6KCGIXXIIJ9IDIl/YQUxNmAEKILva8BROY2Kg1FkypCTTSs
+WOSNpIKHpS2/XzptZhIkMuqGI61dRQH1u/zIiCRLk/bz+UHUXzbfzgvHlwX9uIp4
+BLkCDQRXayN2ARAArZQAFLFIz6pc/3rjBvpugLroZyypIzFKUomH+8URtJzYdeb8
+SN2Xn73LKnfADlkny4wX89Shl7ZW82GzANB4pXj2nCBNct6g/vMzbDWFWZgPVYHM
+130KFsaFnex1NRheiVdTai4s6QspqpAlfeNmrjxB53dYDCFbx96MuCaT0ZGnUt27
+PhM3kfMAZG1N37Lub8HHjaS9lDyca0Ofs6IlJX0y9SgoeAfeXQl2JsNRwjqblhEV
+MwDH78mrmJYVJkeKEDeOTe2X+vxoYdkDIxBAcoYulfkvvIoQ8dnNUQZiJ3crqRWY
+D+oQlrsRI1Yaeb6H6JhaTnQPgbPu6WkNs0SzdBLzOMfPPnDxYwbK4PMgqxOxqpXX
+3ooyBFxzi2bjeABJdnkyFs3Ewn1WbK3tB4rZi/xZghhRJPaQgPHVME8kCswh82lo
+3lzQy7tD5Rep+7N5OzhwmqSlhgqeIR+D/5WZazsEwF7vmtcLGQVQt9Y/0sZrcD8Y
+2vmwaAB/yWU9aXDg1yad13k1LfEDsf7mvVWnvKdIkkUXqkRSl+pbHvU2JqZ9kPvZ
+VrII6rwUdpp7qRTDtJeNaIlGZkpfVlfNEsXttS6VpIohhvnEi73OG9ClQfUznkzS
+1yyBhdQg5kGN75BbAOLYzWd/c9kXup0Axv/V9+EHwG0Or2+uVG1AWjRes0cAEQEA
+AYkCHwQYAQIACQUCV2sjdgIbDAAKCRANSO8XefqK6/wPD/49Y33UPALWQxrrK8jm
+vUo4p+sND4bZlKYw8225AUxrL0x5CEh/l2tLcX2Y23i5XI1WUnz8pfdFhCLSlssl
+RHz7YcFrXM2qXHvjd76nQ58pHY8mdvY5KJI0GTjjWz21fxE5sKSuVi9SampoaVHk
+n+pyVcNzZMZiy7wz+gtlQID2wkk5soPjbuEGyVgHavEsYd7l1NZOphBz0iMjsCL6
+aUpW+o12mNCenc1aJB2s9zMTlL+C6tdGrnHHT9hSbSWeuplN+u/jPTY18YE0I9nV
+U5wc2uwmgbwam5i2G8M1WvbbzS4t2KN/7MJ6zTIS3Om6QD+CMHLr1N5n53K7gai3
+gxXZwCxIbZHZq8yyxzfsa+mTJKqHWiSatyRga9FKt2CodNlUH7MtwlEUICmeWju0
+lph7gRzJ6Iuel5Ya+LIVX9qdULmu9ylCkY6bsZgkQSWCu815gU7g73zqCSrISw1B
+fjgjbagOwp+FWCg0kteVxouzBPSIJXSLiBvfhI3g3X+IF7X0Ul6JoqHv3M18A4VY
+pZdF1i7aGjqd5+fYRMMFYz8vuhkJR71M9E4yAutODpxliTexSQP5dH8MvoLVMiIj
+b4Ki2TbCKOPwRYX6+b2vmmOUlZ28yjeAhzHKXS9lh7nY2T+cv5cwkPZ1uw2AKG55
+pcl2PvSVaZeoTjguW8BqFjkEzA==
+=p/IB
+-----END PGP PUBLIC KEY BLOCK-----


[36/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
index b869ff4..0adcc10 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InstrFunctionIT.java
@@ -28,11 +28,11 @@ import java.sql.ResultSet;
 
 import org.junit.Test;
 
-public class InstrFunctionIT extends BaseHBaseManagedTimeIT {
-    private void initTable(Connection conn, String sortOrder, String s, String subStr) throws Exception {
-        String ddl = "CREATE TABLE SAMPLE (name VARCHAR NOT NULL PRIMARY KEY " + sortOrder + ", substr VARCHAR)";
+public class InstrFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
+    private void initTable(Connection conn, String tableName, String sortOrder, String s, String subStr) throws Exception {
+        String ddl = "CREATE TABLE " + tableName + " (name VARCHAR NOT NULL PRIMARY KEY " + sortOrder + ", substr VARCHAR)";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO SAMPLE VALUES(?,?)";
+        String dml = "UPSERT INTO " + tableName + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setString(1, s);
         stmt.setString(2, subStr);
@@ -61,56 +61,63 @@ public class InstrFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSingleByteInstrAscending() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "ASC", "abcdefghijkl","fgh");
-        String queryToExecute = "SELECT INSTR(name, 'fgh') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "ASC", "abcdefghijkl","fgh");
+        String queryToExecute = "SELECT INSTR(name, 'fgh') FROM " + tableName;
         testInstr(conn, queryToExecute, 6);
     }
     
     @Test
     public void testSingleByteInstrDescending() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "DESC", "abcdefghijkl","fgh");
-        String queryToExecute = "SELECT INSTR(name, 'fgh') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "DESC", "abcdefghijkl","fgh");
+        String queryToExecute = "SELECT INSTR(name, 'fgh') FROM " + tableName;
         testInstr(conn, queryToExecute, 6);
     }
     
     @Test
     public void testSingleByteInstrAscendingNoString() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "ASC", "abcde fghijkl","lmn");
-        String queryToExecute = "SELECT INSTR(name, 'lmn') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "ASC", "abcde fghijkl","lmn");
+        String queryToExecute = "SELECT INSTR(name, 'lmn') FROM " + tableName;
         testInstr(conn, queryToExecute, 0);
     }
     
     @Test
     public void testSingleByteInstrDescendingNoString() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "DESC", "abcde fghijkl","lmn");
-        String queryToExecute = "SELECT INSTR(name, 'lmn') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "DESC", "abcde fghijkl","lmn");
+        String queryToExecute = "SELECT INSTR(name, 'lmn') FROM " + tableName;
         testInstr(conn, queryToExecute, 0);
     }
 
     @Test
     public void testMultiByteInstrAscending() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "ASC", "A\u025a\u0266FGH","\u025a\u0266");
-        String queryToExecute = "SELECT INSTR(name, '\u025a\u0266') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "ASC", "A\u025a\u0266FGH","\u025a\u0266");
+        String queryToExecute = "SELECT INSTR(name, '\u025a\u0266') FROM " + tableName;
         testInstr(conn, queryToExecute, 2);
     }
     
     @Test
     public void testMultiByteInstrDecending() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "DESC", "A\u025a\u0266FGH","\u025a\u0266");
-        String queryToExecute = "SELECT INSTR(name, '\u025a\u0266') FROM SAMPLE";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "DESC", "A\u025a\u0266FGH","\u025a\u0266");
+        String queryToExecute = "SELECT INSTR(name, '\u025a\u0266') FROM " + tableName;
         testInstr(conn, queryToExecute, 2);
     } 
 
     @Test
     public void testByteInstrAscendingFilter() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "ASC", "abcdefghijkl","fgh");
-        String queryToExecute = "select NAME from sample where instr(name, 'fgh') > 0";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "ASC", "abcdefghijkl","fgh");
+        String queryToExecute = "select NAME from " + tableName + " where instr(name, 'fgh') > 0";
         testInstrFilter(conn, queryToExecute,"abcdefghijkl");
     }
     
@@ -118,8 +125,9 @@ public class InstrFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testByteInstrDecendingFilter() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, "DESC", "abcdefghijkl","fgh");
-        String queryToExecute = "select NAME from sample where instr(name, 'fgh') > 0";
+        String tableName = generateRandomString();
+        initTable(conn, tableName, "DESC", "abcdefghijkl","fgh");
+        String queryToExecute = "select NAME from " + tableName + " where instr(name, 'fgh') > 0";
         testInstrFilter(conn, queryToExecute,"abcdefghijkl");
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
index fb49378..504b18c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IsNullIT.java
@@ -30,29 +30,30 @@ import java.sql.ResultSet;
 import org.junit.Test;
 
 
-public class IsNullIT extends BaseHBaseManagedTimeIT {
+public class IsNullIT extends BaseHBaseManagedTimeTableReuseIT {
     @Test
     public void testIsNullInPk() throws Exception {
-        ensureTableCreated(getUrl(),"IntIntKeyTest");
+        String tableName = generateRandomString();
+        ensureTableCreated(getUrl(), tableName, "IntIntKeyTest");
         Connection conn = DriverManager.getConnection(getUrl());
-        String upsert = "UPSERT INTO IntIntKeyTest VALUES(4,2)";
+        String upsert = "UPSERT INTO " + tableName + " VALUES(4,2)";
         PreparedStatement upsertStmt = conn.prepareStatement(upsert);
         int rowsInserted = upsertStmt.executeUpdate();
         assertEquals(1, rowsInserted);
-        upsert = "UPSERT INTO IntIntKeyTest VALUES(6)";
+        upsert = "UPSERT INTO " + tableName + " VALUES(6)";
         upsertStmt = conn.prepareStatement(upsert);
         rowsInserted = upsertStmt.executeUpdate();
         assertEquals(1, rowsInserted);
         conn.commit();
         
-        String select = "SELECT i/j FROM IntIntKeyTest WHERE j IS NULL";
+        String select = "SELECT i/j FROM " + tableName + " WHERE j IS NULL";
         ResultSet rs;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(0,rs.getInt(1));
         assertTrue(rs.wasNull());
         assertFalse(rs.next());
-        select = "SELECT i/j FROM IntIntKeyTest WHERE j IS NOT NULL";
+        select = "SELECT i/j FROM " + tableName + " WHERE j IS NOT NULL";
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(2,rs.getInt(1));
@@ -61,15 +62,16 @@ public class IsNullIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIsNullWithLastPKColDesc() throws Exception {
+        String tableName = generateRandomString();
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE T(k1 VARCHAR NOT NULL, k2 VARCHAR, k3 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2, k3 DESC))");
-        conn.createStatement().execute("UPSERT INTO T VALUES ('a')");
-        conn.createStatement().execute("UPSERT INTO T VALUES ('b')");
-        conn.createStatement().execute("UPSERT INTO T VALUES ('b',null,'c')");
-        conn.createStatement().execute("UPSERT INTO T VALUES ('ba', null, 'd')");
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(k1 VARCHAR NOT NULL, k2 VARCHAR, k3 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2, k3 DESC))");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('b',null,'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('ba', null, 'd')");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1,k2,k3 FROM T WHERE k1='b' AND k2 IS NULL");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1,k2,k3 FROM " + tableName + " WHERE k1='b' AND k2 IS NULL");
         assertTrue(rs.next());
         assertEquals("b",rs.getString(1));
         assertNull(rs.getString(2));
@@ -87,15 +89,16 @@ public class IsNullIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testIsNullInCompositeKey() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE T(k1 VARCHAR, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1,k2))");
-        conn.createStatement().execute("UPSERT INTO T VALUES (null,'a')");
-        conn.createStatement().execute("UPSERT INTO T VALUES ('a','a')");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(k1 VARCHAR, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1,k2))");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES (null,'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES ('a','a')");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("SELECT count(*) FROM T");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName);
         assertTrue(rs.next());
         assertEquals(2,rs.getInt(1));
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM T WHERE k1 = 'a' or k1 is null");
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName + " WHERE k1 = 'a' or k1 is null");
         assertTrue(rs.next());
         assertEquals(2,rs.getInt(1));
         conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
index dca57b4..4497c49 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
@@ -60,7 +60,7 @@ public class KeyOnlyIT extends BaseOwnClusterClientManagedTimeIT {
     @Test
     public void testKeyOnly() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),KEYONLY_NAME,null, ts);
+        ensureTableCreated(getUrl(),KEYONLY_NAME,KEYONLY_NAME,null, ts);
         initTableValues(ts+1);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+30));
@@ -139,7 +139,7 @@ public class KeyOnlyIT extends BaseOwnClusterClientManagedTimeIT {
     @Test
     public void testOr() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),KEYONLY_NAME,null, ts);
+        ensureTableCreated(getUrl(),KEYONLY_NAME,KEYONLY_NAME,null, ts);
         initTableValues(ts+1);
         Properties props = new Properties();
         
@@ -164,7 +164,7 @@ public class KeyOnlyIT extends BaseOwnClusterClientManagedTimeIT {
     @Test
     public void testQueryWithLimitAndStats() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),KEYONLY_NAME,null, ts);
+        ensureTableCreated(getUrl(),KEYONLY_NAME,KEYONLY_NAME,null, ts);
         initTableValues(ts+1, 100);
         
         TestUtil.analyzeTable(getUrl(), ts+10, KEYONLY_NAME);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
index cbadc35..f8d2d32 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LastValueFunctionIT.java
@@ -28,33 +28,33 @@ import java.sql.ResultSet;
 import org.junit.Test;
 
 
-public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
+public class LastValueFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void unsignedLong() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date DATE, \"value\" UNSIGNED_LONG)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (1, 8, TO_DATE('2013-01-01 00:00:00'), 300)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (2, 8, TO_DATE('2013-01-01 00:01:00'), 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (3, 8, TO_DATE('2013-01-01 00:02:00'), 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (4, 8, TO_DATE('2013-01-01 00:03:00'), 4)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (5, 8, TO_DATE('2013-01-01 00:04:00'), 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") "
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") "
                 + "VALUES (6, 8, TO_DATE('2013-01-01 00:05:00'), 150)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getLong(1), 150);
@@ -65,20 +65,21 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void signedInteger() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_test_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG, date INTEGER, \"value\" INTEGER)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (5, 8, 5, -255)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (4, 8, 4, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, 5, -255)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, 4, 4)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_test_table GROUP BY page_id"
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id"
         );
 
         assertTrue(rs.next());
@@ -89,21 +90,22 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void unsignedInteger() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_test_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date UNSIGNED_INT, \"value\" UNSIGNED_INT)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_test_table (id, page_id, date, \"value\") VALUES (4, 8, 5, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, 5, 4)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_test_table GROUP BY page_id"
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id"
         );
         assertTrue(rs.next());
         assertEquals(rs.getInt(1), 4);
@@ -113,22 +115,23 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void simpleTestDescOrder() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " dates INTEGER, val INTEGER)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (1, 8, 0, 300)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (2, 8, 1, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (3, 8, 2, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (4, 8, 3, 4)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (5, 8, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (6, 8, 5, 150)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (1, 8, 0, 300)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (2, 8, 1, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (3, 8, 2, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (4, 8, 3, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (5, 8, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (6, 8, 5, 150)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(val) WITHIN GROUP (ORDER BY dates DESC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(val) WITHIN GROUP (ORDER BY dates DESC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getInt(1), 300);
@@ -138,22 +141,23 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void simpleTestAscOrder() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " dates INTEGER, val INTEGER)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (1, 8, 0, 300)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (2, 8, 1, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (3, 8, 2, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (4, 8, 3, 4)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (5, 8, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, dates, val) VALUES (6, 8, 5, 150)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (1, 8, 0, 300)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (2, 8, 1, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (3, 8, 2, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (4, 8, 3, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (5, 8, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, dates, val) VALUES (6, 8, 5, 150)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(val) WITHIN GROUP (ORDER BY dates ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(val) WITHIN GROUP (ORDER BY dates ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getInt(1), 150);
@@ -163,21 +167,22 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void charDatatype() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG, "
                 + "date CHAR(3), \"value\" CHAR(3))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (1, 8, '1', '300')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (2, 8, '2', '7')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (3, 8, '3', '9')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (5, 8, '4', '2')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (4, 8, '5', '400')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, '1', '300')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, '2', '7')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, '3', '9')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, '4', '2')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, '5', '400')");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getString(1), "400");
@@ -187,21 +192,22 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void varcharVariableLenghtDatatype() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date VARCHAR, \"value\" VARCHAR)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (1, 8, '1', '3')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (2, 8, '2', '7')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (3, 8, '3', '9')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (5, 8, '4', '2')");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (4, 8, '5', '4')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, '1', '3')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, '2', '7')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, '3', '9')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, '4', '2')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, '5', '4')");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getString(1), "4");
@@ -211,30 +217,31 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void groupMultipleValues() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date UNSIGNED_INT, \"value\" UNSIGNED_INT)";
         conn.createStatement().execute(ddl);
 
         //first page_id
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (4, 8, 5, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, 5, 4)");
 
         //second page_id
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (11, 9, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (12, 9, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (13, 9, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (15, 9, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (14, 9, 5, 40)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (11, 9, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (12, 9, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (13, 9, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (15, 9, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (14, 9, 5, 40)");
 
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         assertEquals(rs.getInt(1), 4);
@@ -247,21 +254,22 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void nullValuesInAggregatingColumns() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date UNSIGNED_INT, \"value\" UNSIGNED_INT)";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (1, 8, 1)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (2, 8, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (3, 8, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (5, 8, 4)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (4, 8, 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (1, 8, 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (2, 8, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (3, 8, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (5, 8, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (4, 8, 5)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         byte[] nothing = rs.getBytes(1);
@@ -271,22 +279,23 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void nullValuesInAggregatingColumnsSecond() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_LONG,"
                 + " date UNSIGNED_INT, \"value\" UNSIGNED_INT)";
         conn.createStatement().execute(ddl);
 
         //first page_id
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (1, 8, 1)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (2, 8, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (3, 8, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (5, 8, 4)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date) VALUES (4, 8, 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (1, 8, 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (2, 8, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (3, 8, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (5, 8, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date) VALUES (4, 8, 5)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
-                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM last_value_table GROUP BY page_id");
+                "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) FROM " + tableName + " GROUP BY page_id");
 
         assertTrue(rs.next());
         byte[] nothing = rs.getBytes(1);
@@ -296,31 +305,32 @@ public class LastValueFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void inOrderByClausule() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE IF NOT EXISTS last_value_table "
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName 
                 + "(id INTEGER NOT NULL PRIMARY KEY, page_id UNSIGNED_INT,"
                 + " date UNSIGNED_INT, \"value\" UNSIGNED_INT)";
         conn.createStatement().execute(ddl);
 
         //first page
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (4, 8, 5, 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (1, 8, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (2, 8, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (3, 8, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 8, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (4, 8, 5, 5)");
 
         //second page
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (5, 2, 1, 3)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (6, 2, 2, 7)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (7, 2, 3, 9)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (8, 2, 4, 2)");
-        conn.createStatement().execute("UPSERT INTO last_value_table (id, page_id, date, \"value\") VALUES (9, 2, 5, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (5, 2, 1, 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (6, 2, 2, 7)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (7, 2, 3, 9)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (8, 2, 4, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (id, page_id, date, \"value\") VALUES (9, 2, 5, 4)");
 
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT LAST_VALUE(\"value\") WITHIN GROUP (ORDER BY date ASC) AS val "
-                + "FROM last_value_table GROUP BY page_id ORDER BY val DESC");
+                + "FROM " + tableName + " GROUP BY page_id ORDER BY val DESC");
 
         assertTrue(rs.next());
         assertEquals(rs.getInt(1), 5);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/LnLogFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LnLogFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LnLogFunctionEnd2EndIT.java
index e2c72ca..466e71d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LnLogFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LnLogFunctionEnd2EndIT.java
@@ -33,10 +33,12 @@ import org.junit.Test;
 /**
  * End to end tests for {@link LnFunction} and {@link LogFunction}
  */
-public class LnLogFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
+public class LnLogFunctionEnd2EndIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final String KEY = "key";
     private static final double ZERO = 1e-9;
+    private String signedTableName;
+    private String unsignedTableName;
 
     private static boolean twoDoubleEquals(double a, double b) {
         if (Double.isNaN(a) ^ Double.isNaN(b)) return false;
@@ -57,14 +59,17 @@ public class LnLogFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     public void initTable() throws Exception {
         Connection conn = null;
         PreparedStatement stmt = null;
+        signedTableName = generateRandomString();
+        unsignedTableName = generateRandomString();
+
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
             ddl =
-                    "CREATE TABLE testSigned (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
+                    "CREATE TABLE " + signedTableName + " (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
             conn.createStatement().execute(ddl);
             ddl =
-                    "CREATE TABLE testUnsigned (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
+                    "CREATE TABLE " + unsignedTableName + " (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
             conn.createStatement().execute(ddl);
             conn.commit();
         } finally {
@@ -136,8 +141,8 @@ public class LnLogFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     public void test() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         for (double d : new double[] { 0.0, 1.0, -1.0, 123.1234, -123.1234 }) {
-            testNumberSpec(conn, d, "testSigned");
-            if (d >= 0) testNumberSpec(conn, d, "testUnsigned");
+            testNumberSpec(conn, d, signedTableName );
+            if (d >= 0) testNumberSpec(conn, d, unsignedTableName );
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/MapReduceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MapReduceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MapReduceIT.java
index f030701..275d524 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MapReduceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MapReduceIT.java
@@ -41,56 +41,63 @@ import static org.junit.Assert.*;
 /**
  * Test that our MapReduce basic tools work as expected
  */
-public class MapReduceIT extends BaseHBaseManagedTimeIT {
+public class MapReduceIT extends BaseHBaseManagedTimeTableReuseIT {
 
-    private static final String STOCK_TABLE_NAME = "stock";
-    private static final String STOCK_STATS_TABLE_NAME = "stock_stats";
     private static final String STOCK_NAME = "STOCK_NAME";
     private static final String RECORDING_YEAR = "RECORDING_YEAR";
     private static final String RECORDINGS_QUARTER = "RECORDINGS_QUARTER";
-    private static final String CREATE_STOCK_TABLE = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE_NAME + " ( " +
-            STOCK_NAME + " VARCHAR NOT NULL ," + RECORDING_YEAR + " INTEGER NOT  NULL, " + RECORDINGS_QUARTER +
-            " DOUBLE array[] CONSTRAINT pk PRIMARY KEY (" + STOCK_NAME + " , " + RECORDING_YEAR + "))";
+    private  String CREATE_STOCK_TABLE = "CREATE TABLE IF NOT EXISTS %s ( " +
+            " STOCK_NAME VARCHAR NOT NULL , RECORDING_YEAR  INTEGER NOT  NULL,  RECORDINGS_QUARTER " +
+            " DOUBLE array[] CONSTRAINT pk PRIMARY KEY ( STOCK_NAME, RECORDING_YEAR ))";
 
     private static final String MAX_RECORDING = "MAX_RECORDING";
-    private static final String CREATE_STOCK_STATS_TABLE =
-            "CREATE TABLE IF NOT EXISTS " + STOCK_STATS_TABLE_NAME + "(" + STOCK_NAME + " VARCHAR NOT NULL , "
-                    + MAX_RECORDING + " DOUBLE CONSTRAINT pk PRIMARY KEY (" + STOCK_NAME + "))";
-    private static final String UPSERT = "UPSERT into " + STOCK_TABLE_NAME + " values (?, ?, ?)";
+    private  String CREATE_STOCK_STATS_TABLE =
+            "CREATE TABLE IF NOT EXISTS %s(STOCK_NAME VARCHAR NOT NULL , "
+                    + " MAX_RECORDING DOUBLE CONSTRAINT pk PRIMARY KEY (STOCK_NAME ))";
+    private String UPSERT = "UPSERT into %s values (?, ?, ?)";
 
     @Before
     public void setupTables() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute(CREATE_STOCK_TABLE);
-        conn.createStatement().execute(CREATE_STOCK_STATS_TABLE);
-        conn.commit();
+
     }
 
     @Test
     public void testNoConditionsOnSelect() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String stockTableName = generateRandomString();
+        String stockStatsTableName = generateRandomString();
+        conn.createStatement().execute(String.format(CREATE_STOCK_TABLE, stockTableName));
+        conn.createStatement().execute(String.format(CREATE_STOCK_STATS_TABLE, stockStatsTableName));
+        conn.commit();
         final Configuration conf = getUtility().getConfiguration();
         Job job = Job.getInstance(conf);
-        PhoenixMapReduceUtil.setInput(job, StockWritable.class, STOCK_TABLE_NAME, null,
+        PhoenixMapReduceUtil.setInput(job, StockWritable.class, stockTableName, null,
                 STOCK_NAME, RECORDING_YEAR, "0." + RECORDINGS_QUARTER);
-        testJob(job, 91.04);
+        testJob(job, stockTableName, stockStatsTableName, 91.04);
     }
 
     @Test
     public void testConditionsOnSelect() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String stockTableName = generateRandomString();
+        String stockStatsTableName = generateRandomString();
+        conn.createStatement().execute(String.format(CREATE_STOCK_TABLE, stockTableName));
+        conn.createStatement().execute(String.format(CREATE_STOCK_STATS_TABLE, stockStatsTableName));
+        conn.commit();
         final Configuration conf = getUtility().getConfiguration();
         Job job = Job.getInstance(conf);
-        PhoenixMapReduceUtil.setInput(job, StockWritable.class, STOCK_TABLE_NAME, RECORDING_YEAR+"  < 2009",
+        PhoenixMapReduceUtil.setInput(job, StockWritable.class, stockTableName, RECORDING_YEAR+"  < 2009",
                 STOCK_NAME, RECORDING_YEAR, "0." + RECORDINGS_QUARTER);
-        testJob(job, 81.04);
+        testJob(job, stockTableName, stockStatsTableName, 81.04);
     }
 
-    private void testJob(Job job, double expectedMax)
+    private void testJob(Job job, String stockTableName, String stockStatsTableName, double expectedMax)
             throws SQLException, InterruptedException, IOException, ClassNotFoundException {
-        upsertData();
+        upsertData(stockTableName);
 
         // only run locally, rather than having to spin up a MiniMapReduce cluster and lets us use breakpoints
         job.getConfiguration().set("mapreduce.framework.name", "local");
-        setOutput(job);
+        setOutput(job, stockStatsTableName);
 
         job.setMapperClass(StockMapper.class);
         job.setReducerClass(StockReducer.class);
@@ -106,7 +113,7 @@ public class MapReduceIT extends BaseHBaseManagedTimeIT {
 
         // verify
         ResultSet stats = DriverManager.getConnection(getUrl()).createStatement()
-                .executeQuery("SELECT * FROM " + STOCK_STATS_TABLE_NAME);
+                .executeQuery("SELECT * FROM " + stockStatsTableName);
         assertTrue("No data stored in stats table!", stats.next());
         String name = stats.getString(1);
         double max = stats.getDouble(2);
@@ -120,17 +127,17 @@ public class MapReduceIT extends BaseHBaseManagedTimeIT {
      *
      * @param job to update
      */
-    private void setOutput(Job job) {
+    private void setOutput(Job job, String stockStatsTableName) {
         final Configuration configuration = job.getConfiguration();
-        PhoenixConfigurationUtil.setOutputTableName(configuration, STOCK_STATS_TABLE_NAME);
-        configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, "UPSERT into " + STOCK_STATS_TABLE_NAME +
+        PhoenixConfigurationUtil.setOutputTableName(configuration, stockStatsTableName);
+        configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, "UPSERT into " + stockStatsTableName +
                 " (" + STOCK_NAME + ", " + MAX_RECORDING + ") values (?,?)");
         job.setOutputFormatClass(PhoenixOutputFormat.class);
     }
 
-    private void upsertData() throws SQLException {
+    private void upsertData(String stockTableName) throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        PreparedStatement stmt = conn.prepareStatement(UPSERT);
+        PreparedStatement stmt = conn.prepareStatement(String.format(UPSERT, stockTableName));
         upsertData(stmt, "AAPL", 2009, new Double[]{85.88, 91.04, 88.5, 90.3});
         upsertData(stmt, "AAPL", 2008, new Double[]{75.88, 81.04, 78.5, 80.3});
         conn.commit();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
index 6c51ebd..3ae33fb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MappingTableDataTypeIT.java
@@ -49,7 +49,7 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
 
-public class MappingTableDataTypeIT extends BaseHBaseManagedTimeIT {
+public class MappingTableDataTypeIT extends BaseHBaseManagedTimeTableReuseIT {
     @Test
     public void testMappingHbaseTableToPhoenixTable() throws Exception {
         String mtest = generateRandomString();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ModulusExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ModulusExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ModulusExpressionIT.java
index 7de85ea..8cb061f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ModulusExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ModulusExpressionIT.java
@@ -32,7 +32,7 @@ import java.sql.SQLException;
 import org.junit.Test;
 
 
-public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
+public class ModulusExpressionIT extends BaseHBaseManagedTimeTableReuseIT {
     
     private static final long SMALL_VALUE = 31L;
     private static final long LARGE_VALUE = 0x5dec6f3847021a9bL;
@@ -40,10 +40,11 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     private static final long[] DIVIDENDS = {Long.MAX_VALUE, LARGE_VALUE, SMALL_VALUE, 0, -SMALL_VALUE, -LARGE_VALUE, Long.MIN_VALUE};
     private static final long[] DIVISORS = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 14, 31, 127, 1024};
     
-    private void initTable(Connection conn, long value) throws SQLException {
-        String ddl = "CREATE TABLE MODULUS_TEST (pk BIGINT NOT NULL PRIMARY KEY, kv BIGINT)";
+    private void initTable(Connection conn, long value, String tableName) throws SQLException {
+
+        String ddl = "CREATE TABLE " + tableName + " (pk BIGINT NOT NULL PRIMARY KEY, kv BIGINT)";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO MODULUS_TEST VALUES(?)";
+        String dml = "UPSERT INTO " + tableName + " VALUES(?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setLong(1, value);
         stmt.execute();
@@ -52,11 +53,12 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     
     private void testDividend(long dividend) throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, dividend);
-        
+        String tableName = generateRandomString();
+        initTable(conn, dividend, tableName);
+
         for(long divisor : DIVISORS) {
             long remainder = dividend % divisor;
-            String sql = "SELECT pk % " + divisor + " FROM MODULUS_TEST";
+            String sql = "SELECT pk % " + divisor + " FROM " + tableName;
             
             ResultSet rs = conn.createStatement().executeQuery(sql);
             assertTrue(rs.next());
@@ -103,16 +105,17 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testZeroDivisor() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, 0);
+        String tableName = generateRandomString();
+        initTable(conn, 0, tableName);
         
         for(long dividend : DIVIDENDS) {
             try {
-                String sql = "SELECT " + dividend + " % pk FROM MODULUS_TEST";
+                String sql = "SELECT " + dividend + " % pk FROM " + tableName;
 
                 // workaround for parser not being able to parse Long.MIN_VALUE
                 // see: https://issues.apache.org/jira/browse/PHOENIX-1061
                 if(dividend == Long.MIN_VALUE) {
-                    sql = "SELECT (" + (dividend + 1) + " + -1) % pk FROM MODULUS_TEST";
+                    sql = "SELECT (" + (dividend + 1) + " + -1) % pk FROM " + tableName;
                 }
 
                 ResultSet rs = conn.createStatement().executeQuery(sql);
@@ -129,10 +132,11 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testNullDividend() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, SMALL_VALUE);
+        String tableName = generateRandomString();
+        initTable(conn, SMALL_VALUE, tableName);
         
         for(long divisor : DIVISORS) {
-            String sql = "SELECT kv % " + divisor + " FROM MODULUS_TEST";
+            String sql = "SELECT kv % " + divisor + " FROM " + tableName;
             
             ResultSet rs = conn.createStatement().executeQuery(sql);
             assertTrue(rs.next());
@@ -144,15 +148,16 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testNullDivisor() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, SMALL_VALUE);
+        String tableName = generateRandomString();
+        initTable(conn, SMALL_VALUE, tableName);
         
         for(long dividend : DIVIDENDS) {
-            String sql = "SELECT " + dividend + " % kv FROM MODULUS_TEST";
+            String sql = "SELECT " + dividend + " % kv FROM " + tableName;
             
             // workaround for parser not being able to parse Long.MIN_VALUE
             // see: https://issues.apache.org/jira/browse/PHOENIX-1061
             if(dividend == Long.MIN_VALUE) {
-                sql = "SELECT (" + (dividend + 1) + " + -1) % kv FROM MODULUS_TEST";
+                sql = "SELECT (" + (dividend + 1) + " + -1) % kv FROM " + tableName;
             }
             
             ResultSet rs = conn.createStatement().executeQuery(sql);
@@ -165,21 +170,23 @@ public class ModulusExpressionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testNullEverything() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTable(conn, SMALL_VALUE);
+        String tableName = generateRandomString();
+        initTable(conn, SMALL_VALUE, tableName);
         
-        String sql = "SELECT null % kv FROM MODULUS_TEST";
+        String sql = "SELECT null % kv FROM " + tableName;
         
         ResultSet rs = conn.createStatement().executeQuery(sql);
         assertTrue(rs.next());
         assertNull(rs.getObject(1));
         assertFalse(rs.next());
         
-        sql = "SELECT kv % null FROM MODULUS_TEST";
+        sql = "SELECT kv % null FROM " + tableName;
         
         rs = conn.createStatement().executeQuery(sql);
         assertTrue(rs.next());
         assertNull(rs.getObject(1));
         assertFalse(rs.next());
     }
-    
+
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
index 2b14fe9..15fc01d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
@@ -57,7 +57,7 @@ public class MultiCfQueryExecIT extends BaseOwnClusterClientManagedTimeIT {
     }
     
     protected static void initTableValues(long ts) throws Exception {
-        ensureTableCreated(getUrl(),MULTI_CF,null, ts-2);
+        ensureTableCreated(getUrl(),MULTI_CF,MULTI_CF,null, ts-2);
         
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/NamespaceSchemaMappingIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NamespaceSchemaMappingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NamespaceSchemaMappingIT.java
index 1db74aa..444cf0d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NamespaceSchemaMappingIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NamespaceSchemaMappingIT.java
@@ -44,7 +44,7 @@ import org.junit.Test;
 /*
  * since 4.8
  */
-public class NamespaceSchemaMappingIT extends BaseHBaseManagedTimeIT {
+public class NamespaceSchemaMappingIT extends BaseHBaseManagedTimeTableReuseIT {
     /**
      * Tests that when: There is a table created with older version of phoenix and a table created with newer version
      * having {@code QueryServices#IS_NAMESPACE_MAPPING_ENABLED} true, then there is only a flag
@@ -57,7 +57,7 @@ public class NamespaceSchemaMappingIT extends BaseHBaseManagedTimeIT {
 
         String namespace = "TEST_SCHEMA";
         String schemaName = namespace;
-        String tableName = "TEST";
+        String tableName = generateRandomString();
 
         String phoenixFullTableName = schemaName + "." + tableName;
         String hbaseFullTableName = schemaName + ":" + tableName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
index 2c5de02..3d0fa2c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
@@ -151,7 +151,7 @@ public class NativeHBaseTypesIT extends BaseClientManagedTimeIT {
         }
         // Create Phoenix table after HBase table was created through the native APIs
         // The timestamp of the table creation must be later than the timestamp of the data
-        ensureTableCreated(getUrl(),HBASE_NATIVE,null, ts+1);
+        ensureTableCreated(getUrl(),HBASE_NATIVE,HBASE_NATIVE,null, ts+1);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index 2c880e7..dcb2a0d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -43,13 +43,13 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
 
-public class OrderByIT extends BaseHBaseManagedTimeIT {
+public class OrderByIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testMultiOrderByExpr() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
-        String query = "SELECT entity_id FROM aTable ORDER BY b_string, entity_id";
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String query = "SELECT entity_id FROM " + tableName + " ORDER BY b_string, entity_id";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -84,8 +84,8 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testDescMultiOrderByExpr() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
-        String query = "SELECT entity_id FROM aTable ORDER BY b_string || entity_id desc";
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String query = "SELECT entity_id FROM " + tableName + " ORDER BY b_string || entity_id desc";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -123,12 +123,13 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
         conn.setAutoCommit(false);
 
         try {
-            String ddl = "CREATE TABLE t_table " +
+            String tableName = generateRandomString();
+            String ddl = "CREATE TABLE " + tableName +
                     "  (a_string varchar not null, col1 integer" +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
 
-            String dml = "UPSERT INTO t_table VALUES(?, ?)";
+            String dml = "UPSERT INTO " + tableName + " VALUES(?, ?)";
             PreparedStatement stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -141,7 +142,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            String query = "select count(*), col1 from t_table group by col1 order by 2";
+            String query = "select count(*), col1 from " + tableName + " group by col1 order by 2";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(1,rs.getInt(1));
@@ -151,7 +152,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             assertEquals(1,rs.getInt(1));  
             assertFalse(rs.next());  
 
-            query = "select a_string x, col1 y from t_table order by x";
+            query = "select a_string x, col1 y from " + tableName + " order by x";
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a",rs.getString(1));
@@ -164,7 +165,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             assertEquals(30,rs.getInt(2));
             assertFalse(rs.next());  
 
-            query = "select * from t_table order by 2";
+            query = "select * from " + tableName + " order by 2";
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("b",rs.getString(1));
@@ -189,11 +190,12 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
         conn.setAutoCommit(false);
 
         try {
-            String ddl = "CREATE TABLE x_table " +
+            String tableName = generateRandomString();
+            String ddl = "CREATE TABLE " + tableName +
                     "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer, col2 integer" +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
-            String dml = "UPSERT INTO x_table VALUES(?,?,?,?,?,?,?)";
+            String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -221,7 +223,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            String query = "select * from x_table order by 2, 5";
+            String query = "select * from " + tableName + " order by 2, 5";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("c",rs.getString(1));
@@ -249,7 +251,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             assertEquals(1,rs.getInt(7));         
             assertFalse(rs.next());  
 
-            query = "select * from x_table order by 7";
+            query = "select * from " + tableName + " order by 7";
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a",rs.getString(1));  
@@ -288,11 +290,12 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
         conn.setAutoCommit(false);
 
         try {
-            String ddl = "CREATE TABLE s_table " +
+            String tableName1 = generateRandomString();
+            String ddl = "CREATE TABLE " + tableName1 +
                     "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer " +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
-            String dml = "UPSERT INTO s_table VALUES(?,?,?,?,?,?)";
+            String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -317,12 +320,13 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            ddl = "CREATE TABLE t_table " +
+            String tableName2 = generateRandomString();
+            ddl = "CREATE TABLE " + tableName2 +
                     "  (a_string varchar not null, col1 integer" +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
 
-            dml = "UPSERT INTO t_table VALUES(?, ?)";
+            dml = "UPSERT INTO " + tableName2 + " VALUES(?, ?)";
             stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -335,7 +339,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            String query = "select t1.* from s_table t1 join t_table t2 on t1.a_string = t2.a_string order by 3";
+            String query = "select t1.* from " + tableName1 + " t1 join " + tableName2 + " t2 on t1.a_string = t2.a_string order by 3";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a",rs.getString(1));  
@@ -360,7 +364,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             assertEquals(60,rs.getInt(6));
             assertFalse(rs.next());  
 
-            query = "select t1.a_string, t2.col1 from s_table t1 join t_table t2 on t1.a_string = t2.a_string order by 2";
+            query = "select t1.a_string, t2.col1 from " + tableName1 + " t1 join " + tableName2 + " t2 on t1.a_string = t2.a_string order by 2";
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("b",rs.getString(1));  
@@ -385,11 +389,12 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
         conn.setAutoCommit(false);
 
         try {
-            String ddl = "CREATE TABLE x_table " +
+            String tableName1 = generateRandomString();
+            String ddl = "CREATE TABLE  " + tableName1 +
                     "  (a_string varchar not null, cf1.a integer, cf1.b varchar, col1 integer, cf2.c varchar, cf2.d integer " +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
-            String dml = "UPSERT INTO x_table VALUES(?,?,?,?,?,?)";
+            String dml = "UPSERT INTO " + tableName1 + " VALUES(?,?,?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -414,12 +419,13 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            ddl = "CREATE TABLE y_table " +
+            String tableName2 = generateRandomString();
+            ddl = "CREATE TABLE " + tableName2 +
                     "  (a_string varchar not null, col1 integer" +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
 
-            dml = "UPSERT INTO y_table VALUES(?, ?)";
+            dml = "UPSERT INTO " + tableName2 + " VALUES(?, ?)";
             stmt = conn.prepareStatement(dml);
             stmt.setString(1, "aa");
             stmt.setInt(2, 40);
@@ -432,7 +438,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            String query = "select a_string, cf2.d from x_table union all select * from y_table order by 2";
+            String query = "select a_string, cf2.d from " + tableName1 + " union all select * from " + tableName2 + " order by 2";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("bb",rs.getString(1));  
@@ -465,13 +471,14 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
         conn.setAutoCommit(false);
 
         try {
-            String ddl = "CREATE TABLE e_table " +
+            String tableName = generateRandomString();
+            String ddl = "CREATE TABLE " + tableName +
                     "  (a_string varchar not null, col1 integer, col2 integer, col3 timestamp, col4 varchar" +
                     "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
             createTestTable(getUrl(), ddl);
 
             Date date = new Date(System.currentTimeMillis());
-            String dml = "UPSERT INTO e_table VALUES(?, ?, ?, ?, ?)";
+            String dml = "UPSERT INTO " + tableName + " VALUES(?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(dml);
             stmt.setString(1, "a");
             stmt.setInt(2, 40);
@@ -493,7 +500,7 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            String query = "SELECT col1+col2, col4, a_string FROM e_table ORDER BY 1, 2";
+            String query = "SELECT col1+col2, col4, a_string FROM " + tableName + " ORDER BY 1, 2";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a", rs.getString(3));
@@ -512,19 +519,20 @@ public class OrderByIT extends BaseHBaseManagedTimeIT {
     public void testOrderByRVC() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = "create table test1 (testpk varchar not null primary key, l_quantity decimal(15,2), l_discount decimal(15,2))";
+        String tableName = generateRandomString();
+        String ddl = "create table " + tableName + " (testpk varchar not null primary key, l_quantity decimal(15,2), l_discount decimal(15,2))";
         conn.createStatement().execute(ddl);
 
-        PreparedStatement stmt = conn.prepareStatement("upsert into test1 values ('a',0.1,0.9)");
+        PreparedStatement stmt = conn.prepareStatement("upsert into " + tableName + " values ('a',0.1,0.9)");
         stmt.execute();
-        stmt = conn.prepareStatement(" upsert into test1 values ('b',0.5,0.5)");
+        stmt = conn.prepareStatement(" upsert into " + tableName + " values ('b',0.5,0.5)");
         stmt.execute();
-        stmt = conn.prepareStatement(" upsert into test1 values ('c',0.9,0.1)");
+        stmt = conn.prepareStatement(" upsert into " + tableName + " values ('c',0.9,0.1)");
         stmt.execute();
         conn.commit();
 
         ResultSet rs;
-        stmt = conn.prepareStatement("select l_discount,testpk from test1 order by (l_discount,l_quantity)");
+        stmt = conn.prepareStatement("select l_discount,testpk from " + tableName + " order by (l_discount,l_quantity)");
         rs = stmt.executeQuery();
         assertTrue(rs.next());
         assertEquals(0.1, rs.getDouble(1), 0.01);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
index e86cf27..4e1e983 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelIteratorsIT.java
@@ -116,7 +116,7 @@ public class ParallelIteratorsIT extends BaseOwnClusterHBaseManagedTimeIT {
     public void testServerNameOnScan() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl(), TEST_PROPERTIES);
         byte[][] splits = new byte[][] { K3, K9, KR };
-        ensureTableCreated(getUrl(), STABLE_NAME, splits);
+        ensureTableCreated(getUrl(), STABLE_NAME, STABLE_NAME, splits);
         
         PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
         ResultSet rs = stmt.executeQuery("SELECT * FROM " + STABLE_NAME + " LIMIT 1");
@@ -138,7 +138,7 @@ public class ParallelIteratorsIT extends BaseOwnClusterHBaseManagedTimeIT {
     public void testGuidePostsLifeCycle() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl(), TEST_PROPERTIES);
         byte[][] splits = new byte[][] { K3, K9, KR };
-        ensureTableCreated(getUrl(), STABLE_NAME, splits);
+        ensureTableCreated(getUrl(), STABLE_NAME, STABLE_NAME, splits);
         // create index
         conn.createStatement().execute("CREATE INDEX " + STABLE_INDEX + " ON " + STABLE_NAME + "( \"value\")");
         // before upserting
@@ -206,7 +206,7 @@ public class ParallelIteratorsIT extends BaseOwnClusterHBaseManagedTimeIT {
     
     private static void initTableValues(Connection conn) throws Exception {
         byte[][] splits = new byte[][] {K3,K4,K9,K11};
-        ensureTableCreated(getUrl(),STABLE_NAME,splits);
+        ensureTableCreated(getUrl(),STABLE_NAME, STABLE_NAME, splits);
         PreparedStatement stmt = conn.prepareStatement("upsert into " + STABLE_NAME + " VALUES (?, ?)");
         stmt.setString(1, new String(KMIN));
         stmt.setInt(2, 1);


[30/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
index 5b2b15a..e47c055 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
@@ -29,27 +29,24 @@ import java.sql.ResultSet;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
-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.apache.phoenix.util.*;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
 
-public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
+public class SaltedIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     private static final int TABLE_SPLITS = 3;
     private static final int INDEX_SPLITS = 4;
     
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Forces server cache to be used
@@ -60,15 +57,15 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
-    private static void makeImmutableAndDeleteData() throws Exception {
+    private static void makeImmutableAndDeleteData(String tableName, String fullTableName) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
         try {
             conn.setAutoCommit(true);
-            conn.createStatement().execute("DELETE FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME);
-            conn.createStatement().execute("ALTER TABLE " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " SET IMMUTABLE_ROWS=true");
-            conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME).next();
+            conn.createStatement().execute("DELETE FROM " + fullTableName);
+            conn.createStatement().execute("ALTER TABLE " + fullTableName + " SET IMMUTABLE_ROWS=true");
+            conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName).next();
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-            assertTrue(pconn.getTable(new PTableKey(pconn.getTenantId(), TestUtil.DEFAULT_DATA_TABLE_FULL_NAME)).isImmutableRows());
+            assertTrue(pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isImmutableRows());
         } finally {
             conn.close();
         }
@@ -76,43 +73,55 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testMutableTableIndexMaintanenceSaltedSalted() throws Exception {
-        testMutableTableIndexMaintanence(TABLE_SPLITS, INDEX_SPLITS);
-        makeImmutableAndDeleteData();
-        testMutableTableIndexMaintanence(TABLE_SPLITS, INDEX_SPLITS);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, TABLE_SPLITS, INDEX_SPLITS);
+        makeImmutableAndDeleteData(tableName, fullTableName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, TABLE_SPLITS, INDEX_SPLITS);
     }
 
     @Test
     public void testMutableTableIndexMaintanenceSalted() throws Exception {
-        testMutableTableIndexMaintanence(null, INDEX_SPLITS);
-        makeImmutableAndDeleteData();
-        testMutableTableIndexMaintanence(null, INDEX_SPLITS);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, null, INDEX_SPLITS);
+        makeImmutableAndDeleteData(tableName, fullTableName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, null, INDEX_SPLITS);
     }
 
     @Test
     public void testMutableTableIndexMaintanenceUnsalted() throws Exception {
-        testMutableTableIndexMaintanence(null, null);
-        makeImmutableAndDeleteData();
-        testMutableTableIndexMaintanence(null, null);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, null, null);
+        makeImmutableAndDeleteData(tableName, fullTableName);
+        testMutableTableIndexMaintanence(tableName, fullTableName, indexName, fullIndexName, null, null);
     }
 
-    private void testMutableTableIndexMaintanence(Integer tableSaltBuckets, Integer indexSaltBuckets) throws Exception {
+    private void testMutableTableIndexMaintanence(String dataTableName , String dataTableFullName, String indexTableName, String indexTableFullName, Integer tableSaltBuckets, Integer indexSaltBuckets) throws Exception {
         String query;
         ResultSet rs;
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
-        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR)  " +  (tableSaltBuckets == null ? "" : " SALT_BUCKETS=" + tableSaltBuckets));
-        query = "SELECT * FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME;
+        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + dataTableFullName + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR)  " +  (tableSaltBuckets == null ? "" : " SALT_BUCKETS=" + tableSaltBuckets));
+        query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
         
-        conn.createStatement().execute("CREATE INDEX IF NOT EXISTS " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " (v DESC)" + (indexSaltBuckets == null ? "" : " SALT_BUCKETS=" + indexSaltBuckets));
-        query = "SELECT * FROM " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME;
+        conn.createStatement().execute("CREATE INDEX IF NOT EXISTS " + indexTableName + " ON " + dataTableFullName + " (v DESC)" + (indexSaltBuckets == null ? "" : " SALT_BUCKETS=" + indexSaltBuckets));
+        query = "SELECT * FROM " + indexTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
 
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " VALUES(?,?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableFullName + " VALUES(?,?)");
         stmt.setString(1,"a");
         stmt.setString(2, "x");
         stmt.execute();
@@ -121,7 +130,7 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         stmt.execute();
         conn.commit();
 
-        query = "SELECT * FROM " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME;
+        query = "SELECT * FROM " + indexTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("y",rs.getString(1));
@@ -131,7 +140,7 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         assertEquals("a",rs.getString(2));
         assertFalse(rs.next());
 
-        query = "SELECT k,v FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " WHERE v = 'y'";
+        query = "SELECT k,v FROM " + dataTableFullName + " WHERE v = 'y'";
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("b",rs.getString(1));
@@ -141,9 +150,10 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         String expectedPlan;
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = indexSaltBuckets == null ? 
-             "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME + " [~'y']\n" + 
-             "    SERVER FILTER BY FIRST KEY ONLY" : 
-            ("CLIENT PARALLEL 4-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME + " [0,~'y'] - ["+(indexSaltBuckets.intValue()-1)+",~'y']\n" + 
+             "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableFullName + " [~'y']\n" +
+             "    SERVER FILTER BY FIRST KEY ONLY" :
+            ("CLIENT PARALLEL 4-WAY RANGE SCAN OVER " + indexTableFullName + " [0,~'y'] - ["+(indexSaltBuckets.intValue()-1)+",~'y']\n" +
+
              "    SERVER FILTER BY FIRST KEY ONLY\n" +
              "CLIENT MERGE SORT");
         assertEquals(expectedPlan,QueryUtil.getExplainPlan(rs));
@@ -151,7 +161,7 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         // Will use index, so rows returned in DESC order.
         // This is not a bug, though, because we can
         // return in any order.
-        query = "SELECT k,v FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " WHERE v >= 'x'";
+        query = "SELECT k,v FROM " + dataTableFullName + " WHERE v >= 'x'";
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("b",rs.getString(1));
@@ -162,15 +172,16 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = indexSaltBuckets == null ? 
-            "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME + " [*] - [~'x']\n"
+            "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableFullName + " [*] - [~'x']\n"
           + "    SERVER FILTER BY FIRST KEY ONLY" :
-            ("CLIENT PARALLEL 4-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_FULL_NAME + " [0,*] - ["+(indexSaltBuckets.intValue()-1)+",~'x']\n"
-           + "    SERVER FILTER BY FIRST KEY ONLY\n" + 
+            ("CLIENT PARALLEL 4-WAY RANGE SCAN OVER " + indexTableFullName + " [0,*] - ["+(indexSaltBuckets.intValue()-1)+",~'x']\n"
+
+           + "    SERVER FILTER BY FIRST KEY ONLY\n" +
              "CLIENT MERGE SORT");
         assertEquals(expectedPlan,QueryUtil.getExplainPlan(rs));
         
         // Use data table, since point lookup trumps order by
-        query = "SELECT k,v FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " WHERE k = 'a' ORDER BY v";
+        query = "SELECT k,v FROM " + dataTableFullName + " WHERE k = 'a' ORDER BY v";
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("a",rs.getString(1));
@@ -178,10 +189,10 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-                "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "\n" +
+                "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + dataTableFullName + "\n" +
                 "    SERVER SORTED BY [V]\n" + 
                 "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "\n" + 
+                    "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + dataTableFullName + "\n" +
                     "    SERVER SORTED BY [V]\n" + 
                     "CLIENT MERGE SORT";
         String explainPlan2 = QueryUtil.getExplainPlan(rs);
@@ -190,7 +201,7 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         // Will use data table now, since there's a LIMIT clause and
         // we're able to optimize out the ORDER BY, unless the data
         // table is salted.
-        query = "SELECT k,v FROM " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " WHERE v >= 'x' ORDER BY k LIMIT 2";
+        query = "SELECT k,v FROM " + dataTableFullName + " WHERE v >= 'x' ORDER BY k LIMIT 2";
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("a",rs.getString(1));
@@ -201,11 +212,11 @@ public class SaltedIndexIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-             "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "\n" +
+             "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableFullName + "\n" +
              "    SERVER FILTER BY V >= 'x'\n" + 
              "    SERVER 2 ROW LIMIT\n" + 
              "CLIENT 2 ROW LIMIT" :
-                 "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "\n" +
+                 "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + dataTableFullName + "\n" +
                  "    SERVER FILTER BY V >= 'x'\n" + 
                  "    SERVER 2 ROW LIMIT\n" + 
                  "CLIENT MERGE SORT\n" + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index 8a2f281..d97a749 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.end2end.index;
 
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -35,7 +37,7 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.phoenix.compile.QueryPlan;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -47,6 +49,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -55,18 +58,15 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class ViewIndexIT extends BaseHBaseManagedTimeIT {
+public class ViewIndexIT extends BaseHBaseManagedTimeTableReuseIT {
+
 
-    private String VIEW_NAME = "MY_VIEW";
     private String schemaName="TEST";
     private boolean isNamespaceMapped;
-    private String tableName = schemaName + ".T";
-    private String indexName = "I";
-    private String viewIndexPhysicalTableName;
-    private TableName physicalTableName;
+
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Drop the HBase table metadata for this test to confirm that view index table dropped
@@ -116,16 +116,20 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
     
     public ViewIndexIT(boolean isNamespaceMapped) {
         this.isNamespaceMapped = isNamespaceMapped;
-        this.physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
-        this.viewIndexPhysicalTableName = this.physicalTableName.getNameAsString();
     }
 
     @Test
     public void testDeleteViewIndexSequences() throws Exception {
+        String tableName = schemaName + "." + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String VIEW_NAME = "VIEW_" + generateRandomString();
+        TableName physicalTableName = SchemaUtil.getPhysicalTableName(tableName.getBytes(), isNamespaceMapped);
+        String viewIndexPhysicalTableName = physicalTableName.getNameAsString();
+        String viewName = schemaName + "." + VIEW_NAME;
+
         createBaseTable(tableName, false, null, null);
         Connection conn1 = getConnection();
         Connection conn2 = getConnection();
-        String viewName = schemaName + "." + VIEW_NAME;
         conn1.createStatement().execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName);
         conn1.createStatement().execute("CREATE INDEX " + indexName + " ON " + viewName + " (v1)");
         conn2.createStatement().executeQuery("SELECT * FROM " + tableName).next();
@@ -145,19 +149,21 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
         conn1.createStatement().execute("DROP TABLE "+ tableName);
         admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
         assertFalse("View index table should be deleted.", admin.tableExists(TableName.valueOf(viewIndexPhysicalTableName)));
-        rs = conn2.createStatement().executeQuery("SELECT "
-                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
-                + PhoenixDatabaseMetaData.SEQUENCE_NAME
-                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
-        assertFalse("View index sequences should be deleted.", rs.next());
+        String sequenceName = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("a"), isNamespaceMapped);
+        String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceMapped);
+        verifySequence(null, sequenceName, sequenceSchemaName, false);
+
     }
     
     @Test
     public void testMultiTenantViewLocalIndex() throws Exception {
-        createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, true, null, null);
+        String tableName =  generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String VIEW_NAME = "VIEW_" + generateRandomString();
+        createBaseTable(tableName, true, null, null);
         Connection conn = DriverManager.getConnection(getUrl());
         PreparedStatement stmt = conn.prepareStatement(
-                "UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME
+                "UPSERT INTO " + tableName
                 + " VALUES(?,?,?,?,?)");
         stmt.setString(1, "10");
         stmt.setString(2, "a");
@@ -178,16 +184,16 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
         props.setProperty("TenantId", "10");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute("CREATE VIEW " + VIEW_NAME
-                + " AS select * from " + TestUtil.DEFAULT_DATA_TABLE_NAME);
+                + " AS select * from " + tableName);
         conn1.createStatement().execute("CREATE LOCAL INDEX "
-                + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON "
+                + indexName + " ON "
                 + VIEW_NAME + "(v2)");
         conn1.commit();
         
         String sql = "SELECT * FROM " + VIEW_NAME + " WHERE v2 = 100";
         ResultSet rs = conn1.prepareStatement("EXPLAIN " + sql).executeQuery();
         assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,'10',100]\n" +
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,'10',100]\n" +
                 "    SERVER FILTER BY FIRST KEY ONLY\n" +
                 "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
         rs = conn1.prepareStatement(sql).executeQuery();
@@ -197,9 +203,9 @@ public class ViewIndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreatingIndexOnGlobalView() throws Exception {
-        String baseTable = "testCreatingIndexOnGlobalView".toUpperCase();
-        String globalView = "globalView".toUpperCase();
-        String globalViewIdx = "globalView_idx".toUpperCase();
+        String baseTable =  generateRandomString();
+        String globalView = generateRandomString();
+        String globalViewIdx =  generateRandomString();
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE TABLE " + baseTable + " (TENANT_ID CHAR(15) NOT NULL, PK2 DATE NOT NULL, PK3 INTEGER NOT NULL, KV1 VARCHAR, KV2 VARCHAR, KV3 CHAR(15) CONSTRAINT PK PRIMARY KEY(TENANT_ID, PK2 ROW_TIMESTAMP, PK3)) MULTI_TENANT=true");
             conn.createStatement().execute("CREATE VIEW " + globalView + " AS SELECT * FROM " + baseTable);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
index 558bdcb..9d61247 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/MutableRollbackIT.java
@@ -35,7 +35,7 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
@@ -52,28 +52,16 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
+public class MutableRollbackIT extends BaseHBaseManagedTimeTableReuseIT {
 	
 	private final boolean localIndex;
-	private String tableName1;
-    private String indexName1;
-    private String fullTableName1;
-    private String tableName2;
-    private String indexName2;
-    private String fullTableName2;
 
 	public MutableRollbackIT(boolean localIndex) {
 		this.localIndex = localIndex;
-		this.tableName1 = TestUtil.DEFAULT_DATA_TABLE_NAME + "_1_";
-        this.indexName1 = "IDX1";
-        this.fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
-        this.tableName2 = TestUtil.DEFAULT_DATA_TABLE_NAME + "_2_";
-        this.indexName2 = "IDX2";
-        this.fullTableName2 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName2);
 	}
 	
 	@BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
@@ -83,14 +71,18 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
 	
 	@Parameters(name="localIndex = {0}")
     public static Collection<Boolean> data() {
-        return Arrays.asList(new Boolean[] {     
-                 false, true  
-           });
+        return Arrays.asList(new Boolean[] { false, true});
     }
 	
     public void testRollbackOfUncommittedExistingKeyValueIndexUpdate() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName1 = "TBL1_" + generateRandomString();
+        String indexName1 = "IDX1_" + generateRandomString();
+        String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
+        String tableName2 = "TBL2_" + generateRandomString();
+        String indexName2 = "IDX2_" + generateRandomString();
+        String fullTableName2 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName2);
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -190,7 +182,7 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             stmt.executeUpdate("upsert into " + fullTableName2 + " values('a', 'b', 'c')");
             conn.commit();
 
-            assertDataAndIndexRows(stmt);
+            assertDataAndIndexRows(stmt, fullTableName1, fullTableName2, indexName1);
             stmt.executeUpdate("delete from " + fullTableName1 + " where  k='x'");
             stmt.executeUpdate("delete from " + fullTableName2 + " where  v1='b'");
             
@@ -209,7 +201,7 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             conn.rollback();
-            assertDataAndIndexRows(stmt);
+            assertDataAndIndexRows(stmt, fullTableName1, fullTableName2, indexName1);
         } finally {
             conn.close();
         }
@@ -217,6 +209,12 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
 
 	@Test
     public void testRollbackOfUncommittedExistingRowKeyIndexUpdate() throws Exception {
+        String tableName1 = "TBL1_" + generateRandomString();
+        String indexName1 = "IDX1_" + generateRandomString();
+        String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
+        String tableName2 = "TBL2_" + generateRandomString();
+        String indexName2 = "IDX2_" + generateRandomString();
+        String fullTableName2 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName2);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
@@ -255,9 +253,8 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             
             stmt.executeUpdate("upsert into " + fullTableName1 + " values('x', 'z', 'a')");
             stmt.executeUpdate("upsert into " + fullTableName2 + " values('a', 'b', 'c')");
-            
-            assertDataAndIndexRows(stmt);
-            
+
+            assertDataAndIndexRows(stmt, fullTableName1, fullTableName2, indexName1);
             conn.rollback();
             
             //assert original row exists in fullTableName1
@@ -288,8 +285,7 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             stmt.executeUpdate("upsert into " + fullTableName2 + " values('a', 'b', 'c')");
             conn.commit();
 
-            assertDataAndIndexRows(stmt);
-            stmt.executeUpdate("delete from " + fullTableName1 + " where  k='x'");
+            assertDataAndIndexRows(stmt, fullTableName1, fullTableName2, indexName1);            stmt.executeUpdate("delete from " + fullTableName1 + " where  k='x'");
             stmt.executeUpdate("delete from " + fullTableName2 + " where  v1='b'");
             
             //assert no rows exists in fullTableName1
@@ -307,7 +303,7 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             conn.rollback();
-            assertDataAndIndexRows(stmt);
+            assertDataAndIndexRows(stmt, fullTableName1, fullTableName2, indexName1);
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             if(localIndex) {
                 dropTable(phoenixConn.getQueryServices().getAdmin(), conn, fullTableName1);
@@ -318,7 +314,7 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
         }
     }
 	
-    private void assertDataAndIndexRows(Statement stmt) throws SQLException, IOException {
+    private void assertDataAndIndexRows(Statement stmt, String fullTableName1, String fullTableName2, String indexName1) throws SQLException, IOException {
         ResultSet rs;
         //assert new covered row key value exists in fullTableName1
         rs = stmt.executeQuery("select /*+ NO_INDEX */ k, v1, v2 from " + fullTableName1);
@@ -356,6 +352,10 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
     public void testMultiRollbackOfUncommittedExistingRowKeyIndexUpdate() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName1 = "TBL1_" + generateRandomString();
+        String indexName1 = "IDX1_" + generateRandomString();
+        String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
+        String tableName2 = "TBL2_" + generateRandomString();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -458,6 +458,10 @@ public class MutableRollbackIT extends BaseHBaseManagedTimeIT {
     public void testCheckpointAndRollback() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName1 = "TBL1_" + generateRandomString();
+        String indexName1 = "IDX1_" + generateRandomString();
+        String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
+        String tableName2 = "TBL2_" + generateRandomString();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
index f3fb46a..963263f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/RollbackIT.java
@@ -31,7 +31,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -47,24 +47,18 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class RollbackIT extends BaseHBaseManagedTimeIT {
+public class RollbackIT extends BaseHBaseManagedTimeTableReuseIT {
 	
 	private final boolean localIndex;
 	private final boolean mutable;
-	private final String tableName;
-    private final String indexName;
-    private final String fullTableName;
 
 	public RollbackIT(boolean localIndex, boolean mutable) {
 		this.localIndex = localIndex;
 		this.mutable = mutable;
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME;
-		this.indexName = "IDX";
-		this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 	}
 	
 	@BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
@@ -82,6 +76,9 @@ public class RollbackIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testRollbackOfUncommittedKeyValueIndexInsert() throws Exception {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
@@ -127,6 +124,9 @@ public class RollbackIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IDX_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try {
             Statement stmt = conn.createStatement();
             stmt.execute("CREATE TABLE " + fullTableName + "(k VARCHAR, v1 VARCHAR, v2 VARCHAR, CONSTRAINT pk PRIMARY KEY (v1, v2))"+(!mutable? " IMMUTABLE_ROWS=true" : ""));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
index 06e63b2..29f3758 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
@@ -58,7 +58,7 @@ public class SaltedTableIT extends BaseClientManagedTimeIT {
         // 4abc123jkl444
         try {
             // Upsert with no column specifies.
-            ensureTableCreated(getUrl(), TABLE_WITH_SALTING, splits, ts-2);
+            ensureTableCreated(getUrl(), TABLE_WITH_SALTING, TABLE_WITH_SALTING, splits, ts-2);
             String query = "UPSERT INTO " + TABLE_WITH_SALTING + " VALUES(?,?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.setInt(1, 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
index b21655a..a7ac817 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
@@ -31,7 +31,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
index e94ae3a..e3688ca 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableVarLengthRowKeyIT.java
@@ -29,7 +29,6 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/iterate/PhoenixQueryTimeoutIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/iterate/PhoenixQueryTimeoutIT.java b/phoenix-core/src/it/java/org/apache/phoenix/iterate/PhoenixQueryTimeoutIT.java
index 6bf3790..7a93333 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/iterate/PhoenixQueryTimeoutIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/iterate/PhoenixQueryTimeoutIT.java
@@ -24,11 +24,8 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.HashSet;
 import java.util.Properties;
-import java.util.Set;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorIT.java
index dd384cc..f065f28 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/iterate/RoundRobinResultIteratorIT.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.StatementContext;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
@@ -58,12 +58,12 @@ import org.junit.Test;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
+public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final int NUM_SALT_BUCKETS = 4; 
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(32));
@@ -78,7 +78,7 @@ public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testRoundRobinAfterTableSplit() throws Exception {
-        String tableName = "ROUNDROBINSPLIT";
+        String tableName = generateRandomString();
         byte[] tableNameBytes = Bytes.toBytes(tableName);
         int numRows = setupTableForSplit(tableName);
         Connection conn = DriverManager.getConnection(getUrl());
@@ -128,7 +128,7 @@ public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
     }
 
     private void testSelectAllRowsWithDifferentFetchSizes(boolean salted) throws Exception {
-        String tableName = "ALLROWS" + (salted ? "_SALTED" : "_UNSALTED");
+        String tableName = generateRandomString();
         int numRows = 9;
         Set<String> expectedKeys = Collections.unmodifiableSet(createTableAndInsertRows(tableName, numRows, salted, false));
         Connection conn = DriverManager.getConnection(getUrl());
@@ -152,7 +152,7 @@ public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
     }
 
     private void testSelectRowsWithFilterAndDifferentFetchSizes(boolean salted) throws Exception {
-        String tableName = "ROWSWITHFILTER" + (salted ? "_SALTED" : "_UNSALTED");
+        String tableName = generateRandomString();
         int numRows = 6;
         Set<String> insertedKeys = createTableAndInsertRows(tableName, numRows, salted, false);
         Connection conn = DriverManager.getConnection(getUrl());
@@ -205,7 +205,7 @@ public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testFetchSizesAndRVCExpression() throws Exception {
-        String tableName = "RVCTest";
+        String tableName = generateRandomString();
         Set<String> insertedKeys = Collections.unmodifiableSet(createTableAndInsertRows(tableName, 4, false, false));
         Connection conn = DriverManager.getConnection(getUrl());
         PreparedStatement stmt = conn.prepareStatement("SELECT K FROM " + tableName + " WHERE (K, V)  > (?, ?)");
@@ -280,10 +280,14 @@ public class RoundRobinResultIteratorIT extends BaseHBaseManagedTimeIT {
         int insertedRowsA = 10;
         int insertedRowsB = 5;
         int insertedRowsC = 7;
-        Set<String> keySetA = createTableAndInsertRows("TABLEA", insertedRowsA, true, true);
-        Set<String> keySetB = createTableAndInsertRows("TABLEB", insertedRowsB, true, true);
-        Set<String> keySetC = createTableAndInsertRows("TABLEC", insertedRowsC, false, true);
-        String query = "SELECT K FROM TABLEA UNION ALL SELECT K FROM TABLEB UNION ALL SELECT K FROM TABLEC";
+        String baseTableName = generateRandomString();
+        String tableA = "TABLEA" + baseTableName;
+        String tableB = "TABLEB" + baseTableName;
+        String tableC = "TABLEC" + baseTableName;
+        Set<String> keySetA = createTableAndInsertRows(tableA, insertedRowsA, true, true);
+        Set<String> keySetB = createTableAndInsertRows(tableB, insertedRowsB, true, true);
+        Set<String> keySetC = createTableAndInsertRows(tableC, insertedRowsC, false, true);
+        String query = "SELECT K FROM " + tableA + " UNION ALL SELECT K FROM " + tableB + " UNION ALL SELECT K FROM " + tableC;
         Connection conn = DriverManager.getConnection(getUrl());
         PreparedStatement stmt = conn.prepareStatement(query);
         stmt.setFetchSize(2); // force parallel fetch of scanner cache

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index 20a3c48..00de6b0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -37,7 +37,7 @@ import java.sql.SQLException;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.query.ConnectionQueryServices;
@@ -61,12 +61,12 @@ import com.google.common.collect.Maps;
  * Verifies the number of rpcs calls from {@link MetaDataClient} updateCache() 
  * for transactional and non-transactional tables.
  */
-public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
+public class UpdateCacheIT extends BaseHBaseManagedTimeTableReuseIT {
 	
 	public static final int NUM_MILLIS_IN_DAY = 86400000;
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
@@ -84,28 +84,30 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testUpdateCacheForTxnTable() throws Exception {
         String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + TRANSACTIONAL_DATA_TABLE;
-        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
+        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE, TRANSACTIONAL_DATA_TABLE);
         helpTestUpdateCache(fullTableName, null, new int[] {1, 1});
     }
     
     @Test
     public void testUpdateCacheForNonTxnTable() throws Exception {
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
-        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+        String tableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + tableName;
+        ensureTableCreated(getUrl(), tableName, MUTABLE_INDEX_DATA_TABLE);
         helpTestUpdateCache(fullTableName, null, new int[] {1, 3});
     }
 	
     @Test
     public void testUpdateCacheForNonTxnSystemTable() throws Exception {
-        String fullTableName = QueryConstants.SYSTEM_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        String fullTableName = QueryConstants.SYSTEM_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + generateRandomString();
         setupSystemTable(fullTableName);
         helpTestUpdateCache(fullTableName, null, new int[] {0, 0});
     }
     
     @Test
     public void testUpdateCacheForNeverUpdatedTable() throws Exception {
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
-        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+        String tableName = generateRandomString();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + tableName;
+        ensureTableCreated(getUrl(), tableName, MUTABLE_INDEX_DATA_TABLE);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute(
@@ -116,7 +118,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpdateCacheForAlwaysUpdatedTable() throws Exception {
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=always");
@@ -126,7 +128,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpdateCacheForTimeLimitedUpdateTable() throws Exception {
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=" + 10000);
@@ -138,7 +140,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpdateCacheForChangingUpdateTable() throws Exception {
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=never");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
index 04f751b..44e3115 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
@@ -30,7 +30,7 @@ public class UpdateCacheWithScnIT extends BaseClientManagedTimeIT {
 	public void testUpdateCacheWithScn() throws Exception {
         long ts = nextTimestamp();
         String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
-        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE, ts);
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE, MUTABLE_INDEX_DATA_TABLE, ts);
         // FIXME: given that the scn is advancing in the test, why aren't there more RPCs?
 		UpdateCacheIT.helpTestUpdateCache(fullTableName, ts+2, new int[] {1, 1});
 	}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/trace/BaseTracingTestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/trace/BaseTracingTestIT.java b/phoenix-core/src/it/java/org/apache/phoenix/trace/BaseTracingTestIT.java
index 96e1646..66f3e87 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/trace/BaseTracingTestIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/trace/BaseTracingTestIT.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.impl.ExposedMetricCounterLong;
 import org.apache.hadoop.metrics2.impl.ExposedMetricsRecordImpl;
 import org.apache.hadoop.metrics2.lib.ExposedMetricsInfoImpl;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.metrics.MetricInfo;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -51,7 +51,7 @@ import org.junit.Before;
  * connections, as well as any supporting utils.
  */
 
-public class BaseTracingTestIT extends BaseHBaseManagedTimeIT {
+public class BaseTracingTestIT extends BaseHBaseManagedTimeTableReuseIT {
     @Before
     public void resetTracingTableIfExists() throws Exception {
         Connection conn = getConnectionWithoutTracing();


[37/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
index ce073f6..16ad47f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
@@ -32,6 +32,7 @@ import static org.apache.phoenix.util.TestUtil.ROW7;
 import static org.apache.phoenix.util.TestUtil.ROW8;
 import static org.apache.phoenix.util.TestUtil.ROW9;
 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;
@@ -73,7 +74,7 @@ public class DerivedTableIT extends BaseClientManagedTimeIT {
     @Before
     public void initTable() throws Exception {
          ts = nextTimestamp();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         if (indexDDL != null && indexDDL.length > 0) {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
index 5f18a1c..38dfa8c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DisableLocalIndexIT.java
@@ -43,9 +43,9 @@ import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class DisableLocalIndexIT extends BaseHBaseManagedTimeIT {
+public class DisableLocalIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Must update config before starting server
@@ -57,7 +57,11 @@ public class DisableLocalIndexIT extends BaseHBaseManagedTimeIT {
     public void testDisabledLocalIndexes() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(true);
-        String tableName = "DISABLE_LOCAL_INDEX_TEST";
+        String baseName = generateRandomString();
+        String tableName = baseName+ "_TABLE";
+        String viewName = baseName + "_VIEW";
+        String indexName1 = baseName + "_INDEX1";
+        String indexName2 = baseName + "_INDEX2";
         conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2)) MULTI_TENANT=true");
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t1','x')");
         conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('t2','y')");
@@ -78,18 +82,18 @@ public class DisableLocalIndexIT extends BaseHBaseManagedTimeIT {
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "t1");
         Connection tsconn = DriverManager.getConnection(getUrl(), props);
         
-        tsconn.createStatement().execute("CREATE VIEW A.BAR(V1 VARCHAR) AS SELECT * FROM " + tableName);
-        tsconn.createStatement().execute("CREATE INDEX I1 ON A.BAR(V1)");
+        tsconn.createStatement().execute("CREATE VIEW " + viewName + "(V1 VARCHAR) AS SELECT * FROM " + tableName);
+        tsconn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + viewName + "(V1)");
         tsconn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(MetaDataUtil.VIEW_INDEX_TABLE_PREFIX + tableName));
 
         try {
-            conn.createStatement().execute("CREATE LOCAL INDEX I2 ON " + tableName + "(k2)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + indexName2 + " ON " + tableName + "(k2)");
             fail("Should not allow creation of local index");
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.UNALLOWED_LOCAL_INDEXES.getErrorCode(), e.getErrorCode());
         }
         try {
-            tsconn.createStatement().execute("CREATE LOCAL INDEX I2 ON A.BAR(k2, v1)");
+            tsconn.createStatement().execute("CREATE LOCAL INDEX " + indexName2 + " ON " + viewName + "(k2, v1)");
             fail("Should not allow creation of local index");
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.UNALLOWED_LOCAL_INDEXES.getErrorCode(), e.getErrorCode());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
index 500998b..4484fc1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
@@ -287,9 +287,9 @@ public class DistinctCountIT extends BaseClientManagedTimeIT {
     
     protected static void initATableValues(String tenantId1, String tenantId2, byte[][] splits, Date date, Long ts) throws Exception {
         if (ts == null) {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits);
+            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits);
         } else {
-            ensureTableCreated(getUrl(), ATABLE_NAME, splits, ts-2);
+            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits, ts-2);
         }
         
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
index 31b0dd7..e30c474 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
@@ -59,31 +59,25 @@ import org.junit.Test;
  */
 
 
-public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
+public class DynamicColumnIT extends BaseHBaseManagedTimeTableReuseIT {
     private static final byte[] FAMILY_NAME_A = Bytes.toBytes(SchemaUtil.normalizeIdentifier("A"));
     private static final byte[] FAMILY_NAME_B = Bytes.toBytes(SchemaUtil.normalizeIdentifier("B"));
 
+    private static String tableName = "TESTTBL";
+
     @BeforeClass
     public static void doBeforeTestSetup() throws Exception {
         try (PhoenixConnection pconn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class)) {
             ConnectionQueryServices services = pconn.getQueryServices();
             try (HBaseAdmin admin = services.getAdmin()) {
-                HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(HBASE_DYNAMIC_COLUMNS));
+                HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TESTTBL"));
                 htd.addFamily(new HColumnDescriptor(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES));
                 htd.addFamily(new HColumnDescriptor(FAMILY_NAME_A));
                 htd.addFamily(new HColumnDescriptor(FAMILY_NAME_B));
                 admin.createTable(htd);
             }
-        }
-    }
-    
-    @SuppressWarnings("deprecation")
-    @Before
-    public void createTable() throws Exception {
-        try (PhoenixConnection pconn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class)) {
-            ConnectionQueryServices services = pconn.getQueryServices();
-            HTableInterface hTable = services.getTable(Bytes.toBytes(HBASE_DYNAMIC_COLUMNS));
-            try {
+
+            try (HTableInterface hTable = services.getTable(Bytes.toBytes(tableName))) {
                 // Insert rows using standard HBase mechanism with standard HBase "types"
                 List<Row> mutations = new ArrayList<Row>();
                 byte[] dv = Bytes.toBytes("DV");
@@ -93,7 +87,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
                 byte[] f2v1 = Bytes.toBytes("F2V1");
                 byte[] f2v2 = Bytes.toBytes("F2V2");
                 byte[] key = Bytes.toBytes("entry1");
-    
+
                 Put put = new Put(key);
                 put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, dv, Bytes.toBytes("default"));
                 put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, first, Bytes.toBytes("first"));
@@ -102,15 +96,14 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
                 put.add(FAMILY_NAME_B, f2v1, Bytes.toBytes("f2value1"));
                 put.add(FAMILY_NAME_B, f2v2, Bytes.toBytes("f2value2"));
                 mutations.add(put);
-    
+
                 hTable.batch(mutations);
-    
+
                 // Create Phoenix table after HBase table was created through the native APIs
                 // The timestamp of the table creation must be later than the timestamp of the data
-                ensureTableCreated(getUrl(), HBASE_DYNAMIC_COLUMNS);
-            } finally {
-                hTable.close();
+                ensureTableCreated(getUrl(), tableName, HBASE_DYNAMIC_COLUMNS);
             }
+
         }
     }
 
@@ -119,7 +112,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
      */
     @Test
     public void testDynamicColums() throws Exception {
-        String query = "SELECT * FROM HBASE_DYNAMIC_COLUMNS (DV varchar)";
+        String query = "SELECT * FROM " + tableName + " (DV varchar)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -143,7 +136,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
      */
     @Test
     public void testDynamicColumsFamily() throws Exception {
-        String query = "SELECT * FROM HBASE_DYNAMIC_COLUMNS (DV varchar,B.F2V2 varchar)";
+        String query = "SELECT * FROM " + tableName + " (DV varchar,B.F2V2 varchar)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -169,7 +162,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testDynamicColumsSpecificQuery() throws Exception {
-        String query = "SELECT entry,F2V2 FROM HBASE_DYNAMIC_COLUMNS (DV varchar,B.F2V2 varchar)";
+        String query = "SELECT entry,F2V2 FROM " + tableName + " (DV varchar,B.F2V2 varchar)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -189,7 +182,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
      */
     @Test(expected = ColumnAlreadyExistsException.class)
     public void testAmbiguousStaticSelect() throws Exception {
-        String upsertquery = "Select * FROM HBASE_DYNAMIC_COLUMNS(A.F1V1 INTEGER)";
+        String upsertquery = "Select * FROM " + tableName + "(A.F1V1 INTEGER)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -205,7 +198,7 @@ public class DynamicColumnIT extends BaseHBaseManagedTimeIT {
      */
     @Test(expected = ColumnFamilyNotFoundException.class)
     public void testFakeCFDynamicUpsert() throws Exception {
-        String upsertquery = "Select * FROM HBASE_DYNAMIC_COLUMNS(fakecf.DynCol VARCHAR)";
+        String upsertquery = "Select * FROM " + tableName + "(fakecf.DynCol VARCHAR)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
index ceafc5b..d32b900 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EncodeFunctionIT.java
@@ -35,7 +35,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Lists;
 
-public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
+public class EncodeFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     /**
      * Helper to test ENCODE function
@@ -46,13 +46,13 @@ public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
      *            name of column to query
      * @param sortOrder
      *            sort order of the pk column
-     * @param expectedOutput
+     * @param expectedOutputList
      *            expected output of ENCODE function
      */
-    private void testEncodeHelper(Connection conn, String colName, List<String> expectedOutputList, String sortOrder)
+    private void testEncodeHelper(Connection conn, String tableName, String colName, List<String> expectedOutputList, String sortOrder)
         throws Exception {
         for (int id = 0; id < expectedOutputList.size(); ++id) {
-            String sql = String.format("SELECT ENCODE(%s, 'base62') FROM TEST_TABLE_%s WHERE id=?", colName, sortOrder);
+            String sql = String.format("SELECT ENCODE(%s, 'base62') FROM " + tableName + "_%s WHERE id=?", colName, sortOrder);
             PreparedStatement stmt = conn.prepareStatement(sql);
             stmt.setInt(1, id);
 
@@ -74,9 +74,9 @@ public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
      *            expected output of ENCODE function
      */
     private void testEncode(Connection conn, List<Object> inputList, List<String> expectedOutputList) throws Exception {
-        TestUtil.initTables(conn, "BIGINT", inputList);
-        testEncodeHelper(conn, "pk", expectedOutputList, "ASC");
-        testEncodeHelper(conn, "pk", expectedOutputList, "DESC");
+        String tableName = TestUtil.initTables(conn, "BIGINT", inputList);
+        testEncodeHelper(conn, tableName, "pk", expectedOutputList, "ASC");
+        testEncodeHelper(conn, tableName, "pk", expectedOutputList, "DESC");
     }
 
     @Test
@@ -89,45 +89,48 @@ public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testEncodeNullInput() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        TestUtil.initTables(conn, "BIGINT", Collections.<Object> singletonList(0l));
-        testEncodeHelper(conn, "kv", Collections.<String> singletonList(null), "ASC");
-        testEncodeHelper(conn, "kv", Collections.<String> singletonList(null), "DESC");
+        String tableName = TestUtil.initTables(conn, "BIGINT", Collections.<Object> singletonList(0l));
+        testEncodeHelper(conn, tableName, "kv", Collections.<String> singletonList(null), "ASC");
+        testEncodeHelper(conn, tableName, "kv", Collections.<String> singletonList(null), "DESC");
     }
 
     @Test
     public void testUpperCaseEncodingType() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE TEST_TABLE ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
 
         conn.createStatement().execute(ddl);
-        PreparedStatement ps = conn.prepareStatement("UPSERT INTO TEST_TABLE (pk) VALUES (?)");
+        PreparedStatement ps = conn.prepareStatement("UPSERT INTO " + tableName + " (pk) VALUES (?)");
         ps.setString(1, "1");
 
         ps.execute();
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM TEST_TABLE WHERE pk = ENCODE(1, 'BASE62')");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName + " WHERE pk = ENCODE(1, 'BASE62')");
         assertTrue(rs.next());
     }
 
     @Test
     public void testNullEncodingType() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE TEST_TABLE ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
         conn.createStatement().execute(ddl);
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM TEST_TABLE WHERE pk = ENCODE(1, NULL)");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName + " WHERE pk = ENCODE(1, NULL)");
         assertFalse(rs.next());
     }
 
     @Test
     public void testUnsupportedEncodingType() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE TEST_TABLE ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " ( pk VARCHAR(10) NOT NULL CONSTRAINT PK PRIMARY KEY (pk))";
         conn.createStatement().execute(ddl);
 
         try {
-            conn.createStatement().executeQuery("SELECT * FROM TEST_TABLE WHERE pk = ENCODE(1, 'HEX')");
+            conn.createStatement().executeQuery("SELECT * FROM " + tableName + " WHERE pk = ENCODE(1, 'HEX')");
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.ILLEGAL_DATA.getErrorCode(), e.getErrorCode());
@@ -137,13 +140,14 @@ public class EncodeFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testInvalidEncodingType() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
         String ddl =
-            "CREATE TABLE test_table ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
+            "CREATE TABLE " + tableName + " ( some_column BINARY(12) NOT NULL CONSTRAINT PK PRIMARY KEY (some_column))";
         conn.createStatement().execute(ddl);
 
         try {
             conn.createStatement().executeQuery(
-                "SELECT * FROM test_table WHERE some_column = ENCODE(1, 'invalidEncodingFormat')");
+                "SELECT * FROM " + tableName + " WHERE some_column = ENCODE(1, 'invalidEncodingFormat')");
             fail();
         } catch (SQLException e) {
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
index 4355036..3b40d51 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
@@ -33,8 +33,8 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
 
 
-public class EvaluationOfORIT extends BaseHBaseManagedTimeIT{
-		
+public class EvaluationOfORIT extends BaseHBaseManagedTimeTableReuseIT{
+
     @Test
     public void testFalseOrFalse() throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -48,15 +48,16 @@ public class EvaluationOfORIT extends BaseHBaseManagedTimeIT{
 	@Test
 	public void testPKOrNotPKInOREvaluation() throws SQLException {
 	    Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-	    Connection conn = DriverManager.getConnection(getUrl(), props);	    
+	    Connection conn = DriverManager.getConnection(getUrl(), props);
+        String tableName = generateRandomString();
 	    conn.setAutoCommit(false);
 	    
-            String create = "CREATE TABLE DIE ( ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR(50))";
+            String create = "CREATE TABLE " + tableName + " ( ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR(50))";
             PreparedStatement createStmt = conn.prepareStatement(create);
             createStmt.execute();
             PreparedStatement stmt = conn.prepareStatement(
                     "upsert into " +
-                    "DIE VALUES (?, ?)");
+                     tableName + " VALUES (?, ?)");
 
             stmt.setInt(1, 1);
             stmt.setString(2, "Tester1");
@@ -163,7 +164,7 @@ public class EvaluationOfORIT extends BaseHBaseManagedTimeIT{
             stmt.execute();		   
             conn.commit();
             
-            String select = "Select * from DIE where ID=6 or Name between 'Tester1' and 'Tester3'";
+            String select = "Select * from " + tableName + " where ID=6 or Name between 'Tester1' and 'Tester3'";
             ResultSet rs;
             rs = conn.createStatement().executeQuery(select);
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
index 1b41bdd..41d3ee7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExecuteStatementsIT.java
@@ -48,33 +48,34 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
 
 
-public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
+public class ExecuteStatementsIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @Test
     public void testExecuteStatements() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String ptsdbTableName = generateRandomString();
         String statements = 
-            "create table if not exists " + ATABLE_NAME + // Shouldn't error out b/c of if not exists clause
+            "create table if not exists " + tableName + // Shouldn't error out b/c of if not exists clause
             "   (organization_id char(15) not null, \n" + 
             "    entity_id char(15) not null,\n" + 
             "    a_string varchar(100),\n" + 
             "    b_string varchar(100)\n" +
             "    CONSTRAINT pk PRIMARY KEY (organization_id,entity_id));\n" + 
-            "create table " + PTSDB_NAME +
+            "create table " + ptsdbTableName +
             "   (inst varchar null,\n" + 
             "    host varchar null,\n" + 
             "    date date not null,\n" + 
             "    val decimal\n" +
             "    CONSTRAINT pk PRIMARY KEY (inst,host,date))\n" +
             "    split on (?,?,?);\n" +
-            "alter table " + PTSDB_NAME + " add if not exists val decimal;\n" +  // Shouldn't error out b/c of if not exists clause
-            "alter table " + PTSDB_NAME + " drop column if exists blah;\n" +  // Shouldn't error out b/c of if exists clause
+            "alter table " + ptsdbTableName + " add if not exists val decimal;\n" +  // Shouldn't error out b/c of if not exists clause
+            "alter table " + ptsdbTableName + " drop column if exists blah;\n" +  // Shouldn't error out b/c of if exists clause
             "drop table if exists FOO.BAR;\n" + // Shouldn't error out b/c of if exists clause
-            "UPSERT INTO " + PTSDB_NAME + "(date, val, host) " +
-            "    SELECT current_date(), x_integer+2, entity_id FROM ATABLE WHERE a_integer >= ?;" +
-            "UPSERT INTO " + PTSDB_NAME + "(date, val, inst)\n" +
-            "    SELECT date+1, val*10, host FROM " + PTSDB_NAME + ";";
+            "UPSERT INTO " + ptsdbTableName + "(date, val, host) " +
+            "    SELECT current_date(), x_integer+2, entity_id FROM " + tableName + " WHERE a_integer >= ?;" +
+            "UPSERT INTO " + ptsdbTableName + "(date, val, inst)\n" +
+            "    SELECT date+1, val*10, host FROM " + ptsdbTableName + ";";
         
         Date now = new Date(System.currentTimeMillis());
         Connection conn = DriverManager.getConnection(getUrl());
@@ -84,7 +85,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
         assertEquals(7, nStatements);
 
         Date then = new Date(System.currentTimeMillis() + QueryConstants.MILLIS_IN_DAY);
-        String query = "SELECT host,inst, date,val FROM " + PTSDB_NAME + " where inst is not null";
+        String query = "SELECT host,inst, date,val FROM " + ptsdbTableName + " where inst is not null";
         PreparedStatement statement = conn.prepareStatement(query);
         
         ResultSet rs = statement.executeQuery();
@@ -119,7 +120,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testCharPadding() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String tableName = "foo";
+        String tableName = generateRandomString();
         String rowKey = "hello"; 
         String testString = "world";
         String query = "create table " + tableName +
@@ -142,10 +143,11 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
         statement.setString(3, testString);
         statement.execute();       
         conn.commit();
-        
-        ensureTableCreated(getUrl(),BTABLE_NAME, null, nextTimestamp()-2);
+
+        String btableName = generateRandomString();
+        ensureTableCreated(getUrl(),btableName, BTABLE_NAME, nextTimestamp()-2);
         statement = conn.prepareStatement(
-                "upsert into BTABLE VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)");
+                "upsert into " + btableName + " VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)");
         statement.setString(1, "abc");
         statement.setString(2, "xyz");
         statement.setString(3, "x");
@@ -197,10 +199,10 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
             
             // test upsert statement with padding
             String tenantId = getOrganizationId();
-            initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+            String atableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
             
             upsert = "UPSERT INTO " + tableName + "(a_id, a_string, b_string) " +
-                    "SELECT A_INTEGER, A_STRING, B_STRING FROM ATABLE WHERE a_string = ?";
+                    "SELECT A_INTEGER, A_STRING, B_STRING FROM " + atableName + " WHERE a_string = ?";
             
             statement = conn.prepareStatement(upsert);
             statement.setString(1, A_VALUE);
@@ -249,7 +251,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
             try {
                 
                 upsert = "UPSERT INTO " + tableName + "(a_id, a_string, b_string) " +
-                        "SELECT x_integer, organization_id, b_string FROM ATABLE WHERE a_string = ?";
+                        "SELECT x_integer, organization_id, b_string FROM " + atableName + " WHERE a_string = ?";
                 
                 statement = conn.prepareStatement(upsert);
                 statement.setString(1, A_VALUE);
@@ -263,7 +265,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
             try {
                 
                 upsert = "UPSERT INTO " + tableName + "(a_id, a_string, b_string) " +
-                        "SELECT y_integer, a_string, entity_id FROM ATABLE WHERE a_string = ?";
+                        "SELECT y_integer, a_string, entity_id FROM " + atableName + " WHERE a_string = ?";
                 
                 statement = conn.prepareStatement(upsert);
                 statement.setString(1, A_VALUE);
@@ -276,7 +278,7 @@ public class ExecuteStatementsIT extends BaseHBaseManagedTimeIT {
                         
             //where selecting from a CHAR(x) and upserting into a CHAR(y) where x<=y.
             upsert = "UPSERT INTO " + tableName + "(a_id, a_string, b_string) " +
-                    "SELECT a_integer, e_string, a_id FROM BTABLE";
+                    "SELECT a_integer, e_string, a_id FROM " + btableName ;
             
             statement = conn.prepareStatement(upsert);
             rowsInserted = statement.executeUpdate();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExpFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExpFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExpFunctionEnd2EndIT.java
index a723c6b..88bf763 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExpFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExpFunctionEnd2EndIT.java
@@ -34,21 +34,26 @@ import org.junit.Test;
 /**
  * End to end tests for {@link ExpFunction}
  */
-public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
+public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final String KEY = "key";
     private static final double ZERO = 1e-8;
+    private String signedTableName;
+    private String unsignedTableName;
 
     @Before
     public void initTable() throws Exception {
         Connection conn = null;
         PreparedStatement stmt = null;
+        signedTableName = generateRandomString();
+        unsignedTableName = generateRandomString();
+
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE testSigned (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
+            ddl = "CREATE TABLE " + signedTableName + " (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
             conn.createStatement().execute(ddl);
-            ddl = "CREATE TABLE testUnsigned (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
+            ddl = "CREATE TABLE " + unsignedTableName + " (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
             conn.createStatement().execute(ddl);
             conn.commit();
         } finally {
@@ -57,7 +62,8 @@ public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     }
 
     private void updateSignedTable(Connection conn, double data) throws Exception {
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO testSigned VALUES (?, ?, ?, ?, ?, ?, ?)");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + signedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -71,7 +77,7 @@ public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     }
 
     private void updateUnsignedTable(Connection conn, double data) throws Exception {
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO testUnsigned VALUES (?, ?, ?, ?, ?, ?, ?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + unsignedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -86,7 +92,7 @@ public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
 
     private void testSignedNumberSpec(Connection conn, double data) throws Exception {
         updateSignedTable(conn, data);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT EXP(doub),EXP(fl),EXP(inte),EXP(lon),EXP(smalli),EXP(tinyi) FROM testSigned");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT EXP(doub),EXP(fl),EXP(inte),EXP(lon),EXP(smalli),EXP(tinyi) FROM " + signedTableName);
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.exp(d.doubleValue())) < ZERO);
@@ -100,7 +106,7 @@ public class ExpFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
 
     private void testUnsignedNumberSpec(Connection conn, double data) throws Exception {
         updateUnsignedTable(conn, data);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT EXP(doub),EXP(fl),EXP(inte),EXP(lon),EXP(smalli),EXP(tinyi) FROM testUnsigned");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT EXP(doub),EXP(fl),EXP(inte),EXP(lon),EXP(smalli),EXP(tinyi) FROM " + unsignedTableName);
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.exp(d.doubleValue())) < ZERO);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
index 377f6d6..f5c1df2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 
 import java.sql.Connection;
 import java.sql.Date;
@@ -51,7 +52,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(tenantId, getDefaultSplits(tenantId),date, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
@@ -74,7 +75,8 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
     public void testTypeMismatchToDateFunctionBind() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId),null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),null, ts, getUrl());
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -101,8 +103,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(tenantId, getDefaultSplits(tenantId),date, ts);
-        
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -147,8 +148,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(tenantId, getDefaultSplits(tenantId),date, ts);
-        
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
index 11f91cd..26e13eb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
@@ -39,7 +39,7 @@ import org.junit.Test;
 public class FunkyNamesIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(),FUNKY_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(), FUNKY_NAME, FUNKY_NAME,splits, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
index b0524da..86a1f64 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.GROUPBYTEST_NAME;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.getTableName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -40,41 +41,25 @@ import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
 
 
-public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
-
-    private static String GROUPBY1 = "select " +
-            "case when uri LIKE 'Report%' then 'Reports' else 'Other' END category" +
-            ", avg(appcpu) from " + GROUPBYTEST_NAME +
-            " group by category";
-
-    private static String GROUPBY2 = "select " +
-            "case uri when 'Report%' then 'Reports' else 'Other' END category" +
-            ", avg(appcpu) from " + GROUPBYTEST_NAME +
-            " group by appcpu, category";
-
-    private static String GROUPBY3 = "select " +
-            "case uri when 'Report%' then 'Reports' else 'Other' END category" +
-            ", avg(appcpu) from " + GROUPBYTEST_NAME +
-            " group by avg(appcpu), category";
-    
+public class GroupByCaseIT extends BaseHBaseManagedTimeTableReuseIT {
     private static int id;
 
-    private static void initData(Connection conn) throws SQLException {
-        ensureTableCreated(getUrl(), GROUPBYTEST_NAME);
-        insertRow(conn, "Report1", 10);
-        insertRow(conn, "Report2", 10);
-        insertRow(conn, "Report3", 30);
-        insertRow(conn, "Report4", 30);
-        insertRow(conn, "SOQL1", 10);
-        insertRow(conn, "SOQL2", 10);
-        insertRow(conn, "SOQL3", 30);
-        insertRow(conn, "SOQL4", 30);
+    private static void initData(Connection conn, String tableName) throws SQLException {
+        ensureTableCreated(getUrl(), tableName, GROUPBYTEST_NAME);
+        insertRow(conn, tableName, "Report1", 10);
+        insertRow(conn, tableName, "Report2", 10);
+        insertRow(conn, tableName, "Report3", 30);
+        insertRow(conn, tableName, "Report4", 30);
+        insertRow(conn, tableName, "SOQL1", 10);
+        insertRow(conn, tableName, "SOQL2", 10);
+        insertRow(conn, tableName, "SOQL3", 30);
+        insertRow(conn, tableName, "SOQL4", 30);
         conn.commit();
         conn.close();
     }
 
-    private static void insertRow(Connection conn, String uri, int appcpu) throws SQLException {
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + GROUPBYTEST_NAME + "(id, uri, appcpu) values (?,?,?)");
+    private static void insertRow(Connection conn, String tableName, String uri, int appcpu) throws SQLException {
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + tableName + "(id, uri, appcpu) values (?,?,?)");
         statement.setString(1, "id" + id);
         statement.setString(2, uri);
         statement.setInt(3, appcpu);
@@ -86,13 +71,14 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testExpressionInGroupBy() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = " create table tgb_counter(tgb_id integer NOT NULL,utc_date_epoch integer NOT NULL,tgb_name varchar(40),ack_success_count integer" +
+        String tableName = generateRandomString();
+        String ddl = " create table " + tableName + "(tgb_id integer NOT NULL,utc_date_epoch integer NOT NULL,tgb_name varchar(40),ack_success_count integer" +
                 ",ack_success_one_ack_count integer, CONSTRAINT pk_tgb_counter PRIMARY KEY(tgb_id, utc_date_epoch))";
         String query = "SELECT tgb_id, tgb_name, (utc_date_epoch/10)*10 AS utc_epoch_hour,SUM(ack_success_count + ack_success_one_ack_count) AS ack_tx_sum" +
-                " FROM tgb_counter GROUP BY tgb_id, tgb_name, utc_epoch_hour";
+                " FROM " + tableName + " GROUP BY tgb_id, tgb_name, utc_epoch_hour";
 
         createTestTable(getUrl(), ddl);
-        String dml = "UPSERT INTO tgb_counter VALUES(?,?,?,?,?)";
+        String dml = "UPSERT INTO " + tableName + " VALUES(?,?,?,?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setInt(1, 1);
         stmt.setInt(2, 1000);
@@ -128,16 +114,17 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testBooleanInGroupBy() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = " create table bool_gb(id varchar primary key,v1 boolean, v2 integer, v3 integer)";
+        String tableName = generateRandomString();
+        String ddl = " create table " + tableName + "(id varchar primary key,v1 boolean, v2 integer, v3 integer)";
 
         createTestTable(getUrl(), ddl);
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO bool_gb(id,v2,v3) VALUES(?,?,?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + "(id,v2,v3) VALUES(?,?,?)");
         stmt.setString(1, "a");
         stmt.setInt(2, 1);
         stmt.setInt(3, 1);
         stmt.execute();
         stmt.close();
-        stmt = conn.prepareStatement("UPSERT INTO bool_gb VALUES(?,?,?,?)");
+        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?,?)");
         stmt.setString(1, "b");
         stmt.setBoolean(2, false);
         stmt.setInt(3, 2);
@@ -152,7 +139,7 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
 
         String[] gbs = {"v1,v2,v3","v1,v3,v2","v2,v1,v3"};
         for (String gb : gbs) {
-            ResultSet rs = conn.createStatement().executeQuery("SELECT v1, v2, v3 from bool_gb group by " + gb);
+            ResultSet rs = conn.createStatement().executeQuery("SELECT v1, v2, v3 from " + tableName + " group by " + gb);
             assertTrue(rs.next());
             assertEquals(false,rs.getBoolean("v1"));
             assertTrue(rs.wasNull());
@@ -177,9 +164,10 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testScanUri() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        initData(conn);
+        String tableName = generateRandomString();
+        initData(conn, tableName);
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select uri from " + GROUPBYTEST_NAME);
+        ResultSet rs = stmt.executeQuery("select uri from " + tableName);
         assertTrue(rs.next());
         assertEquals("Report1", rs.getString(1));
         assertTrue(rs.next());
@@ -204,9 +192,10 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testCount() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        initData(conn);
+        String tableName = generateRandomString();
+        initData(conn, tableName);
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select count(1) from " + GROUPBYTEST_NAME);
+        ResultSet rs = stmt.executeQuery("select count(1) from " + tableName);
         assertTrue(rs.next());
         assertEquals(8, rs.getInt(1));
         assertFalse(rs.next());
@@ -217,12 +206,27 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testGroupByCase() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        initData(conn);
-        conn.createStatement().executeQuery(GROUPBY1);
-        conn.createStatement().executeQuery(GROUPBY2);
+        String tableName = generateRandomString();
+        String groupBy1 = "select " +
+                "case when uri LIKE 'Report%' then 'Reports' else 'Other' END category" +
+                ", avg(appcpu) from " + tableName +
+                " group by category";
+
+        String groupBy2 = "select " +
+                "case uri when 'Report%' then 'Reports' else 'Other' END category" +
+                ", avg(appcpu) from " + tableName +
+                " group by appcpu, category";
+        
+        String groupBy3 = "select " +
+                "case uri when 'Report%' then 'Reports' else 'Other' END category" +
+                ", avg(appcpu) from " + tableName +
+                " group by avg(appcpu), category";
+        initData(conn, tableName);
+        conn.createStatement().executeQuery(groupBy1);
+        conn.createStatement().executeQuery(groupBy2);
         // TODO: validate query results
         try {
-            conn.createStatement().executeQuery(GROUPBY3);
+            conn.createStatement().executeQuery(groupBy3);
             fail();
         } catch (SQLException e) {
             assertTrue(e.getMessage().contains("Aggregate expressions may not be used in GROUP BY"));
@@ -235,7 +239,9 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testGroupByArray() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE test1(\n" + 
+
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(\n" + 
                 "  a VARCHAR NOT NULL,\n" + 
                 "  b VARCHAR,\n" + 
                 "  c INTEGER,\n" + 
@@ -245,13 +251,13 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
                 "  g BIGINT,\n" + 
                 "  CONSTRAINT pk PRIMARY KEY(a)\n" + 
                 ")");
-        conn.createStatement().execute("UPSERT INTO test1 VALUES('1', 'val', 100, 'a', ARRAY ['b'], 1, 2)");
-        conn.createStatement().execute("UPSERT INTO test1 VALUES('2', 'val', 100, 'a', ARRAY ['b'], 3, 4)");
-        conn.createStatement().execute("UPSERT INTO test1 VALUES('3', 'val', 100, 'a', ARRAY ['b','c'], 5, 6)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('1', 'val', 100, 'a', ARRAY ['b'], 1, 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('2', 'val', 100, 'a', ARRAY ['b'], 3, 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('3', 'val', 100, 'a', ARRAY ['b','c'], 5, 6)");
         conn.commit();
         
         ResultSet rs = conn.createStatement().executeQuery("SELECT c, SUM(f + g) AS sumone, d, e\n" + 
-                "FROM test1\n" + 
+                "FROM " + tableName + "\n" + 
                 "WHERE b = 'val'\n" + 
                 "  AND a IN ('1','2','3')\n" + 
                 "GROUP BY c, d, e\n" + 
@@ -270,7 +276,9 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testGroupByOrderPreserving() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE T (ORGANIZATION_ID char(15) not null, \n" + 
+        String tableName = generateRandomString();
+
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(ORGANIZATION_ID char(15) not null, \n" + 
                 "JOURNEY_ID char(15) not null, \n" + 
                 "DATASOURCE SMALLINT not null, \n" + 
                 "MATCH_STATUS TINYINT not null, \n" + 
@@ -283,13 +291,13 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
                 "    MATCH_STATUS,\n" + 
                 "    EXTERNAL_DATASOURCE_KEY,\n" + 
                 "    ENTITY_ID))");
-        conn.createStatement().execute("UPSERT INTO T VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788888')");
-        conn.createStatement().execute("UPSERT INTO T VALUES('000001111122222', '333334444455555', 0, 0, 'abcd', '666667777788889')");
-        conn.createStatement().execute("UPSERT INTO T VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788899')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788888')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abcd', '666667777788889')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('000001111122222', '333334444455555', 0, 0, 'abc', '666667777788899')");
         conn.commit();
         String query =
                 "SELECT COUNT(1), EXTERNAL_DATASOURCE_KEY As DUP_COUNT\n" + 
-                "    FROM T \n" + 
+                "    FROM " + tableName + " \n" + 
                 "   WHERE JOURNEY_ID='333334444455555' AND \n" + 
                 "                 DATASOURCE=0 AND MATCH_STATUS <= 1 and \n" + 
                 "                 ORGANIZATION_ID='000001111122222' \n" + 
@@ -303,7 +311,7 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
         
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         assertEquals(
-                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T ['000001111122222','333334444455555',0,*] - ['000001111122222','333334444455555',0,1]\n" + 
+                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " ['000001111122222','333334444455555',0,*] - ['000001111122222','333334444455555',0,1]\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [MATCH_STATUS, EXTERNAL_DATASOURCE_KEY]\n" + 
                 "CLIENT FILTER BY COUNT(1) > 1",QueryUtil.getExplainPlan(rs));
@@ -313,21 +321,22 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testGroupByOrderPreservingDescSort() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE GROUP_BY_DESC (k1 char(1) not null, k2 char(1) not null, constraint pk primary key (k1,k2)) split on ('ac','jc','nc')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 'a')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 'b')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 'c')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 'd')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 'a')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 'b')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 'c')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 'd')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 'a')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 'b')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 'c')");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 'd')");
+        String tableName = generateRandomString(); 
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 char(1) not null, constraint pk primary key (k1,k2)) split on ('ac','jc','nc')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 'd')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 'd')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'b')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'c')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 'd')");
         conn.commit();
-        String query = "SELECT k1,count(*) FROM GROUP_BY_DESC GROUP BY k1 ORDER BY k1 DESC";
+        String query = "SELECT k1,count(*) FROM " + tableName + " GROUP BY k1 ORDER BY k1 DESC";
         ResultSet rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("n", rs.getString(1));
@@ -341,7 +350,7 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         assertEquals(
-                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER GROUP_BY_DESC\n" + 
+                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
     }
@@ -350,26 +359,28 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testSumGroupByOrderPreservingDesc() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE GROUP_BY_DESC (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
+        String tableName = generateRandomString();
+
+        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
         stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
         stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
         stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
         stmt.execute();
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 4)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('b', 5)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 4)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 4)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 4)");
         conn.commit();
-        String query = "SELECT k1,sum(k2) FROM GROUP_BY_DESC GROUP BY k1 ORDER BY k1 DESC";
+        String query = "SELECT k1,sum(k2) FROM " + tableName + " GROUP BY k1 ORDER BY k1 DESC";
         ResultSet rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("n", rs.getString(1));
@@ -386,7 +397,7 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         assertEquals(
-                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER GROUP_BY_DESC\n" + 
+                "CLIENT PARALLEL 1-WAY REVERSE FULL SCAN OVER " + tableName + "\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
     }
@@ -395,26 +406,28 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
     public void testAvgGroupByOrderPreserving() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE GROUP_BY_DESC (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
+        String tableName = generateRandomString();
+
+        PreparedStatement stmt = conn.prepareStatement("CREATE TABLE " + tableName + " (k1 char(1) not null, k2 integer not null, constraint pk primary key (k1,k2)) split on (?,?,?)");
         stmt.setBytes(1, ByteUtil.concat(PChar.INSTANCE.toBytes("a"), PInteger.INSTANCE.toBytes(3)));
         stmt.setBytes(2, ByteUtil.concat(PChar.INSTANCE.toBytes("j"), PInteger.INSTANCE.toBytes(3)));
         stmt.setBytes(3, ByteUtil.concat(PChar.INSTANCE.toBytes("n"), PInteger.INSTANCE.toBytes(3)));
         stmt.execute();
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('a', 6)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('b', 5)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('j', 10)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 1)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 2)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 3)");
-        conn.createStatement().execute("UPSERT INTO GROUP_BY_DESC VALUES('n', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a', 6)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('b', 5)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('j', 10)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 3)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('n', 2)");
         conn.commit();
-        String query = "SELECT k1,avg(k2) FROM GROUP_BY_DESC GROUP BY k1";
+        String query = "SELECT k1,avg(k2) FROM " + tableName + " GROUP BY k1";
         ResultSet rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals("a", rs.getString(1));
@@ -431,7 +444,7 @@ public class GroupByCaseIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         assertEquals(
-                "CLIENT PARALLEL 1-WAY FULL SCAN OVER GROUP_BY_DESC\n" + 
+                "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY\n" + 
                 "    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [K1]", QueryUtil.getExplainPlan(rs));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
index c27db6b..943a376 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
@@ -43,7 +43,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
+public class HashJoinMoreIT extends BaseHBaseManagedTimeTableReuseIT {
     private final String[] plans = new String[] {
             /*
              * testJoinWithKeyRangeOptimization()
@@ -51,10 +51,10 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
              *     FROM TEMP_TABLE_COMPOSITE_PK lhs 
              *     JOIN TEMP_TABLE_COMPOSITE_PK rhs ON lhs.col1 = rhs.col2
              */
-            "CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "CLIENT MERGE SORT\n" +
             "    PARALLEL INNER-JOIN TABLE 0\n" +
-            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "        CLIENT MERGE SORT",
             /*
              * testJoinWithKeyRangeOptimization()
@@ -62,10 +62,10 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
              *     FROM TEMP_TABLE_COMPOSITE_PK lhs 
              *     JOIN TEMP_TABLE_COMPOSITE_PK rhs ON lhs.col0 = rhs.col2
              */
-            "CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "CLIENT MERGE SORT\n" +
             "    PARALLEL INNER-JOIN TABLE 0\n" +
-            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "        CLIENT MERGE SORT\n" +
             "    DYNAMIC SERVER FILTER BY LHS.COL0 IN (RHS.COL2)",
             /*
@@ -74,10 +74,10 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
              *     FROM TEMP_TABLE_COMPOSITE_PK lhs 
              *     JOIN TEMP_TABLE_COMPOSITE_PK rhs ON lhs.col0 = rhs.col1 AND lhs.col1 = rhs.col2
              */
-            "CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "CLIENT MERGE SORT\n" +
             "    PARALLEL INNER-JOIN TABLE 0\n" +
-            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "        CLIENT MERGE SORT\n" +
             "    DYNAMIC SERVER FILTER BY (LHS.COL0, LHS.COL1) IN ((RHS.COL1, RHS.COL2))",
             /*
@@ -86,16 +86,16 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
              *     FROM TEMP_TABLE_COMPOSITE_PK lhs 
              *     JOIN TEMP_TABLE_COMPOSITE_PK rhs ON lhs.col0 = rhs.col1 AND lhs.col2 = rhs.col3 - 1 AND lhs.col1 = rhs.col2
              */
-            "CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "CLIENT MERGE SORT\n" +
             "    PARALLEL INNER-JOIN TABLE 0\n" +
-            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER TEMP_TABLE_COMPOSITE_PK\n" +
+            "        CLIENT PARALLEL 4-WAY FULL SCAN OVER %s\n" +
             "        CLIENT MERGE SORT\n" +
             "    DYNAMIC SERVER FILTER BY (LHS.COL0, LHS.COL1, LHS.COL2) IN ((RHS.COL1, RHS.COL2, TO_INTEGER((RHS.COL3 - 1))))",            
     };
     
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Forces server cache to be used
@@ -117,8 +117,8 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testJoinOverSaltedTables() throws Exception {
-        String tempTableNoSalting = "TEMP_TABLE_NO_SALTING";
-        String tempTableWithSalting = "TEMP_TABLE_WITH_SALTING";
+        String tempTableNoSalting = generateRandomString();
+        String tempTableWithSalting = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -255,8 +255,8 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testJoinOnDynamicColumns() throws Exception {
-        String tableA = "tableA";
-        String tableB = "tableB";
+        String tableA = generateRandomString();
+        String tableB = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = null;
         PreparedStatement stmt = null;
@@ -274,7 +274,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             stmt.close();
 
-            String upsertA = "UPSERT INTO TABLEA (pkA, colA1, colA2) VALUES(?, ?, ?)";
+            String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
             stmt = conn.prepareStatement(upsertA);
             int i = 0;
             for (i = 0; i < 5; i++) {
@@ -286,19 +286,20 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             conn.commit();
             stmt.close();
 
+            String sequenceB = generateRandomString();
             // upsert select dynamic columns in tableB
-            conn.createStatement().execute("CREATE SEQUENCE SEQB");
-            String upsertBSelectA = "UPSERT INTO TABLEB (pkB, pkA INTEGER)"
-                    + "SELECT NEXT VALUE FOR SEQB, pkA FROM TABLEA";
+            conn.createStatement().execute("CREATE SEQUENCE " + sequenceB );
+            String upsertBSelectA = "UPSERT INTO " + tableB + " (pkB, pkA INTEGER)"
+                    + "SELECT NEXT VALUE FOR " + sequenceB + ", pkA FROM " + tableA ;
             stmt = conn.prepareStatement(upsertBSelectA);
             stmt.executeUpdate();
             stmt.close();
             conn.commit();
-            conn.createStatement().execute("DROP SEQUENCE SEQB");
+            conn.createStatement().execute("DROP SEQUENCE " + sequenceB );
 
             // perform a join between tableB and tableA by joining on the dynamic column that we upserted in
             // tableB. This join should return all the rows from table A.
-            String joinSql = "SELECT A.pkA, A.COLA1, A.colA2 FROM TABLEB B(pkA INTEGER) JOIN TABLEA A ON a.pkA = b.pkA";
+            String joinSql = "SELECT A.pkA, A.COLA1, A.colA2 FROM " + tableB + " B(pkA INTEGER) JOIN " + tableA + " A ON a.pkA = b.pkA";
             stmt = conn.prepareStatement(joinSql);
             ResultSet rs = stmt.executeQuery();
             i = 0;
@@ -324,7 +325,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testJoinWithKeyRangeOptimization() throws Exception {
-        String tempTableWithCompositePK = "TEMP_TABLE_COMPOSITE_PK";
+        String tempTableWithCompositePK = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -375,7 +376,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-            assertEquals(plans[0], QueryUtil.getExplainPlan(rs));
+            assertEquals(String.format(plans[0],tempTableWithCompositePK,tempTableWithCompositePK), QueryUtil.getExplainPlan(rs));
             
             // Two parts of PK but only one leading part
             query = "SELECT lhs.col0, lhs.col1, lhs.col2, lhs.col3, rhs.col0, rhs.col1, rhs.col2, rhs.col3 FROM " 
@@ -396,7 +397,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-            assertEquals(plans[1], QueryUtil.getExplainPlan(rs));
+            assertEquals(String.format(plans[1],tempTableWithCompositePK,tempTableWithCompositePK), QueryUtil.getExplainPlan(rs));
             
             // Two leading parts of PK
             query = "SELECT lhs.col0, lhs.col1, lhs.col2, lhs.col3, rhs.col0, rhs.col1, rhs.col2, rhs.col3 FROM " 
@@ -426,7 +427,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-            assertEquals(plans[2], QueryUtil.getExplainPlan(rs));
+            assertEquals(String.format(plans[2],tempTableWithCompositePK,tempTableWithCompositePK), QueryUtil.getExplainPlan(rs));
             
             // All parts of PK
             query = "SELECT lhs.col0, lhs.col1, lhs.col2, lhs.col3, rhs.col0, rhs.col1, rhs.col2, rhs.col3 FROM " 
@@ -456,7 +457,7 @@ public class HashJoinMoreIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-            assertEquals(plans[3], QueryUtil.getExplainPlan(rs));
+            assertEquals(String.format(plans[3],tempTableWithCompositePK,tempTableWithCompositePK), QueryUtil.getExplainPlan(rs));
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
index 4aff12b..0038c3a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InListIT.java
@@ -45,23 +45,24 @@ import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 
 
-public class InListIT extends BaseHBaseManagedTimeIT {
+public class InListIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testLeadingPKWithTrailingRVC() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE in_test "
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName
                 + "( col1 VARCHAR NOT NULL,"
                 + "  col2 VARCHAR NOT NULL, "
                 + "  id VARCHAR NOT NULL,"
                 + "  CONSTRAINT pk PRIMARY KEY (col1, col2, id))");
 
-        conn.createStatement().execute("upsert into in_test (col1, col2, id) values ('a', 'b', 'c')");
-        conn.createStatement().execute("upsert into in_test (col1, col2, id) values ('a', 'b', 'd')");
+        conn.createStatement().execute("upsert into " + tableName + " (col1, col2, id) values ('a', 'b', 'c')");
+        conn.createStatement().execute("upsert into " + tableName + " (col1, col2, id) values ('a', 'b', 'd')");
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("select id from in_test WHERE col1 = 'a' and ((col2, id) IN (('b', 'c'),('b', 'e')))");
+        ResultSet rs = conn.createStatement().executeQuery("select id from " + tableName + " WHERE col1 = 'a' and ((col2, id) IN (('b', 'c'),('b', 'e')))");
         assertTrue(rs.next());
         assertEquals("c", rs.getString(1));
         assertFalse(rs.next());
@@ -73,13 +74,14 @@ public class InListIT extends BaseHBaseManagedTimeIT {
     public void testLeadingPKWithTrailingRVC2() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE in_test ( user VARCHAR, tenant_id VARCHAR(5) NOT NULL,tenant_type_id VARCHAR(3) NOT NULL,  id INTEGER NOT NULL CONSTRAINT pk PRIMARY KEY (tenant_id, tenant_type_id, id))");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " ( user VARCHAR, tenant_id VARCHAR(5) NOT NULL,tenant_type_id VARCHAR(3) NOT NULL,  id INTEGER NOT NULL CONSTRAINT pk PRIMARY KEY (tenant_id, tenant_type_id, id))");
 
-        conn.createStatement().execute("upsert into in_test (tenant_id, tenant_type_id, id, user) values ('a', 'a', 1, 'BonA')");
-        conn.createStatement().execute("upsert into in_test (tenant_id, tenant_type_id, id, user) values ('a', 'a', 2, 'BonB')");
+        conn.createStatement().execute("upsert into " + tableName + " (tenant_id, tenant_type_id, id, user) values ('a', 'a', 1, 'BonA')");
+        conn.createStatement().execute("upsert into " + tableName + " (tenant_id, tenant_type_id, id, user) values ('a', 'a', 2, 'BonB')");
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("select id from in_test WHERE tenant_id = 'a' and tenant_type_id = 'a' and ((id, user) IN ((1, 'BonA'),(1, 'BonB')))");
+        ResultSet rs = conn.createStatement().executeQuery("select id from " + tableName + " WHERE tenant_id = 'a' and tenant_type_id = 'a' and ((id, user) IN ((1, 'BonA'),(1, 'BonB')))");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertFalse(rs.next());
@@ -142,8 +144,7 @@ public class InListIT extends BaseHBaseManagedTimeIT {
      * @return  the table or view name that should be used to access the created table
      */
     private static String initializeAndGetTable(Connection baseConn, Connection conn, boolean isMultiTenant, PDataType pkType, int saltBuckets) throws SQLException {
-            String tableName = "in_test" + pkType.getSqlTypeName() + saltBuckets + (isMultiTenant ? "_multi" : "_single");
-            
+            String tableName = generateRandomString() + "in_test" + pkType.getSqlTypeName() + saltBuckets + (isMultiTenant ? "_multi" : "_single");
             String tableDDL = createTableDDL(tableName, pkType, saltBuckets, isMultiTenant);
             baseConn.createStatement().execute(tableDDL);
             
@@ -444,13 +445,14 @@ public class InListIT extends BaseHBaseManagedTimeIT {
     public void testWithFixedLengthKV() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE in_test ( id INTEGER PRIMARY KEY, k CHAR(3))");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " ( id INTEGER PRIMARY KEY, k CHAR(3))");
 
-        conn.createStatement().execute("upsert into in_test values (1, 'aa')");
-        conn.createStatement().execute("upsert into in_test values (2, 'bb')");
+        conn.createStatement().execute("upsert into " + tableName + " values (1, 'aa')");
+        conn.createStatement().execute("upsert into " + tableName + " values (2, 'bb')");
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("select k from in_test WHERE k IN ('aa','bb')");
+        ResultSet rs = conn.createStatement().executeQuery("select k from " + tableName + " WHERE k IN ('aa','bb')");
         assertTrue(rs.next());
         assertEquals("aa", rs.getString(1));
         assertTrue(rs.next());
@@ -463,13 +465,14 @@ public class InListIT extends BaseHBaseManagedTimeIT {
     private void testWithFixedLengthPK(SortOrder sortOrder) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE in_test ( k CHAR(3) PRIMARY KEY " + (sortOrder == SortOrder.DESC ? "DESC" : "") + ")");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " ( k CHAR(3) PRIMARY KEY " + (sortOrder == SortOrder.DESC ? "DESC" : "") + ")");
 
-        conn.createStatement().execute("upsert into in_test (k) values ('aa')");
-        conn.createStatement().execute("upsert into in_test (k) values ('bb')");
+        conn.createStatement().execute("upsert into " + tableName + " (k) values ('aa')");
+        conn.createStatement().execute("upsert into " + tableName + " (k) values ('bb')");
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("select k from in_test WHERE k IN ('aa','bb')");
+        ResultSet rs = conn.createStatement().executeQuery("select k from " + tableName + " WHERE k IN ('aa','bb')");
         assertTrue(rs.next());
         assertEquals(sortOrder == SortOrder.ASC ? "aa" : "bb", rs.getString(1));
         assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
index 232f76f..c6bfe9a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
@@ -32,7 +32,7 @@ public class InMemoryOrderByIT extends OrderByIT {
     }
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(1024*1024));


[49/50] [abbrv] phoenix git commit: Fix compilation errors from sync with master

Posted by ma...@apache.org.
Fix compilation errors from sync with master


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

Branch: refs/heads/calcite
Commit: a9526a94bd603aceeb5c83d4371830815c57e2ad
Parents: 9a08a9d 1ce9845
Author: maryannxue <ma...@gmail.com>
Authored: Wed Sep 14 10:31:57 2016 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Wed Sep 14 10:31:57 2016 -0700

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 KEYS                                            |   58 +
 LICENSE                                         |  174 +
 NOTICE                                          |   29 +-
 README.md                                       |   17 +
 bin/hadoop-metrics2-hbase.properties            |   16 +
 bin/hadoop-metrics2-phoenix.properties          |   19 +-
 bin/hbase-site.xml                              |    2 -
 bin/log4j.properties                            |    2 +-
 bin/phoenix_utils.py                            |   12 +-
 bin/queryserver.py                              |   38 +-
 bin/sqlline-thin.py                             |    4 +-
 bin/sqlline.py                                  |   39 +-
 bin/tephra                                      |    6 +-
 build.txt                                       |    2 -
 dev/make_rc.sh                                  |   12 +-
 dev/release_files/LICENSE                       | 1615 ++-
 dev/release_files/NOTICE                        |  320 +-
 dev/test-patch.properties                       |    1 +
 dev/test-patch.sh                               |   32 +
 examples/STOCK_SYMBOL.sql                       |    1 -
 examples/WEB_STAT_QUERIES.sql                   |    1 -
 examples/pig/test.pig                           |   17 +
 phoenix-assembly/pom.xml                        |   91 +-
 phoenix-assembly/src/build/client-minimal.xml   |   38 -
 phoenix-assembly/src/build/client-spark.xml     |   74 -
 .../src/build/client-without-hbase.xml          |   40 -
 phoenix-assembly/src/build/client.xml           |   98 -
 .../src/build/components-major-client.xml       |   57 -
 .../src/build/components-minimal.xml            |   77 -
 .../components/all-common-dependencies.xml      |    2 +-
 .../src/build/components/all-common-jars.xml    |   42 +-
 phoenix-assembly/src/build/server.xml           |   53 -
 phoenix-client/pom.xml                          |  378 +
 phoenix-core/pom.xml                            |   45 +-
 ...ReplayWithIndexWritesAndCompressedWALIT.java |   18 +-
 .../phoenix/end2end/AbsFunctionEnd2EndIT.java   |   11 +-
 .../phoenix/end2end/AggregateQueryIT.java       |   20 +-
 .../AlterMultiTenantTableWithViewsIT.java       |  704 ++
 .../apache/phoenix/end2end/AlterSessionIT.java  |   20 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |   29 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 1013 +-
 .../phoenix/end2end/AppendOnlySchemaIT.java     |  356 +
 .../phoenix/end2end/ArithmeticQueryIT.java      |  314 +-
 .../phoenix/end2end/ArrayAppendFunctionIT.java  |  186 +-
 .../phoenix/end2end/ArrayConcatFunctionIT.java  |  160 +-
 .../phoenix/end2end/ArrayFillFunctionIT.java    |  175 +-
 .../org/apache/phoenix/end2end/ArrayIT.java     |   55 +-
 .../phoenix/end2end/ArrayPrependFunctionIT.java |  220 +-
 .../end2end/ArrayToStringFunctionIT.java        |  326 +-
 .../phoenix/end2end/ArraysWithNullsIT.java      |  152 +-
 .../apache/phoenix/end2end/AutoCommitIT.java    |   21 +-
 .../phoenix/end2end/AutoPartitionViewsIT.java   |  392 +
 .../end2end/BaseClientManagedTimeIT.java        |    2 +-
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java |    7 +-
 .../BaseHBaseManagedTimeTableReuseIT.java       |   73 +
 .../BaseOwnClusterClientManagedTimeIT.java      |    6 +-
 .../BaseOwnClusterHBaseManagedTimeIT.java       |    6 +-
 .../phoenix/end2end/BaseOwnClusterIT.java       |    4 +-
 .../org/apache/phoenix/end2end/BaseQueryIT.java |    6 +-
 .../end2end/BaseTenantSpecificViewIndexIT.java  |   80 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   15 +-
 .../apache/phoenix/end2end/BinaryRowKeyIT.java  |   33 +-
 .../phoenix/end2end/CSVCommonsLoaderIT.java     |   80 +-
 .../apache/phoenix/end2end/CaseStatementIT.java |    4 +-
 .../apache/phoenix/end2end/CastAndCoerceIT.java |    4 +-
 .../phoenix/end2end/CbrtFunctionEnd2EndIT.java  |   35 +-
 .../phoenix/end2end/ClientManagedTimeTest.java  |    4 +-
 .../end2end/ClientTimeArithmeticQueryIT.java    |    4 +-
 .../phoenix/end2end/CoalesceFunctionIT.java     |  117 +-
 .../end2end/ColumnProjectionOptimizationIT.java |   17 +-
 .../end2end/ConvertTimezoneFunctionIT.java      |   85 +-
 .../end2end/CountDistinctCompressionIT.java     |    3 +-
 .../apache/phoenix/end2end/CreateSchemaIT.java  |   83 +
 .../apache/phoenix/end2end/CreateTableIT.java   |   81 +-
 .../phoenix/end2end/CsvBulkLoadToolIT.java      |    4 +
 .../phoenix/end2end/CustomEntityDataIT.java     |    2 +-
 .../org/apache/phoenix/end2end/DateTimeIT.java  |  209 +-
 .../phoenix/end2end/DecodeFunctionIT.java       |   49 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    |  176 +-
 .../apache/phoenix/end2end/DerivedTableIT.java  |    3 +-
 .../phoenix/end2end/DisableLocalIndexIT.java    |   18 +-
 .../apache/phoenix/end2end/DistinctCountIT.java |   80 +-
 .../phoenix/end2end/DistinctPrefixFilterIT.java |  395 +
 .../apache/phoenix/end2end/DropSchemaIT.java    |  138 +
 .../apache/phoenix/end2end/DynamicColumnIT.java |  119 +-
 .../apache/phoenix/end2end/DynamicFamilyIT.java |   31 +-
 .../apache/phoenix/end2end/DynamicUpsertIT.java |   11 +-
 .../phoenix/end2end/EncodeFunctionIT.java       |   42 +-
 .../phoenix/end2end/End2EndTestDriver.java      |    2 -
 .../phoenix/end2end/EvaluationOfORIT.java       |   13 +-
 .../phoenix/end2end/ExecuteStatementsIT.java    |   42 +-
 .../phoenix/end2end/ExpFunctionEnd2EndIT.java   |   20 +-
 .../phoenix/end2end/ExtendedQueryExecIT.java    |   12 +-
 .../phoenix/end2end/FirstValueFunctionIT.java   |  144 +-
 .../apache/phoenix/end2end/FunkyNamesIT.java    |    2 +-
 .../end2end/GetSetByteBitFunctionEnd2EndIT.java |   26 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   |  245 +-
 .../org/apache/phoenix/end2end/GroupByIT.java   |    4 +-
 .../end2end/HBaseManagedTimeTableReuseTest.java |   47 +
 .../phoenix/end2end/HBaseManagedTimeTest.java   |    8 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  |   55 +-
 .../phoenix/end2end/HashJoinLocalIndexIT.java   |   13 +-
 .../apache/phoenix/end2end/HashJoinMoreIT.java  |  301 +-
 .../org/apache/phoenix/end2end/InListIT.java    |   41 +-
 .../phoenix/end2end/InMemoryOrderByIT.java      |    6 +-
 .../org/apache/phoenix/end2end/IndexToolIT.java |   21 +-
 .../apache/phoenix/end2end/InstrFunctionIT.java |   48 +-
 .../org/apache/phoenix/end2end/IsNullIT.java    |   37 +-
 .../org/apache/phoenix/end2end/KeyOnlyIT.java   |    6 +-
 .../phoenix/end2end/LastValueFunctionIT.java    |  194 +-
 .../phoenix/end2end/LikeExpressionIT.java       |   77 +-
 .../phoenix/end2end/LnLogFunctionEnd2EndIT.java |   15 +-
 .../apache/phoenix/end2end/MD5FunctionIT.java   |   30 +-
 .../org/apache/phoenix/end2end/MapReduceIT.java |   61 +-
 .../phoenix/end2end/MappingTableDataTypeIT.java |   17 +-
 .../end2end/MinMaxAggregateFunctionIT.java      |   24 +-
 .../phoenix/end2end/ModulusExpressionIT.java    |   45 +-
 .../phoenix/end2end/MultiCfQueryExecIT.java     |    2 +-
 .../end2end/NamespaceSchemaMappingIT.java       |  115 +
 .../phoenix/end2end/NativeHBaseTypesIT.java     |    2 +-
 .../end2end/NeedsOwnMiniClusterTest.java        |    4 +-
 .../org/apache/phoenix/end2end/NotQueryIT.java  |    4 +-
 .../phoenix/end2end/NthValueFunctionIT.java     |  252 +-
 .../end2end/OctetLengthFunctionEnd2EndIT.java   |   13 +-
 .../org/apache/phoenix/end2end/OrderByIT.java   |   98 +-
 .../phoenix/end2end/ParallelIteratorsIT.java    |    6 +-
 .../apache/phoenix/end2end/PercentileIT.java    |  137 +-
 .../phoenix/end2end/PhoenixRuntimeIT.java       |   44 +-
 .../phoenix/end2end/PowerFunctionEnd2EndIT.java |   21 +-
 .../apache/phoenix/end2end/PrimitiveTypeIT.java |   63 +-
 .../phoenix/end2end/ProductMetricsIT.java       |    2 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |  162 +-
 .../phoenix/end2end/QueryExecWithoutSCNIT.java  |    6 +-
 .../org/apache/phoenix/end2end/QueryMoreIT.java |   31 +-
 .../apache/phoenix/end2end/QueryTimeoutIT.java  |   36 +-
 .../phoenix/end2end/QueryWithLimitIT.java       |    8 +-
 .../phoenix/end2end/QueryWithOffsetIT.java      |   62 +-
 .../apache/phoenix/end2end/RTrimFunctionIT.java |   24 +-
 .../phoenix/end2end/ReadIsolationLevelIT.java   |    2 +-
 .../org/apache/phoenix/end2end/ReadOnlyIT.java  |   17 +-
 .../end2end/RegexpReplaceFunctionIT.java        |   17 +-
 .../phoenix/end2end/RegexpSplitFunctionIT.java  |   69 +-
 .../phoenix/end2end/RegexpSubstrFunctionIT.java |   12 +-
 .../apache/phoenix/end2end/RenewLeaseIT.java    |   90 +
 .../phoenix/end2end/ReverseFunctionIT.java      |   42 +-
 .../apache/phoenix/end2end/ReverseScanIT.java   |   58 +-
 .../RoundFloorCeilFunctionsEnd2EndIT.java       |  101 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |   83 +-
 .../org/apache/phoenix/end2end/ScanQueryIT.java |    4 +-
 .../org/apache/phoenix/end2end/SequenceIT.java  |   58 +-
 .../phoenix/end2end/SerialIteratorsIT.java      |   90 +
 .../phoenix/end2end/ServerExceptionIT.java      |    9 +-
 .../org/apache/phoenix/end2end/Shadower.java    |    4 +-
 .../phoenix/end2end/SignFunctionEnd2EndIT.java  |   35 +-
 .../end2end/SkipScanAfterManualSplitIT.java     |   44 +-
 .../apache/phoenix/end2end/SkipScanQueryIT.java |  156 +-
 .../apache/phoenix/end2end/SortMergeJoinIT.java |   11 +-
 .../phoenix/end2end/SortMergeJoinMoreIT.java    |  257 +-
 .../org/apache/phoenix/end2end/SortOrderIT.java |  303 +-
 .../phoenix/end2end/SpillableGroupByIT.java     |    2 +-
 .../phoenix/end2end/SpooledOrderByIT.java       |    4 +-
 .../phoenix/end2end/SpooledTmpFileDeleteIT.java |   20 +-
 .../phoenix/end2end/SqrtFunctionEnd2EndIT.java  |   25 +-
 .../phoenix/end2end/StatementHintsIT.java       |   12 +-
 .../org/apache/phoenix/end2end/StddevIT.java    |   18 +-
 .../apache/phoenix/end2end/StoreNullsIT.java    |   51 +-
 .../org/apache/phoenix/end2end/StringIT.java    |   66 +-
 .../end2end/StringToArrayFunctionIT.java        |  164 +-
 .../org/apache/phoenix/end2end/SubqueryIT.java  |   24 +-
 .../end2end/SubqueryUsingSortMergeJoinIT.java   |   12 +-
 .../apache/phoenix/end2end/TenantIdTypeIT.java  |    4 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |    5 +-
 .../end2end/TenantSpecificViewIndexIT.java      |  219 +-
 .../end2end/TimezoneOffsetFunctionIT.java       |   63 +-
 .../phoenix/end2end/ToCharFunctionIT.java       |   54 +-
 .../phoenix/end2end/ToDateFunctionIT.java       |    9 +-
 .../java/org/apache/phoenix/end2end/TopNIT.java |    8 +-
 .../phoenix/end2end/TruncateFunctionIT.java     |    2 +-
 .../org/apache/phoenix/end2end/UnionAllIT.java  |  325 +-
 .../apache/phoenix/end2end/UnnestArrayIT.java   |   18 +-
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  320 +-
 .../phoenix/end2end/UpsertBigValuesIT.java      |   78 +-
 .../end2end/UpsertSelectAutoCommitIT.java       |   50 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |   20 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   44 +-
 .../org/apache/phoenix/end2end/UseSchemaIT.java |  146 +
 .../phoenix/end2end/UserDefinedFunctionsIT.java |   60 +-
 .../phoenix/end2end/VariableLengthPKIT.java     |   38 +-
 .../java/org/apache/phoenix/end2end/ViewIT.java |   35 +-
 .../end2end/index/AsyncImmutableIndexIT.java    |  114 +
 .../end2end/index/AsyncIndexDisabledIT.java     |   78 +
 .../phoenix/end2end/index/DropMetadataIT.java   |  303 +
 .../phoenix/end2end/index/DropViewIT.java       |  105 -
 .../index/GlobalIndexOptimizationIT.java        |  175 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |   36 +-
 .../end2end/index/IndexExpressionIT.java        |  431 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |  144 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |  293 +-
 .../phoenix/end2end/index/LocalIndexIT.java     |  724 +-
 .../end2end/index/MutableIndexFailureIT.java    |   60 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |  331 +-
 .../end2end/index/ReadOnlyIndexFailureIT.java   |   10 +
 .../phoenix/end2end/index/SaltedIndexIT.java    |   95 +-
 .../phoenix/end2end/index/ViewIndexIT.java      |  137 +-
 .../end2end/index/txn/MutableRollbackIT.java    |   82 +-
 .../phoenix/end2end/index/txn/RollbackIT.java   |   20 +-
 .../end2end/index/txn/TxWriteFailureIT.java     |    2 +-
 .../phoenix/end2end/salted/SaltedTableIT.java   |    2 +-
 .../salted/SaltedTableUpsertSelectIT.java       |   87 +-
 .../salted/SaltedTableVarLengthRowKeyIT.java    |   12 +-
 .../apache/phoenix/execute/PartialCommitIT.java |    4 +-
 .../index/balancer/IndexLoadBalancerIT.java     |  485 -
 .../DelayedTableResultIteratorFactory.java      |    9 +-
 .../iterate/MockParallelIteratorFactory.java    |    3 +-
 .../phoenix/iterate/PhoenixQueryTimeoutIT.java  |   26 +-
 .../iterate/RoundRobinResultIteratorIT.java     |   26 +-
 .../apache/phoenix/rpc/PhoenixServerRpcIT.java  |   21 -
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |   26 +-
 .../phoenix/rpc/UpdateCacheWithScnIT.java       |    2 +-
 .../apache/phoenix/trace/BaseTracingTestIT.java |    4 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |  315 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   63 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   22 +
 .../hadoop/hbase/ipc/PhoenixRpcScheduler.java   |    8 +-
 .../ipc/controller/MetadataRpcController.java   |   12 +-
 .../regionserver/IndexHalfStoreFileReader.java  |  412 +-
 .../IndexHalfStoreFileReaderGenerator.java      |  172 +-
 .../regionserver/IndexSplitTransaction.java     |  986 --
 .../hbase/regionserver/LocalIndexMerger.java    |  123 -
 .../hbase/regionserver/LocalIndexSplitter.java  |  174 -
 .../LocalIndexStoreFileScanner.java             |  268 +
 .../org/apache/phoenix/cache/GlobalCache.java   |    4 +-
 .../phoenix/cache/IndexMetaDataCache.java       |    2 +-
 .../apache/phoenix/cache/JodaTimezoneCache.java |   16 +-
 .../apache/phoenix/cache/ServerCacheClient.java |   48 +-
 .../cache/aggcache/SpillableGroupByCache.java   |    5 +-
 .../apache/phoenix/calcite/PhoenixSchema.java   |    3 +-
 .../apache/phoenix/calcite/TableMapping.java    |    5 +-
 .../calcite/rel/PhoenixRelImplementorImpl.java  |    2 +-
 .../phoenix/calcite/rel/PhoenixTableModify.java |    2 +-
 .../rel/PhoenixToEnumerableConverter.java       |    5 +
 .../phoenix/compile/AggregationManager.java     |   60 +
 .../apache/phoenix/compile/ColumnResolver.java  |   10 +
 .../phoenix/compile/CreateSchemaCompiler.java   |   65 +
 .../phoenix/compile/CreateTableCompiler.java    |   18 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   93 +-
 .../phoenix/compile/ExpressionCompiler.java     |    6 +-
 .../apache/phoenix/compile/FromCompiler.java    |  177 +-
 .../apache/phoenix/compile/GroupByCompiler.java |  100 +-
 .../phoenix/compile/IndexStatementRewriter.java |   12 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   38 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    4 +
 .../MutatingParallelIteratorFactory.java        |    2 +-
 .../apache/phoenix/compile/OrderByCompiler.java |   18 +-
 .../phoenix/compile/OrderPreservingTracker.java |    9 +
 .../apache/phoenix/compile/PostDDLCompiler.java |   25 +-
 .../compile/PostLocalIndexDDLCompiler.java      |   23 +-
 .../phoenix/compile/ProjectionCompiler.java     |   63 +-
 .../apache/phoenix/compile/QueryCompiler.java   |   18 +-
 .../org/apache/phoenix/compile/QueryPlan.java   |    2 +
 .../org/apache/phoenix/compile/ScanRanges.java  |  104 +-
 .../phoenix/compile/StatementContext.java       |    5 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    5 +
 .../compile/TupleProjectionCompiler.java        |   16 +-
 .../apache/phoenix/compile/UnionCompiler.java   |  161 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |  103 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   64 +-
 .../coprocessor/BaseScannerRegionObserver.java  |  217 +-
 .../coprocessor/DelegateRegionObserver.java     |  248 +-
 .../coprocessor/DelegateRegionScanner.java      |    2 +-
 .../phoenix/coprocessor/GroupByCache.java       |    5 +-
 .../GroupedAggregateRegionObserver.java         |   34 +-
 .../coprocessor/HashJoinRegionScanner.java      |    5 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  956 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |  169 +-
 .../coprocessor/MetaDataRegionObserver.java     |   25 +-
 .../PhoenixTransactionalProcessor.java          |    2 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |   20 +-
 .../coprocessor/SequenceRegionObserver.java     |    7 +
 .../UngroupedAggregateRegionObserver.java       |  255 +-
 .../coprocessor/generated/MetaDataProtos.java   | 9554 +++++++++++++-----
 .../coprocessor/generated/PSchemaProtos.java    |  666 ++
 .../coprocessor/generated/PTableProtos.java     |  361 +-
 .../phoenix/exception/SQLExceptionCode.java     |   38 +-
 .../exception/UndecodableByteException.java     |    4 +-
 .../apache/phoenix/execute/AggregatePlan.java   |   42 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |   30 +-
 .../phoenix/execute/ClientAggregatePlan.java    |    6 +
 .../apache/phoenix/execute/ClientScanPlan.java  |    7 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |    8 +-
 .../phoenix/execute/DegenerateQueryPlan.java    |    2 +-
 .../apache/phoenix/execute/HashJoinPlan.java    |    8 +-
 .../execute/LiteralResultIterationPlan.java     |    2 +-
 .../apache/phoenix/execute/MutationState.java   |  382 +-
 .../org/apache/phoenix/execute/ScanPlan.java    |  121 +-
 .../phoenix/execute/SortMergeJoinPlan.java      |    8 +-
 .../phoenix/execute/TupleProjectionPlan.java    |    8 +-
 .../org/apache/phoenix/execute/UnionPlan.java   |    6 +-
 .../apache/phoenix/execute/UnnestArrayPlan.java |    8 +-
 .../expression/ByteBasedLikeExpression.java     |    4 +-
 .../phoenix/expression/InListExpression.java    |    4 +-
 .../expression/RowKeyColumnExpression.java      |    6 +-
 .../RowValueConstructorExpression.java          |   30 +-
 .../expression/StringBasedLikeExpression.java   |    3 +-
 .../DistinctValueWithCountServerAggregator.java |   11 +-
 .../function/ArrayLengthFunction.java           |    2 +-
 .../ByteBasedRegexpReplaceFunction.java         |    3 +-
 .../function/ByteBasedRegexpSplitFunction.java  |    3 +-
 .../function/ByteBasedRegexpSubstrFunction.java |    3 +-
 .../expression/function/CeilDateExpression.java |    2 +-
 .../function/CeilDecimalExpression.java         |   12 +-
 .../function/CeilTimestampExpression.java       |   14 +-
 .../expression/function/CoalesceFunction.java   |    6 +-
 .../function/ConvertTimezoneFunction.java       |   14 +-
 .../DistinctCountAggregateFunction.java         |    6 +-
 .../expression/function/EncodeFormat.java       |   14 +-
 .../function/FloorDateExpression.java           |    2 +-
 .../function/FloorDecimalExpression.java        |   12 +-
 .../function/RoundDateExpression.java           |    2 +-
 .../function/RoundDecimalExpression.java        |    2 +-
 .../function/RoundTimestampExpression.java      |    2 +-
 .../StringBasedRegexpReplaceFunction.java       |    3 +-
 .../StringBasedRegexpSubstrFunction.java        |    3 +-
 .../expression/function/ToCharFunction.java     |    3 +
 .../phoenix/filter/BooleanExpressionFilter.java |    1 +
 .../phoenix/filter/DistinctPrefixFilter.java    |  154 +
 .../filter/MultiKeyValueComparisonFilter.java   |    1 -
 .../hbase/index/IndexRegionSplitPolicy.java     |   52 +-
 .../org/apache/phoenix/hbase/index/Indexer.java |   84 +-
 .../hbase/index/balancer/IndexLoadBalancer.java |  671 +-
 .../hbase/index/covered/IndexMetaData.java      |    9 +-
 .../hbase/index/covered/NonTxIndexBuilder.java  |    6 +-
 .../index/covered/update/ColumnReference.java   |    9 +-
 .../hbase/index/master/IndexMasterObserver.java |   88 -
 .../index/util/ReadOnlyImmutableBytesPtr.java   |   59 -
 .../hbase/index/write/IndexCommitter.java       |    2 +-
 .../phoenix/hbase/index/write/IndexWriter.java  |   37 +-
 .../write/ParallelWriterIndexCommitter.java     |   42 +-
 .../hbase/index/write/RecoveryIndexWriter.java  |  134 +
 .../TrackingParallelWriterIndexCommitter.java   |   47 +-
 .../apache/phoenix/index/IndexMaintainer.java   |   74 +-
 .../index/IndexMetaDataCacheFactory.java        |    2 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |   64 -
 .../apache/phoenix/index/PhoenixIndexCodec.java |    6 +-
 .../index/PhoenixIndexFailurePolicy.java        |   13 +-
 .../phoenix/index/PhoenixIndexMetaData.java     |   17 +-
 .../index/PhoenixTransactionalIndexer.java      |  111 +-
 .../iterate/AggregatingResultIterator.java      |    4 +-
 .../BaseGroupedAggregatingResultIterator.java   |    3 +-
 .../phoenix/iterate/BaseResultIterators.java    |  181 +-
 .../phoenix/iterate/ChunkedResultIterator.java  |   69 +-
 .../DefaultTableResultIteratorFactory.java      |    5 +-
 .../DistinctAggregatingResultIterator.java      |   15 +-
 .../apache/phoenix/iterate/ExplainTable.java    |   26 +-
 .../FilterAggregatingResultIterator.java        |    8 +-
 .../phoenix/iterate/OffsetResultIterator.java   |    2 +-
 .../OrderedAggregatingResultIterator.java       |    5 +-
 .../iterate/ParallelIteratorFactory.java        |    5 +-
 .../phoenix/iterate/ParallelIterators.java      |   42 +-
 .../RowKeyOrderedAggregateResultIterator.java   |  190 +
 .../apache/phoenix/iterate/SerialIterators.java |  156 +-
 .../phoenix/iterate/SpoolingResultIterator.java |   20 +-
 .../phoenix/iterate/TableResultIterator.java    |   81 +-
 .../iterate/TableResultIteratorFactory.java     |    3 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  105 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  180 +-
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |   41 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  136 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   93 +-
 .../phoenix/mapreduce/AbstractBulkLoadTool.java |   80 +-
 .../phoenix/mapreduce/CsvBulkImportUtil.java    |    7 +
 .../mapreduce/FormatToBytesWritableMapper.java  |  122 +-
 .../mapreduce/FormatToKeyValueReducer.java      |   71 +-
 .../mapreduce/MultiHfileOutputFormat.java       |    4 +-
 .../phoenix/mapreduce/PhoenixInputFormat.java   |    2 -
 .../phoenix/mapreduce/PhoenixRecordReader.java  |    5 +-
 .../phoenix/mapreduce/index/IndexTool.java      |   74 +-
 .../phoenix/mapreduce/index/IndexToolUtil.java  |    4 +-
 .../index/PhoenixIndexImportDirectReducer.java  |   60 +
 .../index/PhoenixIndexImportMapper.java         |    7 +-
 .../index/PhoenixIndexToolReducer.java          |   60 -
 .../index/automation/PhoenixAsyncIndex.java     |   75 +
 .../index/automation/PhoenixMRJobCallable.java  |   73 +
 .../index/automation/PhoenixMRJobSubmitter.java |  290 +
 .../index/automation/YarnApplication.java       |  208 +
 .../phoenix/memory/GlobalMemoryManager.java     |   30 +-
 .../apache/phoenix/optimize/QueryOptimizer.java |   23 +-
 .../org/apache/phoenix/parse/ColumnDef.java     |    7 +-
 .../phoenix/parse/CreateSchemaStatement.java    |   42 +
 .../phoenix/parse/DropSchemaStatement.java      |   55 +
 .../parse/IndexExpressionParseNodeRewriter.java |    8 +-
 .../java/org/apache/phoenix/parse/PSchema.java  |   86 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |   12 +
 .../phoenix/parse/UseSchemaStatement.java       |   36 +
 .../apache/phoenix/protobuf/ProtobufUtil.java   |   18 +-
 .../phoenix/query/ConnectionQueryServices.java  |   22 +-
 .../query/ConnectionQueryServicesImpl.java      | 1179 ++-
 .../query/ConnectionlessQueryServicesImpl.java  |  108 +-
 .../query/DelegateConnectionQueryServices.java  |   79 +-
 .../apache/phoenix/query/MetaDataMutated.java   |   18 +-
 .../apache/phoenix/query/QueryConstants.java    |   26 +-
 .../org/apache/phoenix/query/QueryServices.java |   19 +-
 .../phoenix/query/QueryServicesOptions.java     |   31 +-
 .../apache/phoenix/query/TableStatsCache.java   |  192 +
 .../apache/phoenix/schema/DelegateTable.java    |   21 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  804 +-
 .../NewerSchemaAlreadyExistsException.java      |   26 +
 .../org/apache/phoenix/schema/PMetaData.java    |    6 +-
 .../apache/phoenix/schema/PMetaDataCache.java   |  221 +
 .../apache/phoenix/schema/PMetaDataImpl.java    |  344 +-
 .../phoenix/schema/PSynchronizedMetaData.java   |  249 +
 .../java/org/apache/phoenix/schema/PTable.java  |   20 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  227 +-
 .../org/apache/phoenix/schema/PTableKey.java    |    2 +
 .../org/apache/phoenix/schema/PTableRef.java    |   56 +-
 .../apache/phoenix/schema/PTableRefFactory.java |   52 +
 .../apache/phoenix/schema/PTableRefImpl.java    |   39 +
 .../schema/SchemaAlreadyExistsException.java    |   53 +
 .../phoenix/schema/SchemaNotFoundException.java |   52 +
 .../org/apache/phoenix/schema/Sequence.java     |   20 +-
 .../phoenix/schema/SerializedPTableRef.java     |   47 +
 .../schema/SerializedPTableRefFactory.java      |   37 +
 .../org/apache/phoenix/schema/SortOrder.java    |    2 -
 .../apache/phoenix/schema/TableProperty.java    |   11 +-
 .../org/apache/phoenix/schema/TableRef.java     |    3 +-
 .../org/apache/phoenix/schema/ValueSchema.java  |   30 +-
 .../stats/DefaultStatisticsCollector.java       |    4 +-
 .../schema/stats/GuidePostsInfoBuilder.java     |    6 +-
 .../stats/StatisticsCollectionRunTracker.java   |    1 -
 .../schema/stats/StatisticsCollectionScope.java |    6 +-
 .../stats/StatisticsCollectorFactory.java       |   16 +-
 .../phoenix/schema/stats/StatisticsScanner.java |   18 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |   27 +
 .../phoenix/schema/stats/StatisticsWriter.java  |   20 +-
 .../phoenix/schema/types/PArrayDataType.java    |    4 +-
 .../org/apache/phoenix/schema/types/PChar.java  |    4 +
 .../apache/phoenix/schema/types/PDataType.java  |   12 +
 .../org/apache/phoenix/schema/types/PDate.java  |    2 +-
 .../apache/phoenix/schema/types/PDecimal.java   |    8 +
 .../apache/phoenix/schema/types/PTimestamp.java |    2 +-
 .../phoenix/trace/PhoenixMetricsSink.java       |    2 +-
 .../org/apache/phoenix/util/ConfigUtil.java     |    2 -
 .../java/org/apache/phoenix/util/DateUtil.java  |   17 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |  135 +-
 .../apache/phoenix/util/InstanceResolver.java   |    7 +
 .../java/org/apache/phoenix/util/JDBCUtil.java  |    5 +
 .../org/apache/phoenix/util/MetaDataUtil.java   |  244 +-
 .../apache/phoenix/util/PhoenixMRJobUtil.java   |  233 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |  128 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |   75 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |   99 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |  166 +-
 .../apache/phoenix/util/TransactionUtil.java    |   20 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  622 +-
 .../phoenix/util/ZKBasedMasterElectionUtil.java |   70 +
 .../phoenix/cache/JodaTimezoneCacheTest.java    |   16 +-
 .../compile/CreateTableCompilerTest.java        |   47 +
 .../phoenix/compile/QueryCompilerTest.java      |  151 +
 .../phoenix/compile/QueryOptimizerTest.java     |   24 +
 .../phoenix/compile/SaltedScanRangesTest.java   |  239 +
 .../compile/ScanRangesIntersectTest.java        |    4 +-
 .../apache/phoenix/compile/ScanRangesTest.java  |    8 +-
 .../compile/StatementHintsCompilationTest.java  |    8 +-
 .../TenantSpecificViewIndexCompileTest.java     |    6 +-
 .../phoenix/compile/WhereOptimizerTest.java     |   61 +-
 .../phoenix/execute/CorrelatePlanTest.java      |    2 +-
 .../execute/LiteralResultIteratorPlanTest.java  |    2 +-
 .../phoenix/execute/MutationStateTest.java      |    4 +-
 .../phoenix/expression/LikeExpressionTest.java  |    8 +
 .../BuiltinFunctionConstructorTest.java         |   25 +-
 .../filter/DistinctPrefixFilterTest.java        |  274 +
 .../phoenix/filter/SkipScanBigFilterTest.java   |   12 +-
 .../index/write/TestParalleIndexWriter.java     |    2 +-
 .../write/TestParalleWriterIndexCommitter.java  |    2 +-
 .../phoenix/index/IndexMaintainerTest.java      |    2 +-
 .../index/automated/MRJobSubmitterTest.java     |  137 +
 .../iterate/AggregateResultScannerTest.java     |  109 +-
 .../iterate/ConcatResultIteratorTest.java       |   31 +-
 .../iterate/MaterializedResultIterators.java    |   66 +
 ...owKeyOrderedAggregateResultIteratorTest.java |  149 +
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |   10 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java |    6 +-
 .../phoenix/jdbc/SecureUserConnectionsTest.java |  382 +
 .../phoenix/mapreduce/BulkLoadToolTest.java     |    5 +-
 .../ColumnInfoToStringEncoderDecoderTest.java   |    4 +-
 .../util/PhoenixConfigurationUtilTest.java      |    4 +-
 .../phoenix/memory/MemoryManagerTest.java       |   93 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  260 +-
 .../query/ParallelIteratorsSplitTest.java       |   20 +-
 .../PhoenixStatsCacheRemovalListenerTest.java   |   45 +
 .../org/apache/phoenix/query/QueryPlanTest.java |   30 +-
 .../phoenix/schema/PMetaDataImplTest.java       |  102 +-
 .../apache/phoenix/schema/SortOrderTest.java    |    2 -
 .../org/apache/phoenix/util/DateUtilTest.java   |   25 +
 .../org/apache/phoenix/util/JDBCUtilTest.java   |    8 +
 .../apache/phoenix/util/MetaDataUtilTest.java   |   36 +
 .../phoenix/util/PhoenixEncodeDecodeTest.java   |    4 +-
 .../util/TenantIdByteConversionTest.java        |    2 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   97 +-
 .../test/resources/hadoop-metrics2.properties   |   18 +-
 .../src/test/resources/log4j.properties         |    2 +
 phoenix-flume/pom.xml                           |   14 +-
 .../org/apache/phoenix/flume/PhoenixSinkIT.java |   17 +-
 .../phoenix/flume/RegexEventSerializerIT.java   |   16 +-
 phoenix-hive/pom.xml                            |  162 +
 .../apache/phoenix/hive/HivePhoenixStoreIT.java |  303 +
 .../org/apache/phoenix/hive/HiveTestUtil.java   | 1296 +++
 .../apache/phoenix/hive/PhoenixMetaHook.java    |  246 +
 .../phoenix/hive/PhoenixRecordUpdater.java      |  336 +
 .../org/apache/phoenix/hive/PhoenixRow.java     |   64 +
 .../org/apache/phoenix/hive/PhoenixRowKey.java  |   69 +
 .../org/apache/phoenix/hive/PhoenixSerDe.java   |  159 +
 .../apache/phoenix/hive/PhoenixSerializer.java  |  169 +
 .../phoenix/hive/PhoenixStorageHandler.java     |  212 +
 .../PhoenixStorageHandlerConstants.java         |  108 +
 .../hive/mapreduce/PhoenixInputFormat.java      |  269 +
 .../hive/mapreduce/PhoenixInputSplit.java       |  160 +
 .../hive/mapreduce/PhoenixOutputFormat.java     |  112 +
 .../hive/mapreduce/PhoenixRecordReader.java     |  216 +
 .../hive/mapreduce/PhoenixRecordWriter.java     |  355 +
 .../hive/mapreduce/PhoenixResultWritable.java   |  211 +
 .../AbstractPhoenixObjectInspector.java         |   59 +
 .../PhoenixBinaryObjectInspector.java           |   58 +
 .../PhoenixBooleanObjectInspector.java          |   50 +
 .../PhoenixByteObjectInspector.java             |   54 +
 .../PhoenixCharObjectInspector.java             |   51 +
 .../PhoenixDateObjectInspector.java             |   63 +
 .../PhoenixDecimalObjectInspector.java          |   63 +
 .../PhoenixDoubleObjectInspector.java           |   54 +
 .../PhoenixFloatObjectInspector.java            |   55 +
 .../PhoenixIntObjectInspector.java              |   51 +
 .../PhoenixListObjectInspector.java             |  105 +
 .../PhoenixLongObjectInspector.java             |   51 +
 .../PhoenixObjectInspectorFactory.java          |  148 +
 .../PhoenixShortObjectInspector.java            |   51 +
 .../PhoenixStringObjectInspector.java           |   72 +
 .../PhoenixTimestampObjectInspector.java        |   61 +
 .../hive/ppd/PhoenixPredicateDecomposer.java    |   82 +
 .../ppd/PhoenixPredicateDecomposerManager.java  |   83 +
 .../hive/ql/index/IndexPredicateAnalyzer.java   |  523 +
 .../hive/ql/index/IndexSearchCondition.java     |  143 +
 .../hive/ql/index/PredicateAnalyzerFactory.java |   40 +
 .../phoenix/hive/query/PhoenixQueryBuilder.java |  760 ++
 .../hive/util/PhoenixConnectionUtil.java        |   97 +
 .../hive/util/PhoenixStorageHandlerUtil.java    |  278 +
 .../apache/phoenix/hive/util/PhoenixUtil.java   |  210 +
 phoenix-pherf/pom.xml                           |  317 +-
 .../apache/phoenix/pherf/ResultBaseTestIT.java  |    4 +-
 .../apache/phoenix/pherf/SchemaReaderIT.java    |    5 +-
 .../phoenix/pherf/result/ResultManager.java     |   11 +-
 .../pherf/result/impl/ImageResultHandler.java   |  126 -
 .../src/test/resources/pherf.test.properties    |   60 +-
 phoenix-pig/pom.xml                             |   10 +-
 .../java/org/apache/phoenix/pig/BasePigIT.java  |    6 +-
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       |    6 +-
 .../phoenix/pig/PhoenixHBaseStorerIT.java       |    4 +-
 .../apache/phoenix/pig/PhoenixHBaseLoader.java  |    4 +-
 .../phoenix/pig/util/PhoenixPigSchemaUtil.java  |    4 +-
 .../pig/util/QuerySchemaParserFunction.java     |    4 +-
 .../pig/util/SqlQueryToColumnInfoFunction.java  |    4 +-
 .../pig/util/TableSchemaParserFunction.java     |    4 +-
 .../pig/util/PhoenixPigSchemaUtilTest.java      |    4 +-
 .../pig/util/QuerySchemaParserFunctionTest.java |    4 +-
 .../util/SqlQueryToColumnInfoFunctionTest.java  |    4 +-
 .../pig/util/TableSchemaParserFunctionTest.java |    4 +-
 phoenix-protocol/src/main/MetaDataService.proto |   54 +-
 phoenix-protocol/src/main/PFunction.proto       |    2 -
 phoenix-protocol/src/main/PGuidePosts.proto     |    4 +-
 phoenix-protocol/src/main/PSchema.proto         |   28 +
 phoenix-protocol/src/main/PTable.proto          |    9 +-
 .../src/main/ServerCacheFactory.proto           |    2 -
 .../src/main/ServerCachingService.proto         |    2 -
 phoenix-queryserver-client/pom.xml              |  187 +
 .../phoenix/queryserver/client/Driver.java      |   49 +
 .../queryserver/client/SqllineWrapper.java      |   87 +
 .../queryserver/client/ThinClientUtil.java      |   42 +
 .../resources/META-INF/services/java.sql.Driver |    1 +
 .../org-apache-phoenix-remote-jdbc.properties   |   25 +
 phoenix-queryserver/pom.xml                     |  170 +
 .../src/build/query-server-runnable.xml         |   52 +
 .../phoenix/end2end/QueryServerBasicsIT.java    |  164 +
 .../phoenix/end2end/QueryServerThread.java      |   45 +
 .../src/it/resources/log4j.properties           |   63 +
 .../apache/phoenix/queryserver/server/Main.java |  337 +
 .../queryserver/server/PhoenixMetaFactory.java  |   28 +
 .../server/PhoenixMetaFactoryImpl.java          |   76 +
 .../apache/phoenix/DriverCohabitationTest.java  |   65 +
 .../server/PhoenixDoAsCallbackTest.java         |   89 +
 phoenix-server-client/pom.xml                   |   66 -
 phoenix-server-client/src/build/thin-client.xml |   49 -
 .../phoenix/queryserver/client/Driver.java      |   49 -
 .../queryserver/client/ThinClientUtil.java      |   42 -
 .../resources/META-INF/services/java.sql.Driver |    1 -
 .../org-apache-phoenix-remote-jdbc.properties   |   25 -
 phoenix-server/pom.xml                          |  195 +-
 .../src/build/query-server-runnable.xml         |   52 -
 .../phoenix/end2end/QueryServerBasicsIT.java    |  157 -
 .../phoenix/end2end/QueryServerThread.java      |   45 -
 .../src/it/resources/log4j.properties           |   63 -
 .../apache/phoenix/queryserver/server/Main.java |  236 -
 .../queryserver/server/PhoenixMetaFactory.java  |   28 -
 .../server/PhoenixMetaFactoryImpl.java          |   76 -
 .../apache/phoenix/DriverCohabitationTest.java  |   65 -
 phoenix-spark/README.md                         |   19 +-
 phoenix-spark/pom.xml                           |    4 +-
 phoenix-spark/src/it/resources/setup.sql        |    6 +
 .../apache/phoenix/spark/PhoenixSparkIT.scala   |   86 +-
 .../apache/phoenix/spark/DefaultSource.scala    |    3 +-
 .../org/apache/phoenix/spark/PhoenixRDD.scala   |   27 +-
 .../apache/phoenix/spark/PhoenixRelation.scala  |   34 +-
 phoenix-tracing-webapp/pom.xml                  |    2 +-
 .../src/main/webapp/js/api/chart-model.js       |   20 +-
 .../src/main/webapp/js/app.js                   |   20 +-
 .../src/main/webapp/js/config/chart-config.js   |   20 +-
 .../js/controllers/accordion-controllers.js     |   18 +
 .../controllers/dependency-tree-controllers.js  |   20 +-
 .../webapp/js/controllers/list-controllers.js   |   20 +-
 .../webapp/js/controllers/search-controllers.js |   20 +-
 .../js/controllers/timeline-controllers.js      |   20 +-
 .../js/controllers/trace-count-controllers.js   |   20 +-
 .../trace-distribution-controllers.js           |   20 +-
 .../js/factories/statement-factory-config.js    |   20 +-
 .../webapp/js/factories/statement-factory.js    |   20 +-
 .../js/services/generate-statement-service.js   |   20 +-
 .../src/test/webapp/js/specs/app-route-spec.js  |   20 +-
 .../test/webapp/js/specs/timeline-ctrl-spec.js  |   20 +-
 .../webapp/js/specs/trace-list-ctrl-spec.js     |   20 +-
 .../webapp/js/specs/tracing-app-ctrl-spec.js    |   20 +-
 pom.xml                                         |  324 +-
 630 files changed, 45460 insertions(+), 16145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
index 46bcaa0,0000000..46a3053
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
@@@ -1,345 -1,0 +1,346 @@@
 +package org.apache.phoenix.calcite;
 +
 +import com.google.common.collect.ImmutableList;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +
 +import org.apache.calcite.jdbc.CalciteSchema;
 +import org.apache.calcite.linq4j.tree.Expression;
 +import org.apache.calcite.materialize.MaterializationService;
 +import org.apache.calcite.schema.*;
 +import org.apache.calcite.schema.impl.ViewTable;
 +import org.apache.phoenix.compile.ColumnResolver;
 +import org.apache.phoenix.compile.FromCompiler;
 +import org.apache.phoenix.jdbc.PhoenixConnection;
 +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 +import org.apache.phoenix.parse.ColumnDef;
 +import org.apache.phoenix.parse.NamedTableNode;
 +import org.apache.phoenix.parse.TableName;
 +import org.apache.phoenix.schema.MetaDataClient;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.PTable.ViewType;
 +import org.apache.phoenix.schema.PTableImpl;
 +import org.apache.phoenix.schema.PTableType;
 +import org.apache.phoenix.schema.TableNotFoundException;
 +import org.apache.phoenix.schema.TableRef;
 +import org.apache.phoenix.util.IndexUtil;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import java.sql.Connection;
 +import java.sql.DriverManager;
 +import java.sql.ResultSet;
 +import java.sql.SQLException;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +
 +/**
 + * Implementation of Calcite's {@link Schema} SPI for Phoenix.
 + * 
 + * TODO
 + * 1) change this to non-caching mode??
 + * 2) how to deal with define indexes and views since they require a CalciteSchema
 + *    instance??
 + *
 + */
 +public class PhoenixSchema implements Schema {
 +    public static final Factory FACTORY = new Factory();
 +
 +    public final PhoenixConnection pc;
 +    
 +    protected final String name;
 +    protected final String schemaName;
 +    protected final SchemaPlus parentSchema;
 +    protected final MetaDataClient client;
 +    
 +    protected final Map<String, Schema> subSchemas;
 +    protected final Map<String, Table> tables;
 +    protected final Map<String, Function> views;
 +    protected final Set<TableRef> viewTables;
 +    
 +    protected PhoenixSchema(String name, String schemaName,
 +            SchemaPlus parentSchema, PhoenixConnection pc) {
 +        this.name = name;
 +        this.schemaName = schemaName;
 +        this.parentSchema = parentSchema;
 +        this.pc = pc;
 +        this.client = new MetaDataClient(pc);
 +        this.subSchemas = Maps.newHashMap();
 +        this.tables = Maps.newHashMap();
 +        this.views = Maps.newHashMap();
 +        this.viewTables = Sets.newHashSet();
 +    }
 +
 +    private static Schema create(SchemaPlus parentSchema,
 +            String name, Map<String, Object> operand) {
 +        String url = (String) operand.get("url");
 +        final Properties properties = new Properties();
 +        for (Map.Entry<String, Object> entry : operand.entrySet()) {
 +            properties.setProperty(entry.getKey(), String.valueOf(entry.getValue()));
 +        }
 +        try {
 +            Class.forName("org.apache.phoenix.jdbc.PhoenixDriver");
 +            final Connection connection =
 +                DriverManager.getConnection(url, properties);
 +            final PhoenixConnection phoenixConnection =
 +                connection.unwrap(PhoenixConnection.class);
 +            return new PhoenixSchema(name, null, parentSchema, phoenixConnection);
 +        } catch (ClassNotFoundException e) {
 +            throw new RuntimeException(e);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    @Override
 +    public Table getTable(String name) {
 +        Table table = tables.get(name);
 +        if (table != null) {
 +            return table;
 +        }
 +        
 +        try {
 +            ColumnResolver x = FromCompiler.getResolver(
 +                    NamedTableNode.create(
 +                            null,
 +                            TableName.create(schemaName, name),
 +                            ImmutableList.<ColumnDef>of()), pc);
 +            final List<TableRef> tables = x.getTables();
 +            assert tables.size() == 1;
 +            TableRef tableRef = tables.get(0);
 +            if (!isView(tableRef.getTable())) {
 +                tableRef = fixTableMultiTenancy(tableRef);
 +                table = new PhoenixTable(pc, tableRef);
 +            }
 +        } catch (TableNotFoundException e) {
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        if (table == null) {
 +            table = resolveSequence(name);
 +        }
 +        
 +        if (table != null) {
 +            tables.put(name, table);
 +        }
 +        return table;
 +    }
 +
 +    @Override
 +    public Set<String> getTableNames() {
 +        return tables.keySet();
 +    }
 +
 +    @Override
 +    public Collection<Function> getFunctions(String name) {
 +        Function func = views.get(name);
 +        if (func != null) {
 +            return ImmutableList.of(func);
 +        }
 +        
 +        try {
 +            ColumnResolver x = FromCompiler.getResolver(
 +                    NamedTableNode.create(
 +                            null,
 +                            TableName.create(schemaName, name),
 +                            ImmutableList.<ColumnDef>of()), pc);
 +            final List<TableRef> tables = x.getTables();
 +            assert tables.size() == 1;
 +            final TableRef tableRef = tables.get(0);
 +            final PTable pTable = tableRef.getTable();
 +            if (isView(pTable)) {
 +                String viewSql = pTable.getViewStatement();
 +                if (viewSql == null) {
 +                    viewSql = "select * from "
 +                            + SchemaUtil.getEscapedFullTableName(
 +                                    pTable.getPhysicalName().getString());
 +                }
 +                SchemaPlus schema = parentSchema.getSubSchema(this.name);
 +                SchemaPlus viewSqlSchema =
 +                        this.schemaName == null ? schema : parentSchema;
 +                func = ViewTable.viewMacro(schema, viewSql,
 +                        CalciteSchema.from(viewSqlSchema).path(null),
 +                        pTable.getViewType() == ViewType.UPDATABLE);
 +                views.put(name, func);
 +                viewTables.add(tableRef);
 +            }
 +        } catch (TableNotFoundException e) {
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return func == null ? Collections.<Function>emptyList() : ImmutableList.of(func);
 +    }
 +
 +    @Override
 +    public Set<String> getFunctionNames() {
 +        return views.keySet();
 +    }
 +
 +    @Override
 +    public Schema getSubSchema(String name) {
 +        if (schemaName != null) {
 +            return null;
 +        }
 +        
 +        Schema schema = subSchemas.get(name);
 +        if (schema != null) {
 +            return schema;
 +        }
 +        
 +        schema = new PhoenixSchema(name, name, parentSchema.getSubSchema(this.name), pc);
 +        subSchemas.put(name, schema);
 +        return schema;
 +    }
 +
 +    @Override
 +    public Set<String> getSubSchemaNames() {
 +        return subSchemas.keySet();
 +    }
 +
 +    @Override
 +    public Expression getExpression(SchemaPlus parentSchema, String name) {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
 +    public boolean isMutable() {
 +        return true;
 +    }
 +
 +    @Override
 +    public boolean contentsHaveChangedSince(long lastCheck, long now) {
 +        return false;
 +    }
 +    
 +    public void defineIndexesAsMaterializations() {
 +        SchemaPlus schema = parentSchema.getSubSchema(this.name);
 +        SchemaPlus viewSqlSchema =
 +                this.schemaName == null ? schema : parentSchema;
 +        CalciteSchema calciteSchema = CalciteSchema.from(schema);
 +        List<String> path = CalciteSchema.from(viewSqlSchema).path(null);
 +        try {
 +            for (Table table : tables.values()) {
 +                if (table instanceof PhoenixTable) {
 +                    TableRef tableRef = ((PhoenixTable) table).tableMapping.getTableRef();
 +                    for (PTable index : tableRef.getTable().getIndexes()) {
 +                        TableRef indexTableRef = new TableRef(null, index,
 +                                tableRef.getTimeStamp(), tableRef.getLowerBoundTimeStamp(),
 +                                false);
 +                        addMaterialization(indexTableRef, path, calciteSchema);
 +                    }
 +                }
 +            }
 +            for (TableRef tableRef : viewTables) {
 +                final PTable pTable = tableRef.getTable();
 +                for (PTable index : pTable.getIndexes()) {
 +                    if (index.getParentName().equals(pTable.getName())) {
 +                        TableRef indexTableRef = new TableRef(null, index,
 +                                tableRef.getTimeStamp(), tableRef.getLowerBoundTimeStamp(),
 +                                false);
 +                        addMaterialization(indexTableRef, path, calciteSchema);
 +                    }
 +                }                
 +            }
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    private void addMaterialization(TableRef indexTableRef, List<String> path,
 +            CalciteSchema calciteSchema) throws SQLException {
 +        indexTableRef = fixTableMultiTenancy(indexTableRef);
 +        final PhoenixTable table = new PhoenixTable(pc, indexTableRef);
 +        final PTable index = indexTableRef.getTable();
 +        tables.put(index.getTableName().getString(), table);
 +        StringBuffer sb = new StringBuffer();
 +        sb.append("SELECT");
 +        for (PColumn column : table.getColumns()) {
 +            String indexColumnName = column.getName().getString();
 +            String dataColumnName = IndexUtil.getDataColumnName(indexColumnName);
 +            sb.append(",").append(SchemaUtil.getEscapedFullColumnName(dataColumnName));
 +            sb.append(" ").append(SchemaUtil.getEscapedFullColumnName(indexColumnName));
 +        }
 +        sb.setCharAt(6, ' '); // replace first comma with space.
 +        sb.append(" FROM ").append(SchemaUtil.getEscapedFullTableName(index.getParentName().getString()));
 +        MaterializationService.instance().defineMaterialization(
 +                calciteSchema, null, sb.toString(), path, index.getTableName().getString(), true, true);        
 +    }
 +    
 +    private boolean isView(PTable table) {
 +        return table.getType() == PTableType.VIEW
 +                && table.getViewType() != ViewType.MAPPED;
 +    }
 +    
 +    private TableRef fixTableMultiTenancy(TableRef tableRef) throws SQLException {
 +        if (pc.getTenantId() != null || !tableRef.getTable().isMultiTenant()) {
 +            return tableRef;
 +        }
 +        PTable table = tableRef.getTable();
 +        table = PTableImpl.makePTable(
 +                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
 +                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), PTableImpl.getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
 +                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
 +                table.isWALDisabled(), false, table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                 table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getTableStats(), table.getBaseColumnCount(), table.getIndexDisableTimestamp());
++                table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getBaseColumnCount(), table.getIndexDisableTimestamp(),
++                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
 +        return new TableRef(null, table, tableRef.getTimeStamp(),
 +                tableRef.getLowerBoundTimeStamp(), tableRef.hasDynamicCols());
 +    }
 +    
 +    private PhoenixSequence resolveSequence(String name) {
 +        try {
 +            // FIXME: Do this the same way as resolving a table after PHOENIX-2489.
 +            String tenantId = pc.getTenantId() == null ? null : pc.getTenantId().getString();
 +            String q = "select 1 from " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE
 +                    + " where " + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA
 +                    + (schemaName == null ? " is null" : " = '" + schemaName + "'")
 +                    + " and " + PhoenixDatabaseMetaData.SEQUENCE_NAME
 +                    + " = '" + name + "'"
 +                    + " and " + PhoenixDatabaseMetaData.TENANT_ID
 +                    + (tenantId == null ? " is null" : " = '" + tenantId + "'");
 +            ResultSet rs = pc.createStatement().executeQuery(q);
 +            if (rs.next()) {
 +                return new PhoenixSequence(schemaName, name, pc);
 +            }
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return null;
 +    }
 +
 +    /** Schema factory that creates a
 +     * {@link org.apache.phoenix.calcite.PhoenixSchema}.
 +     * This allows you to create a Phoenix schema inside a model.json file.
 +     *
 +     * <pre>{@code
 +     * {
 +     *   version: '1.0',
 +     *   defaultSchema: 'HR',
 +     *   schemas: [
 +     *     {
 +     *       name: 'HR',
 +     *       type: 'custom',
 +     *       factory: 'org.apache.phoenix.calcite.PhoenixSchema.Factory',
 +     *       operand: {
 +     *         url: "jdbc:phoenix:localhost",
 +     *         user: "scott",
 +     *         password: "tiger"
 +     *       }
 +     *     }
 +     *   ]
 +     * }
 +     * }</pre>
 +     */
 +    public static class Factory implements SchemaFactory {
 +        public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
 +            return PhoenixSchema.create(parentSchema, name, operand);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
index ccf460f,0000000..a8dd9d2
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/TableMapping.java
@@@ -1,368 -1,0 +1,369 @@@
 +package org.apache.phoenix.calcite;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.sql.SQLException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Set;
 +
 +import org.apache.calcite.plan.RelOptUtil.InputFinder;
 +import org.apache.calcite.rex.RexNode;
 +import org.apache.calcite.util.ImmutableBitSet;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.hadoop.hbase.util.Bytes;
 +import org.apache.hadoop.hbase.util.Pair;
 +import org.apache.hadoop.io.WritableUtils;
 +import org.apache.phoenix.compile.ColumnProjector;
 +import org.apache.phoenix.compile.ExpressionProjector;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.compile.TupleProjectionCompiler;
 +import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 +import org.apache.phoenix.execute.TupleProjector;
 +import org.apache.phoenix.expression.ColumnExpression;
 +import org.apache.phoenix.expression.Expression;
 +import org.apache.phoenix.expression.LiteralExpression;
 +import org.apache.phoenix.index.IndexMaintainer;
 +import org.apache.phoenix.jdbc.PhoenixConnection;
 +import org.apache.phoenix.schema.ColumnRef;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PName;
 +import org.apache.phoenix.schema.PNameFactory;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.PTableImpl;
 +import org.apache.phoenix.schema.PTableType;
 +import org.apache.phoenix.schema.ProjectedColumn;
 +import org.apache.phoenix.schema.TableRef;
 +import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
 +import org.apache.phoenix.schema.PTable.IndexType;
 +import org.apache.phoenix.util.ByteUtil;
 +import org.apache.phoenix.util.IndexUtil;
 +import org.apache.phoenix.util.MetaDataUtil;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.Sets;
 +
 +public class TableMapping {
 +    private final TableRef tableRef;
 +    private final TableRef dataTableRef;
 +    private final List<PColumn> mappedColumns;
 +    private final int extendedColumnsOffset;
 +    private final TableRef extendedTableRef;
 +
 +    public TableMapping(PTable table) {
 +        this.tableRef = new TableRef(table);
 +        this.dataTableRef = null;
 +        this.mappedColumns = getMappedColumns(table);
 +        this.extendedColumnsOffset = mappedColumns.size();
 +        this.extendedTableRef = null;
 +    }
 +
 +    public TableMapping(TableRef tableRef, TableRef dataTableRef, boolean extend) throws SQLException {
 +        this.tableRef = tableRef;
 +        this.dataTableRef = dataTableRef;
 +        if (!extend) {
 +            this.mappedColumns = getMappedColumns(tableRef.getTable());
 +            this.extendedColumnsOffset = mappedColumns.size();
 +            this.extendedTableRef = null;            
 +        } else {
 +            this.mappedColumns = Lists.newArrayList();
 +            this.mappedColumns.addAll(getMappedColumns(tableRef.getTable()));
 +            this.extendedColumnsOffset = mappedColumns.size();
 +            Set<String> names = Sets.newHashSet();
 +            for (PColumn column : this.mappedColumns) {
 +                names.add(column.getName().getString());
 +            }
 +            PTable dataTable = dataTableRef.getTable();
 +            List<PColumn> projectedColumns = new ArrayList<PColumn>();
 +            for (PColumn sourceColumn : dataTable.getColumns()) {
 +                if (!SchemaUtil.isPKColumn(sourceColumn)) {
 +                    String colName = IndexUtil.getIndexColumnName(sourceColumn);
 +                    if (!names.contains(colName)) {
 +                        ColumnRef sourceColumnRef =
 +                                new ColumnRef(dataTableRef, sourceColumn.getPosition());
 +                        PColumn column = new ProjectedColumn(PNameFactory.newName(colName),
 +                                sourceColumn.getFamilyName(), projectedColumns.size(),
 +                                sourceColumn.isNullable(), sourceColumnRef);
 +                        projectedColumns.add(column);
 +                    }
 +                }            
 +            }
 +            this.mappedColumns.addAll(projectedColumns);
 +            PTable extendedTable = PTableImpl.makePTable(dataTable.getTenantId(),
 +                    TupleProjectionCompiler.PROJECTED_TABLE_SCHEMA, dataTable.getName(),
 +                    PTableType.PROJECTED, null, dataTable.getTimeStamp(),
 +                    dataTable.getSequenceNumber(), dataTable.getPKName(), null,
 +                    projectedColumns, null, null, Collections.<PTable>emptyList(),
 +                    dataTable.isImmutableRows(), Collections.<PName>emptyList(), null, null,
 +                    dataTable.isWALDisabled(), false, dataTable.getStoreNulls(),
 +                    dataTable.getViewType(), null, null, dataTable.rowKeyOrderOptimizable(),
 +                    dataTable.isTransactional(), dataTable.getUpdateCacheFrequency(),
-                     dataTable.getIndexDisableTimestamp());
++                    dataTable.getIndexDisableTimestamp(), dataTable.isNamespaceMapped(),
++                    dataTable.getAutoPartitionSeqName(), dataTable.isAppendOnlySchema());
 +            this.extendedTableRef = new TableRef(extendedTable);
 +        }
 +    }
 +    
 +    public TableRef getTableRef() {
 +        return tableRef;
 +    }
 +    
 +    public PTable getPTable() {
 +        return tableRef.getTable();
 +    }
 +    
 +    public TableRef getDataTableRef() {
 +        return dataTableRef;
 +    }
 +    
 +    public List<PColumn> getMappedColumns() {
 +        return mappedColumns;
 +    }
 +    
 +    public boolean hasExtendedColumns() {
 +        return extendedTableRef != null;
 +    }
 +    
 +    public ColumnExpression newColumnExpression(int index) {
 +        ColumnRef colRef = new ColumnRef(
 +                index < extendedColumnsOffset ? tableRef : extendedTableRef,
 +                this.mappedColumns.get(index).getPosition());
 +        return colRef.newColumnExpression();
 +    }
 +    
 +    public ImmutableBitSet getDefaultExtendedColumnRef() {
 +        return ImmutableBitSet.range(extendedColumnsOffset, mappedColumns.size());
 +    }
 +    
 +    public ImmutableBitSet getExtendedColumnRef(List<RexNode> exprs) {
 +        if (!hasExtendedColumns()) {
 +            return ImmutableBitSet.of();
 +        }
 +        
 +        ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
 +        for (RexNode expr : exprs) {
 +            builder.addAll(InputFinder.analyze(expr).inputBitSet.build());
 +        }
 +        for (int i = 0; i < extendedColumnsOffset; i++) {
 +            builder.clear(i);
 +        }
 +        return builder.build();
 +    }
 +   
 +    public Pair<Integer, Integer> getExtendedColumnReferenceCount(ImmutableBitSet columnRef) {
 +        Set<String> cf = Sets.newHashSet();
 +        int columnCount = 0;
 +        for (int i = extendedColumnsOffset; i < mappedColumns.size(); i++) {
 +            if (columnRef.get(i)) {
 +                PColumn dataColumn = ((ProjectedColumn) mappedColumns.get(i))
 +                        .getSourceColumnRef().getColumn();
 +                cf.add(dataColumn.getFamilyName().getString());
 +                columnCount++;
 +            }
 +        }
 +        return new Pair<Integer, Integer>(cf.size(), columnCount);
 +    }
 +    
 +    public PTable createProjectedTable(boolean retainPKColumns) {
 +        List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
 +        List<PColumn> columns = retainPKColumns ?
 +                  tableRef.getTable().getColumns() : mappedColumns.subList(0, extendedColumnsOffset);
 +        for (PColumn column : columns) {
 +            sourceColumnRefs.add(new ColumnRef(tableRef, column.getPosition()));
 +        }
 +        if (extendedColumnsOffset < mappedColumns.size()) {
 +            for (PColumn column : mappedColumns.subList(extendedColumnsOffset, mappedColumns.size())) {
 +                sourceColumnRefs.add(new ColumnRef(extendedTableRef, column.getPosition()));
 +            }
 +        }
 +        
 +        try {
 +            return TupleProjectionCompiler.createProjectedTable(tableRef, sourceColumnRefs, retainPKColumns);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    public TupleProjector createTupleProjector(boolean retainPKColumns) {
 +        KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
 +        List<Expression> exprs = Lists.<Expression> newArrayList();
 +        for (int i = 0; i < mappedColumns.size(); i++) {
 +            if (!SchemaUtil.isPKColumn(mappedColumns.get(i)) || !retainPKColumns) {
 +                Expression expr = newColumnExpression(i);
 +                exprs.add(expr);
 +                builder.addField(expr);
 +            }
 +        }
 +        
 +        return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));
 +    }
 +    
 +    public RowProjector createRowProjector() {
 +        List<ColumnProjector> columnProjectors = Lists.<ColumnProjector>newArrayList();
 +        for (int i = 0; i < mappedColumns.size(); i++) {
 +            PColumn column = mappedColumns.get(i);
 +            Expression expr = newColumnExpression(i); // Do not use column.position() here.
 +            columnProjectors.add(new ExpressionProjector(column.getName().getString(), tableRef.getTable().getName().getString(), expr, false));
 +        }
 +        // TODO get estimate row size
 +        return new RowProjector(columnProjectors, 0, false);        
 +    }
 +    
 +    public void setupScanForExtendedTable(Scan scan, ImmutableBitSet extendedColumnRef,
 +            PhoenixConnection connection) throws SQLException {
 +        if (extendedTableRef == null || extendedColumnRef.isEmpty()) {
 +            return;
 +        }
 +        
 +        TableRef dataTableRef = null;
 +        List<PColumn> dataColumns = Lists.newArrayList();
 +        KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
 +        List<Expression> exprs = Lists.<Expression> newArrayList();
 +        for (int i = extendedColumnsOffset; i < mappedColumns.size(); i++) {
 +            ProjectedColumn column = (ProjectedColumn) mappedColumns.get(i);
 +            builder.addField(column);
 +            if (extendedColumnRef.get(i)) {
 +                dataColumns.add(column.getSourceColumnRef().getColumn());
 +                exprs.add(column.getSourceColumnRef().newColumnExpression());
 +                if (dataTableRef == null) {
 +                    dataTableRef = column.getSourceColumnRef().getTableRef();
 +                }
 +            } else {
 +                exprs.add(LiteralExpression.newConstant(null));
 +            }
 +        }
 +        if (dataColumns.isEmpty()) {
 +            return;
 +        }
 +        
 +        // Set data columns to be join back from data table.
 +        serializeDataTableColumnsToJoin(scan, dataColumns);
 +        // Set tuple projector of the data columns.
 +        TupleProjector projector = new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));
 +        TupleProjector.serializeProjectorIntoScan(scan, projector, IndexUtil.INDEX_PROJECTOR);
 +        PTable dataTable = dataTableRef.getTable();
 +        // Set index maintainer of the local index.
 +        serializeIndexMaintainerIntoScan(scan, dataTable, connection);
 +        // Set view constants if exists.
 +        serializeViewConstantsIntoScan(scan, dataTable);
 +    }
 +
 +    private static void serializeDataTableColumnsToJoin(Scan scan, List<PColumn> dataColumns) {
 +        ByteArrayOutputStream stream = new ByteArrayOutputStream();
 +        try {
 +            DataOutputStream output = new DataOutputStream(stream);
 +            WritableUtils.writeVInt(output, dataColumns.size());
 +            for (PColumn column : dataColumns) {
 +                Bytes.writeByteArray(output, column.getFamilyName().getBytes());
 +                Bytes.writeByteArray(output, column.getName().getBytes());
 +            }
 +            scan.setAttribute(BaseScannerRegionObserver.DATA_TABLE_COLUMNS_TO_JOIN, stream.toByteArray());
 +        } catch (IOException e) {
 +            throw new RuntimeException(e);
 +        } finally {
 +            try {
 +                stream.close();
 +            } catch (IOException e) {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +    }
 +
 +    private void serializeIndexMaintainerIntoScan(Scan scan, PTable dataTable, PhoenixConnection connection) throws SQLException {
 +        PName name = getPTable().getName();
 +        List<PTable> indexes = Lists.newArrayListWithExpectedSize(1);
 +        for (PTable index : dataTable.getIndexes()) {
 +            if (index.getName().equals(name) && index.getIndexType() == IndexType.LOCAL) {
 +                indexes.add(index);
 +                break;
 +            }
 +        }
 +        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +        IndexMaintainer.serialize(dataTable, ptr, indexes, connection);
 +        scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD, ByteUtil.copyKeyBytesIfNecessary(ptr));
 +        if (dataTable.isTransactional()) {
 +            scan.setAttribute(BaseScannerRegionObserver.TX_STATE, connection.getMutationState().encodeTransaction());
 +        }
 +    }
 +
 +    private static void serializeViewConstantsIntoScan(Scan scan, PTable dataTable) {
 +        int dataPosOffset = (dataTable.getBucketNum() != null ? 1 : 0) + (dataTable.isMultiTenant() ? 1 : 0);
 +        int nViewConstants = 0;
 +        if (dataTable.getType() == PTableType.VIEW) {
 +            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +            List<PColumn> dataPkColumns = dataTable.getPKColumns();
 +            for (int i = dataPosOffset; i < dataPkColumns.size(); i++) {
 +                PColumn dataPKColumn = dataPkColumns.get(i);
 +                if (dataPKColumn.getViewConstant() != null) {
 +                    nViewConstants++;
 +                }
 +            }
 +            if (nViewConstants > 0) {
 +                byte[][] viewConstants = new byte[nViewConstants][];
 +                int j = 0;
 +                for (int i = dataPosOffset; i < dataPkColumns.size(); i++) {
 +                    PColumn dataPkColumn = dataPkColumns.get(i);
 +                    if (dataPkColumn.getViewConstant() != null) {
 +                        if (IndexUtil.getViewConstantValue(dataPkColumn, ptr)) {
 +                            viewConstants[j++] = ByteUtil.copyKeyBytesIfNecessary(ptr);
 +                        } else {
 +                            throw new IllegalStateException();
 +                        }
 +                    }
 +                }
 +                serializeViewConstantsIntoScan(viewConstants, scan);
 +            }
 +        }
 +    }
 +
 +    private static void serializeViewConstantsIntoScan(byte[][] viewConstants, Scan scan) {
 +        ByteArrayOutputStream stream = new ByteArrayOutputStream();
 +        try {
 +            DataOutputStream output = new DataOutputStream(stream);
 +            WritableUtils.writeVInt(output, viewConstants.length);
 +            for (byte[] viewConstant : viewConstants) {
 +                Bytes.writeByteArray(output, viewConstant);
 +            }
 +            scan.setAttribute(BaseScannerRegionObserver.VIEW_CONSTANTS, stream.toByteArray());
 +        } catch (IOException e) {
 +            throw new RuntimeException(e);
 +        } finally {
 +            try {
 +                stream.close();
 +            } catch (IOException e) {
 +                throw new RuntimeException(e);
 +            }
 +        }
 +    }
 +    
 +    private static List<PColumn> getMappedColumns(PTable pTable) {
 +        if (pTable.getBucketNum() == null
 +                && !pTable.isMultiTenant()
 +                && pTable.getViewIndexId() == null) {
 +            return pTable.getColumns();
 +        }
 +        
 +        List<PColumn> columns = Lists.newArrayList(pTable.getColumns());
 +        if (pTable.getViewIndexId() != null) {
 +            for (Iterator<PColumn> iter = columns.iterator(); iter.hasNext();) {
-                 if (iter.next().getName().getString().equals(MetaDataUtil.VIEW_INDEX_ID_COLUMN_NAME)) {
++                if (iter.next().getName().getString().equals(MetaDataUtil.getViewIndexIdColumnName())) {
 +                    iter.remove();
 +                    break;
 +                }
 +            }
 +        }
 +        if (pTable.isMultiTenant()) {
 +            columns.remove(pTable.getBucketNum() == null ? 0 : 1);
 +        }
 +        if (pTable.getBucketNum() != null) {
 +            columns.remove(0);
 +        }
 +        
 +        return columns;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a9526a94/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
index 96651e9,0000000..fa4649b
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
@@@ -1,140 -1,0 +1,140 @@@
 +package org.apache.phoenix.calcite.rel;
 +
 +import java.sql.SQLException;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Stack;
 +
 +import org.apache.phoenix.calcite.PhoenixSequence;
 +import org.apache.phoenix.calcite.TableMapping;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.SequenceManager;
 +import org.apache.phoenix.compile.SequenceValueExpression;
 +import org.apache.phoenix.coprocessor.MetaDataProtocol;
 +import org.apache.phoenix.execute.RuntimeContext;
 +import org.apache.phoenix.execute.TupleProjector;
 +import org.apache.phoenix.expression.BindParameterExpression;
 +import org.apache.phoenix.expression.ColumnExpression;
 +import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 +import org.apache.phoenix.expression.Expression;
 +import org.apache.phoenix.parse.ParseNodeFactory;
 +import org.apache.phoenix.parse.SequenceValueParseNode;
 +import org.apache.phoenix.parse.TableName;
 +import org.apache.phoenix.schema.KeyValueSchema;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PColumnImpl;
 +import org.apache.phoenix.schema.PName;
 +import org.apache.phoenix.schema.PNameFactory;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.PTableImpl;
 +import org.apache.phoenix.schema.PTableType;
 +import org.apache.phoenix.schema.types.PDataType;
 +import com.google.common.collect.Lists;
 +
 +public class PhoenixRelImplementorImpl implements PhoenixRelImplementor {
 +    private final RuntimeContext runtimeContext;
 +	private Stack<ImplementorContext> contextStack;
 +	private SequenceManager sequenceManager;
 +	private TableMapping tableMapping;
 +	
 +	public PhoenixRelImplementorImpl(RuntimeContext runtimeContext) {
 +	    this.runtimeContext = runtimeContext;
 +	    this.contextStack = new Stack<ImplementorContext>();
 +	}
 +	
 +    @Override
 +    public QueryPlan visitInput(int i, PhoenixQueryRel input) {
 +        return input.implement(this);
 +    }
 +
 +	@Override
 +	public ColumnExpression newColumnExpression(int index) {
 +		return tableMapping.newColumnExpression(index);
 +	}
 +    
 +    @SuppressWarnings("rawtypes")
 +    @Override
 +    public Expression newBindParameterExpression(int index, PDataType type, Integer maxLength) {
 +        return new BindParameterExpression(index, type, maxLength, runtimeContext);
 +    }
 +    
 +    @SuppressWarnings("rawtypes")
 +    @Override
 +    public Expression newFieldAccessExpression(String variableId, int index, PDataType type) {
 +        Expression fieldAccessExpr = runtimeContext.getCorrelateVariable(variableId).newExpression(index);
 +        return new CorrelateVariableFieldAccessExpression(runtimeContext, variableId, fieldAccessExpr);
 +    }
 +    
 +    @Override
 +    public SequenceValueExpression newSequenceExpression(PhoenixSequence seq, SequenceValueParseNode.Op op) {
 +        PName tenantName = seq.pc.getTenantId();
 +        TableName tableName = TableName.create(seq.schemaName, seq.sequenceName);
 +        try {
 +            return sequenceManager.newSequenceReference(tenantName, tableName, null, op);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    @Override
 +    public RuntimeContext getRuntimeContext() {
 +        return runtimeContext;
 +    }
 +
 +    @Override
 +	public void setTableMapping(TableMapping tableMapping) {
 +		this.tableMapping = tableMapping;
 +	}
 +    
 +    @Override
 +    public TableMapping getTableMapping() {
 +        return this.tableMapping;
 +    }
 +    
 +    @Override
 +    public void setSequenceManager(SequenceManager sequenceManager) {
 +        this.sequenceManager = sequenceManager;
 +    }
 +
 +    @Override
 +    public void pushContext(ImplementorContext context) {
 +        this.contextStack.push(context);
 +    }
 +
 +    @Override
 +    public ImplementorContext popContext() {
 +        return contextStack.pop();
 +    }
 +
 +    @Override
 +    public ImplementorContext getCurrentContext() {
 +        return contextStack.peek();
 +    }
 +    
 +    @Override
 +    public TupleProjector project(List<Expression> exprs) {
 +        KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
 +        List<PColumn> columns = Lists.<PColumn>newArrayList();
 +        for (int i = 0; i < exprs.size(); i++) {
 +            String name = ParseNodeFactory.createTempAlias();
 +            Expression expr = exprs.get(i);
 +            builder.addField(expr);
 +            columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY),
 +                    expr.getDataType(), expr.getMaxLength(), expr.getScale(), expr.isNullable(),
 +                    i, expr.getSortOrder(), null, null, false, name, false, false));
 +        }
 +        try {
 +            PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,
 +                    PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
 +                    null, null, columns, null, null, Collections.<PTable>emptyList(),
 +                    false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                     null, null, true, false, 0, 0);
++                    null, null, true, false, 0, 0, false, null, false);
 +            this.setTableMapping(new TableMapping(pTable));
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));        
 +    }
 +
 +}


[23/50] [abbrv] phoenix git commit: PHOENIX-3195 Addendum. (James Taylor)

Posted by ma...@apache.org.
PHOENIX-3195 Addendum. (James Taylor)


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

Branch: refs/heads/calcite
Commit: 5fecde8d892fdc441aaa4e3ea8e195112da88705
Parents: 1f7a05b
Author: Lars Hofhansl <la...@apache.org>
Authored: Mon Aug 22 20:21:54 2016 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Mon Aug 22 20:26:23 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/compile/ExpressionCompiler.java    | 6 +++++-
 .../main/java/org/apache/phoenix/compile/OrderByCompiler.java  | 3 +++
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5fecde8d/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index 1278494..0fd1876 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -393,6 +393,10 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
         return ref;
     }
 
+    protected void addColumn(PColumn column) {
+        context.getScan().addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+    }
+
     @Override
     public Expression visit(ColumnParseNode node) throws SQLException {
         ColumnRef ref = resolveColumn(node);
@@ -407,7 +411,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
             return LiteralExpression.newConstant(column.getDataType().toObject(ptr), column.getDataType());
         }
         if (tableRef.equals(context.getCurrentTable()) && !SchemaUtil.isPKColumn(column)) { // project only kv columns
-            context.getScan().addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+            addColumn(column);
         }
         Expression expression = ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
         Expression wrappedExpression = wrapGroupByExpression(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5fecde8d/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
index 6804375..9bc0c31 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -36,6 +36,7 @@ import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PInteger;
@@ -97,6 +98,8 @@ public class OrderByCompiler {
             compiler = new ExpressionCompiler(context, GroupBy.EMPTY_GROUP_BY) {
                 @Override
                 protected Expression addExpression(Expression expression) {return expression;}
+                @Override
+                protected void addColumn(PColumn column) {}
             };
         } else {
             compiler = new ExpressionCompiler(context, groupBy);


[05/50] [abbrv] phoenix git commit: PHOENIX-3130 Ignore ASYNC and build indexes synchronously based on config property

Posted by ma...@apache.org.
PHOENIX-3130 Ignore ASYNC and build indexes synchronously based on config property


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

Branch: refs/heads/calcite
Commit: 243e5e22a1338aa60f22b23c64f28d7c364bb53f
Parents: 92c7faf
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Mon Aug 1 12:00:27 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Aug 2 20:30:08 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/IndexToolIT.java |   4 +-
 .../phoenix/end2end/MutableIndexToolIT.java     |   3 +-
 .../end2end/index/AsyncImmutableIndexIT.java    |  73 +++++++------
 .../end2end/index/AsyncIndexAutoBuildIT.java    |  51 ---------
 .../end2end/index/AsyncIndexDisabledIT.java     |  78 ++++++++++++++
 .../end2end/index/AsyncIndexRegularBuildIT.java |  62 -----------
 .../end2end/index/AsyncIndexTestUtil.java       |  58 ----------
 .../apache/phoenix/end2end/index/IndexIT.java   | 105 ++----------------
 .../coprocessor/MetaDataRegionObserver.java     | 107 ++-----------------
 .../phoenix/mapreduce/PhoenixInputFormat.java   |   2 -
 .../phoenix/mapreduce/index/IndexToolUtil.java  |   4 -
 .../index/automation/PhoenixMRJobSubmitter.java |  22 +++-
 .../apache/phoenix/query/QueryConstants.java    |  21 +---
 .../org/apache/phoenix/query/QueryServices.java |   6 +-
 .../phoenix/query/QueryServicesOptions.java     |  13 +--
 .../apache/phoenix/schema/MetaDataClient.java   |  16 +--
 .../phoenix/query/QueryServicesTestImpl.java    |   4 +-
 17 files changed, 173 insertions(+), 456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 5621634..c66fea3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -37,6 +37,7 @@ import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -86,9 +87,8 @@ public class IndexToolIT extends BaseOwnClusterHBaseManagedTimeIT {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        serverProps.put(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, Boolean.toString(false));
         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
         clientProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
         setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
index cb41d2b..c335ff8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java
@@ -45,9 +45,8 @@ public class MutableIndexToolIT extends BaseOwnClusterHBaseManagedTimeIT {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(2);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        serverProps.put(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, Boolean.toString(false));
         setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), ReadOnlyProps.EMPTY_PROPS);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
index 6b65f9a..8c90b6e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncImmutableIndexIT.java
@@ -23,29 +23,41 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
-import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
-import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.schema.PTableType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.IndexToolIT;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.StringUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
-    private static final long MAX_WAIT_FOR_INDEX_BUILD_TIME_MS = 45000;
+import com.google.common.collect.Maps;
 
+public class AsyncImmutableIndexIT extends BaseOwnClusterHBaseManagedTimeIT {
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
+            QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        setUpRealDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+            ReadOnlyProps.EMPTY_PROPS);
+    }
+    
     @Test
     public void testDeleteFromImmutable() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" + 
+            conn.createStatement().execute("CREATE TABLE TEST_TABLE (\n" + 
                     "        pk1 VARCHAR NOT NULL,\n" + 
                     "        pk2 VARCHAR NOT NULL,\n" + 
                     "        pk3 VARCHAR\n" + 
@@ -56,45 +68,37 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
                     "        pk3\n" + 
                     "        )\n" + 
                     "        ) IMMUTABLE_ROWS=true");
-            conn.createStatement().execute("upsert into " + tableName + " (pk1, pk2, pk3) values ('a', '1', '1')");
-            conn.createStatement().execute("upsert into " + tableName + " (pk1, pk2, pk3) values ('b', '2', '2')");
+            conn.createStatement().execute("upsert into TEST_TABLE (pk1, pk2, pk3) values ('a', '1', '1')");
+            conn.createStatement().execute("upsert into TEST_TABLE (pk1, pk2, pk3) values ('b', '2', '2')");
             conn.commit();
-            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + " (pk3, pk2) ASYNC");
+            conn.createStatement().execute("CREATE INDEX TEST_INDEX ON TEST_TABLE (pk3, pk2) ASYNC");
             
             // this delete will be issued at a timestamp later than the above timestamp of the index table
-            conn.createStatement().execute("delete from " + tableName + " where pk1 = 'a'");
+            conn.createStatement().execute("delete from TEST_TABLE where pk1 = 'a'");
             conn.commit();
 
-            DatabaseMetaData dbmd = conn.getMetaData();
-            String escapedTableName = StringUtil.escapeLike(indexName);
-            String[] tableType = new String[] {PTableType.INDEX.toString()};
-            long startTime = System.currentTimeMillis();
-            boolean isIndexActive = false;
-            do {
-                ResultSet rs = dbmd.getTables("", "", escapedTableName, tableType);
-                assertTrue(rs.next());
-                if (PIndexState.valueOf(rs.getString("INDEX_STATE")) == PIndexState.ACTIVE) {
-                    isIndexActive = true;
-                    break;
-                }
-                Thread.sleep(3000);
-            } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_INDEX_BUILD_TIME_MS);
-            assertTrue(isIndexActive);
+            // run the index MR job
+            final IndexTool indexingTool = new IndexTool();
+            indexingTool.setConf(new Configuration(getUtility().getConfiguration()));
+            final String[] cmdArgs =
+                    IndexToolIT.getArgValues(null, "TEST_TABLE", "TEST_INDEX", true);
+            int status = indexingTool.run(cmdArgs);
+            assertEquals(0, status);
 
             // upsert two more rows
             conn.createStatement().execute(
-                "upsert into " + tableName + " (pk1, pk2, pk3) values ('a', '3', '3')");
+                "upsert into TEST_TABLE (pk1, pk2, pk3) values ('a', '3', '3')");
             conn.createStatement().execute(
-                "upsert into " + tableName + " (pk1, pk2, pk3) values ('b', '4', '4')");
+                "upsert into TEST_TABLE (pk1, pk2, pk3) values ('b', '4', '4')");
             conn.commit();
 
             // validate that delete markers were issued correctly and only ('a', '1', 'value1') was
             // deleted
-            String query = "SELECT pk3 from " + tableName + " ORDER BY pk3";
+            String query = "SELECT pk3 from TEST_TABLE ORDER BY pk3";
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             String expectedPlan =
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexName + "\n" + 
-                    "    SERVER FILTER BY FIRST KEY ONLY";
+                    "CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER TEST_INDEX\n"
+                            + "    SERVER FILTER BY FIRST KEY ONLY";
             assertEquals("Wrong plan ", expectedPlan, QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
@@ -108,4 +112,3 @@ public class AsyncImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
deleted file mode 100644
index 9e7862b..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexAutoBuildIT.java
+++ /dev/null
@@ -1,51 +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.end2end.index;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.Statement;
-
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
-import org.junit.Test;
-
-public class AsyncIndexAutoBuildIT extends BaseHBaseManagedTimeTableReuseIT {
-    
-    @Test
-    public void testAsyncIndexAutoBuild() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
-        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
-
-        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
-        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-        
-        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
-
-        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertFalse(rs.next());
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
new file mode 100644
index 0000000..1e4cbcc
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexDisabledIT.java
@@ -0,0 +1,78 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Map;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class AsyncIndexDisabledIT extends BaseHBaseManagedTimeTableReuseIT {
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.INDEX_ASYNC_BUILD_ENABLED, Boolean.toString(false));
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS, new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testAsyncIndexRegularBuild() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            Statement stmt = conn.createStatement();
+            String tableName = "TBL_" + generateRandomString();
+            String indexName = "IND_" + generateRandomString();
+            
+            String ddl = "CREATE TABLE " + tableName + " (pk INTEGER NOT NULL PRIMARY KEY, val VARCHAR)";
+            stmt.execute(ddl);
+            stmt.execute("UPSERT INTO " + tableName + " values(1, 'y')");
+            // create the async index
+            stmt.execute("CREATE INDEX " + indexName + " ON " + tableName + "(val) ASYNC");
+    
+            // it should be built as a regular index
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable table = phxConn.getTable(new PTableKey(null, tableName));
+            assertEquals("Index not built", 1, table.getIndexes().size());
+            assertEquals("Wrong index created", indexName, table.getIndexes().get(0).getName().getString());
+            
+            ResultSet rs = stmt.executeQuery("select /*+ INDEX(" + indexName + ")*/ pk, val from " + tableName);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            assertEquals("y", rs.getString(2));
+            assertFalse(rs.next());
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
deleted file mode 100644
index 5a53333..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexRegularBuildIT.java
+++ /dev/null
@@ -1,62 +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.end2end.index;
-
-import static org.junit.Assert.assertTrue;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.Statement;
-import java.util.Map;
-
-import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.google.common.collect.Maps;
-
-public class AsyncIndexRegularBuildIT extends BaseOwnClusterHBaseManagedTimeIT {
-
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
-        serverProps.put("phoenix.async.index.automatic.build", Boolean.toString(false));
-        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()));
-    }
-    
-    @Test
-    public void testAsyncIndexRegularBuild() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        Statement stmt = conn.createStatement();
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        AsyncIndexTestUtil.createTableAndLoadData(stmt, tableName);
-        AsyncIndexTestUtil.createAsyncIndex(stmt, indexName, tableName);
-
-        String personTableAsyncIndexInfoQuery = AsyncIndexTestUtil.getPersonTableAsyncIndexInfoQuery(tableName);
-        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-
-        AsyncIndexTestUtil.retryWithSleep(tableName, 4, 5, stmt);
-
-        rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        assertTrue(rs.next());
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
deleted file mode 100644
index d025961..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/AsyncIndexTestUtil.java
+++ /dev/null
@@ -1,58 +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.end2end.index;
-
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-public class AsyncIndexTestUtil {
-    public static void createTableAndLoadData(Statement stmt, String tableName) throws SQLException {
-        String ddl = "CREATE TABLE " + tableName + " (ID INTEGER NOT NULL PRIMARY KEY, " +
-                     "FNAME VARCHAR, LNAME VARCHAR)";
-        
-        stmt.execute(ddl);
-        stmt.execute("UPSERT INTO " + tableName + " values(1, 'FIRST', 'F')");
-        stmt.execute("UPSERT INTO " + tableName + " values(2, 'SECOND', 'S')");
-    }
-
-    public static void createAsyncIndex(Statement stmt, String indexName, String tableName) throws SQLException {
-        stmt.execute("CREATE INDEX " + indexName + " ON " + tableName + "(FNAME) ASYNC");
-    }
-
-    public static void retryWithSleep(String tableName, int maxRetries, int sleepInSecs, Statement stmt) throws Exception {
-        String personTableAsyncIndexInfoQuery = getPersonTableAsyncIndexInfoQuery(tableName);
-        ResultSet rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-        // Wait for max of 5 retries with each retry of 5 sec sleep
-        int retries = 0;
-        while(retries <= maxRetries) {
-            Thread.sleep(sleepInSecs * 1000);
-            rs = stmt.executeQuery(personTableAsyncIndexInfoQuery);
-            if (!rs.next()) {
-                break;
-            }
-            retries++;
-        }
-    }
-    
-    public static String getPersonTableAsyncIndexInfoQuery(String tableName) {
-        return ASYNC_INDEX_INFO_QUERY + " and DATA_TABLE_NAME='" + tableName + "'";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index aba7dac..b7537a6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -51,18 +51,15 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
-import org.apache.phoenix.coprocessor.generated.PTableProtos.PTableType;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.DateUtil;
@@ -70,7 +67,6 @@ 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.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.junit.BeforeClass;
@@ -83,12 +79,15 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-    private static final long MAX_WAIT_FOR_ASYNC_INDEX_BUILD = 30000;
-    	
+	
 	private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
 	private final String tableDDLOptions;
+	private final String tableName;
+    private final String indexName;
+    private final String fullTableName;
+    private final String fullIndexName;
 	
 	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
 		this.localIndex = localIndex;
@@ -103,6 +102,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 			optionBuilder.append(" TRANSACTIONAL=true ");
 		}
 		this.tableDDLOptions = optionBuilder.toString();
+		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
 	}
 	
 	@BeforeClass
@@ -123,10 +126,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
 	@Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
-	    String tableName = "TBL_" + generateRandomString();
-	    String indexName = "IND_" + generateRandomString();
-	    String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-	    String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -183,10 +182,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -242,10 +237,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
         if (transactional) {
             testCreateIndexAfterUpsertStarted(true, fullTableName + "2", fullIndexName + "2");
@@ -342,10 +333,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -396,10 +383,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testGroupByCount() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -418,10 +401,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -445,10 +424,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -471,10 +446,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -514,10 +485,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -573,10 +540,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -652,10 +615,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testSelectCF() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -713,10 +672,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -777,10 +732,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
     	String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -869,10 +820,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -984,10 +931,6 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -1005,20 +948,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
     }
 
     @Test
-    public void testSyncIndexWithDecimalCol() throws Exception {
-        testIndexWithDecimalCol(false);
-    }
-    
-    @Test
-    public void testAsyncIndexWithDecimalCol() throws Exception {
-        testIndexWithDecimalCol(true);
-    }
-    
-    private void testIndexWithDecimalCol(boolean async) throws Exception {
-        String tableName = "TBL_" + generateRandomString();
-        String indexName = "IND_" + generateRandomString();
-        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+    public void testIndexWithDecimalCol() throws Exception {
     	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 	        conn.setAutoCommit(false);
@@ -1029,23 +959,10 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
-            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)" + (async ? " ASYNC" : "");
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
             
-            if (async) {
-                boolean wasBuilt = false;
-                long startTime = System.currentTimeMillis();
-                do {
-                    rs = conn.getMetaData().getTables("", TestUtil.DEFAULT_SCHEMA_NAME, StringUtil.escapeLike(indexName), new String[] {PTableType.INDEX.toString()});
-                    if (rs.next() && PIndexState.ACTIVE.toString().equalsIgnoreCase(rs.getString(PhoenixDatabaseMetaData.INDEX_STATE))) {
-                        wasBuilt = true;
-                        break;
-                    }
-                } while (System.currentTimeMillis() - startTime < MAX_WAIT_FOR_ASYNC_INDEX_BUILD);
-                assertTrue("Asyncronous index failed to build", wasBuilt);
-            }
-            
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index f13217a..5243154 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -17,12 +17,8 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-
 import java.io.IOException;
-import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -72,7 +68,6 @@ import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
@@ -92,10 +87,9 @@ import com.google.common.collect.Lists;
  */
 public class MetaDataRegionObserver extends BaseRegionObserver {
     public static final Log LOG = LogFactory.getLog(MetaDataRegionObserver.class);
-    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(2);
+    protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
     private boolean enableRebuildIndex = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD;
     private long rebuildIndexTimeInterval = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL;
-    private boolean autoAsyncIndexBuild = QueryServicesOptions.DEFAULT_ASYNC_INDEX_AUTO_BUILD; 
     private boolean blockWriteRebuildIndex = false;
 
     @Override
@@ -123,8 +117,6 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD);
         rebuildIndexTimeInterval = env.getConfiguration().getLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB,
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
-        autoAsyncIndexBuild = env.getConfiguration().getBoolean(QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB, 
-                QueryServicesOptions.DEFAULT_ASYNC_INDEX_AUTO_BUILD);
         blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
         	QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
     }
@@ -166,102 +158,23 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         };
         (new Thread(r)).start();
 
+        if (!enableRebuildIndex && !blockWriteRebuildIndex) {
+            LOG.info("Failure Index Rebuild is skipped by configuration.");
+            return;
+        }
         // turn off verbose deprecation logging
         Logger deprecationLogger = Logger.getLogger("org.apache.hadoop.conf.Configuration.deprecation");
         if (deprecationLogger != null) {
             deprecationLogger.setLevel(Level.WARN);
         }
-
         try {
             Class.forName(PhoenixDriver.class.getName());
+            // starts index rebuild schedule work
+            BuildIndexScheduleTask task = new BuildIndexScheduleTask(e.getEnvironment());
+            // run scheduled task every 10 secs
+            executor.scheduleAtFixedRate(task, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
         } catch (ClassNotFoundException ex) {
-            LOG.error("Phoenix Driver class is not found. Fix the classpath.", ex);
-        }
-         
-        // Enable async index rebuilder when autoAsyncIndexBuild is set to true 
-        if (autoAsyncIndexBuild)
-        {
-            LOG.info("Enabling Async Index rebuilder");
-            AsyncIndexRebuilderTask asyncIndexRebuilderTask = new AsyncIndexRebuilderTask(e.getEnvironment());
-            // run async index rebuilder task every 10 secs to rebuild any newly created async indexes
-            executor.scheduleAtFixedRate(asyncIndexRebuilderTask, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
-        }
-
-        if (!enableRebuildIndex && !blockWriteRebuildIndex) {
-            LOG.info("Failure Index Rebuild is skipped by configuration.");
-            return;
-        }
-
-        // starts index rebuild schedule work
-        BuildIndexScheduleTask task = new BuildIndexScheduleTask(e.getEnvironment());
-        // run scheduled task every 10 secs
-        executor.scheduleAtFixedRate(task, 10000, rebuildIndexTimeInterval, TimeUnit.MILLISECONDS);
-    }
-    
-    /**
-     * Task runs periodically to re-build async indexes when hbase is running in non-distributed mode or 
-     * when mapreduce is running in local mode
-     *
-     */
-    public static class AsyncIndexRebuilderTask extends TimerTask {
-        RegionCoprocessorEnvironment env;
-
-        public AsyncIndexRebuilderTask(RegionCoprocessorEnvironment env) {
-            this.env = env;
-        }
-
-        @Override
-        public void run() {
-            PhoenixConnection conn = null;
-            try {
-                conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
-                Statement s = conn.createStatement();
-                ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);
-                PhoenixConnection metaDataClientConn = conn;
-                while (rs.next()) {
-                    String tableName = rs.getString(PhoenixDatabaseMetaData.DATA_TABLE_NAME);
-                    String tableSchema = rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM);
-                    String indexName = rs.getString(PhoenixDatabaseMetaData.TABLE_NAME);
-                    
-                    final PTable indexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, indexName));
-                    final PTable dataTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(tableSchema, tableName));
-                    // this is set to ensure index tables remains consistent post population.
-                    long maxTimeRange = indexTable.getTimeStamp()+1;
-
-                    try {
-                        final Properties props = new Properties();
-                        Long txnScn = null;
-                        if (!indexTable.isTransactional()) {
-                            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(maxTimeRange));
-                            metaDataClientConn = QueryUtil.getConnectionOnServer(props, env.getConfiguration()).unwrap(PhoenixConnection.class);
-                            txnScn = maxTimeRange;
-                        }
-                        MetaDataClient client = new MetaDataClient(conn);
-                        LOG.info("Building Index " + SchemaUtil.getTableName(tableSchema, indexName));
-                        client.buildIndex(indexTable, new TableRef(dataTable), txnScn);
-                    } catch (Throwable t) {
-                        LOG.error("AsyncIndexRebuilderTask failed while building index!", t);
-                    } finally {
-                        if (metaDataClientConn != null) {
-                            try {
-                                metaDataClientConn.close();
-                            } catch (SQLException ignored) {
-                                LOG.debug("AsyncIndexRebuilderTask can't close metaDataClientConn", ignored);
-                            }
-                        }
-                    }
-                }
-            } catch (Throwable t) {
-                LOG.error("AsyncIndexRebuilderTask failed!", t);
-            } finally {
-                if (conn != null) {
-                    try {
-                        conn.close();
-                    } catch (SQLException ignored) {
-                        LOG.debug("AsyncIndexRebuilderTask can't close connection", ignored);
-                    }
-                }
-            }
+            LOG.error("BuildIndexScheduleTask cannot start!", ex);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
index 5882c14..df96c7b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixInputFormat.java
@@ -43,7 +43,6 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 
 import com.google.common.base.Preconditions;
@@ -112,7 +111,6 @@ public class PhoenixInputFormat<T extends DBWritable> extends InputFormat<NullWr
             if(txnScnValue==null && currentScnValue!=null) {
                 overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, currentScnValue);
             }
-            overridingProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
             final Connection connection = ConnectionUtil.getInputConnection(configuration, overridingProps);
             final String selectStatement = PhoenixConfigurationUtil.getSelectStatement(configuration);
             Preconditions.checkNotNull(selectStatement);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
index f955e6b..1058670 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
@@ -24,10 +24,7 @@ import java.util.Properties;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.util.ConnectionUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,7 +50,6 @@ public class IndexToolUtil {
 		final String masterTable = PhoenixConfigurationUtil.getInputTableName(configuration);
 		final String indexTable = PhoenixConfigurationUtil.getOutputTableName(configuration);
 		final Properties overrideProps = new Properties();
-		overrideProps.setProperty(QueryServices.TRANSACTIONS_ENABLED, configuration.get(QueryServices.TRANSACTIONS_ENABLED));
 		final Connection connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
 		try {
 			updateIndexState(connection, masterTable, indexTable , state);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
index 99f84ec..f6f2482 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/automation/PhoenixMRJobSubmitter.java
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.mapreduce.index.automation;
 
-import static org.apache.phoenix.query.QueryConstants.ASYNC_INDEX_INFO_QUERY;
-
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -52,6 +50,7 @@ import org.apache.phoenix.mapreduce.index.IndexTool;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.PhoenixMRJobUtil;
 import org.apache.phoenix.util.PhoenixMRJobUtil.MR_SCHEDULER_TYPE;
 import org.apache.phoenix.util.ZKBasedMasterElectionUtil;
@@ -62,6 +61,7 @@ import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
 import com.google.gson.reflect.TypeToken;
 
+
 public class PhoenixMRJobSubmitter {
 
     // Lock to elect a master node that submits the Phoenix Secondary Index MR Jobs
@@ -69,6 +69,22 @@ public class PhoenixMRJobSubmitter {
             "/phoenix/automated-mr-index-build-leader-election";
     private static final String AUTO_INDEX_BUILD_LOCK_NAME = "ActiveStandbyElectorLock";
 
+    public static final String CANDIDATE_INDEX_INFO_QUERY = "SELECT "
+            + PhoenixDatabaseMetaData.INDEX_TYPE + ","
+            + PhoenixDatabaseMetaData.DATA_TABLE_NAME + ", "
+            + PhoenixDatabaseMetaData.TABLE_SCHEM + ", "
+            + PhoenixDatabaseMetaData.TABLE_NAME + ", "
+            + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE 
+            + " FROM "
+            + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA + ".\"" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "\""
+            + " (" + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") "
+            + " WHERE "
+            + PhoenixDatabaseMetaData.COLUMN_NAME + " IS NULL and "
+            + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NULL  and "
+            + PhoenixDatabaseMetaData.ASYNC_CREATED_DATE + " IS NOT NULL and "
+            + PhoenixDatabaseMetaData.TABLE_TYPE + " = '" + PTableType.INDEX.getSerializedValue() + "' and "
+            + PhoenixDatabaseMetaData.INDEX_STATE + " = '" + PIndexState.BUILDING.getSerializedValue() + "'";
+    
     // TODO - Move this to a property?
     private static final int JOB_SUBMIT_POOL_TIMEOUT = 5;
     private Configuration conf;
@@ -144,7 +160,7 @@ public class PhoenixMRJobSubmitter {
     public Map<String, PhoenixAsyncIndex> getCandidateJobs() throws SQLException {
         Connection con = DriverManager.getConnection("jdbc:phoenix:" + zkQuorum);
         Statement s = con.createStatement();
-        ResultSet rs = s.executeQuery(ASYNC_INDEX_INFO_QUERY);
+        ResultSet rs = s.executeQuery(CANDIDATE_INDEX_INFO_QUERY);
         Map<String, PhoenixAsyncIndex> candidateIndexes = new HashMap<String, PhoenixAsyncIndex>();
         while (rs.next()) {
             PhoenixAsyncIndex indexInfo = new PhoenixAsyncIndex();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 28393aa..9f8f58c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -21,7 +21,6 @@ package org.apache.phoenix.query;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
@@ -116,12 +115,9 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
-import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.util.ByteUtil;
 
 
@@ -171,22 +167,7 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
-    public static final String ASYNC_INDEX_INFO_QUERY = "SELECT "
-            + INDEX_TYPE + ","
-            + DATA_TABLE_NAME + ", "
-            + TABLE_SCHEM + ", "
-            + TABLE_NAME + ", "
-            + ASYNC_CREATED_DATE 
-            + " FROM "
-            + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\""
-            + " (" + ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() + ") "
-            + " WHERE "
-            + COLUMN_NAME + " IS NULL and "
-            + COLUMN_FAMILY + " IS NULL  and "
-            + ASYNC_CREATED_DATE + " IS NOT NULL and "
-            + TABLE_TYPE + " = '" + PTableType.INDEX.getSerializedValue() + "' and "
-            + PhoenixDatabaseMetaData.INDEX_STATE + " = '" + PIndexState.BUILDING.getSerializedValue() + "'";
-    
+
     /**
      * Separator used between variable length keys for a composite key.
      * Variable length data types may not use this byte value.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 04e703d..2ad2d43 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -117,8 +117,6 @@ public interface QueryServices extends SQLCloseable {
     public static final String MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxMetaDataCacheTimeToLiveMs";
     public static final String MAX_SERVER_METADATA_CACHE_SIZE_ATTRIB = "phoenix.coprocessor.maxMetaDataCacheSize";
     public static final String MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB = "phoenix.client.maxMetaDataCacheSize";
-    public static final String HBASE_CLUSTER_DISTRIBUTED_ATTRIB = "hbase.cluster.distributed";
-    public static final String MAPRED_FRAMEWORK_NAME = "mapreduce.framework.name";
 
     public static final String AUTO_UPGRADE_WHITELIST_ATTRIB = "phoenix.client.autoUpgradeWhiteList";
     // Mainly for testing to force spilling
@@ -221,9 +219,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB = "phoenix.client.localIndexUpgrade";
     public static final String LIMITED_QUERY_SERIAL_THRESHOLD = "phoenix.limited.query.serial.threshold";
 
-    // A master switch to enable auto rebuild an async secondary index 
-    public static final String ASYNC_INDEX_AUTO_BUILD_ATTRIB = "phoenix.async.index.automatic.build";
-
+    public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index cc23e08..b7ea82e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.query;
 import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD;
 import static org.apache.phoenix.query.QueryServices.ALLOW_ONLINE_TABLE_SCHEMA_UPDATE;
 import static org.apache.phoenix.query.QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE;
-import static org.apache.phoenix.query.QueryServices.ASYNC_INDEX_AUTO_BUILD_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_NAME;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.COLLECT_REQUEST_LEVEL_METRICS;
@@ -139,7 +138,6 @@ public class QueryServicesOptions {
     public static final long DEFAULT_SCAN_RESULT_CHUNK_SIZE = 2999;
     public static final boolean DEFAULT_IS_NAMESPACE_MAPPING_ENABLED = false;
     public static final boolean DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE = true;
-    public static final boolean DEFAULT_ASYNC_INDEX_AUTO_BUILD = false;
 
     //
     // Spillable GroupBy - SPGBY prefix
@@ -251,6 +249,8 @@ public class QueryServicesOptions {
     public static final int DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE = 10;
     public static final boolean DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE = true;
     public static final float DEFAULT_LIMITED_QUERY_SERIAL_THRESHOLD = 0.2f;
+    
+    public static final boolean DEFAULT_INDEX_ASYNC_BUILD_ENABLED = true;
 
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {
@@ -328,9 +328,7 @@ public class QueryServicesOptions {
             .setIfUnset(RENEW_LEASE_THREAD_POOL_SIZE, DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE)
             .setIfUnset(IS_NAMESPACE_MAPPING_ENABLED, DEFAULT_IS_NAMESPACE_MAPPING_ENABLED)
             .setIfUnset(IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE)
-            .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)
-            .setIfUnset(ASYNC_INDEX_AUTO_BUILD_ATTRIB, DEFAULT_ASYNC_INDEX_AUTO_BUILD);
-
+            .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.
@@ -591,11 +589,6 @@ public class QueryServicesOptions {
         return this;
     }
 
-    public QueryServicesOptions setAsyncIndexAutoBuild(boolean autoAsyncIndexBuild) {
-        config.setBoolean(ASYNC_INDEX_AUTO_BUILD_ATTRIB, autoAsyncIndexBuild);
-        return this;
-    }
-
     public QueryServicesOptions setExplainRowCount(boolean showRowCount) {
         config.setBoolean(EXPLAIN_ROW_COUNT_ATTRIB, showRowCount);
         return this;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 729f5f9..1264e32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -1095,7 +1095,7 @@ public class MetaDataClient {
         boolean success = false;
         SQLException sqlException = null;
         try {
-            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef, null);
+            MutationState state = newClientAtNextTimeStamp.buildIndex(index, tableRef);
             success = true;
             return state;
         } catch (SQLException e) {
@@ -1121,7 +1121,7 @@ public class MetaDataClient {
         throw new IllegalStateException(); // impossible
     }
 
-    public MutationState buildIndex(PTable index, TableRef dataTableRef, Long txnScn) throws SQLException {
+    private MutationState buildIndex(PTable index, TableRef dataTableRef) throws SQLException {
         AlterIndexStatement indexStatement = null;
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -1138,9 +1138,6 @@ public class MetaDataClient {
             Scan scan = mutationPlan.getContext().getScan();
             Long scn = connection.getSCN();
             try {
-                if (txnScn!=null) {
-                    scan.setAttribute(BaseScannerRegionObserver.TX_SCN, Bytes.toBytes(Long.valueOf(txnScn)));
-                }
                 if (ScanUtil.isDefaultTimeRange(scan.getTimeRange())) {
                     if (scn == null) {
                         scn = mutationPlan.getContext().getCurrentTime();
@@ -1449,8 +1446,11 @@ public class MetaDataClient {
         }
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
+        boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
+            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
-        if(statement.isAsync()) {
+        if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
         
@@ -1459,7 +1459,7 @@ public class MetaDataClient {
         if (connection.getSCN() != null) {
             return buildIndexAtTimeStamp(table, statement.getTable());
         }
-        return buildIndex(table, tableRef, null);
+        return buildIndex(table, tableRef);
     }
 
     public MutationState dropSequence(DropSequenceStatement statement) throws SQLException {
@@ -3550,7 +3550,7 @@ public class MetaDataClient {
                     return buildIndexAtTimeStamp(index, dataTableNode);
                 }
                 TableRef dataTableRef = FromCompiler.getResolver(dataTableNode, connection).getTables().get(0);
-                return buildIndex(index, dataTableRef, null);
+                return buildIndex(index, dataTableRef);
             }
             return new MutationState(1, connection);
         } catch (TableNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/243e5e22/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 4ec42bd..6ae655c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -58,7 +58,6 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     public static final String DEFAULT_EXTRA_JDBC_ARGUMENTS = PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
     private static final boolean DEFAULT_RUN_UPDATE_STATS_ASYNC = false;
     private static final boolean DEFAULT_COMMIT_STATS_ASYNC = false;
-    private static final boolean DEFAULT_ASYNC_INDEX_AUTO_BUILD = true;
     public static final int DEFAULT_INDEX_HANDLER_COUNT = 5;
     public static final int DEFAULT_METADATA_HANDLER_COUNT = 5;
     public static final int DEFAULT_HCONNECTION_POOL_CORE_SIZE = 10;
@@ -112,8 +111,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setHConnectionPoolCoreSize(DEFAULT_HCONNECTION_POOL_CORE_SIZE)
                 .setHConnectionPoolMaxSize(DEFAULT_HCONNECTION_POOL_MAX_SIZE)
                 .setMaxThreadsPerHTable(DEFAULT_HTABLE_MAX_THREADS)
-                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME)
-                .setAsyncIndexAutoBuild(DEFAULT_ASYNC_INDEX_AUTO_BUILD);
+                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME);
     }
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps, ReadOnlyProps overrideProps) {


[13/50] [abbrv] phoenix git commit: PHOENIX-3152 Incorrect comparator in QueryOptimizer may cause IllegalArgumentException

Posted by ma...@apache.org.
PHOENIX-3152 Incorrect comparator in QueryOptimizer may cause IllegalArgumentException


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

Branch: refs/heads/calcite
Commit: bb0d89bd9f591e056f7ae9cd5892b9f99bd513a6
Parents: 67b8b73
Author: Sergey Soldatov <ss...@apache.org>
Authored: Sat Aug 6 16:31:13 2016 -0700
Committer: Sergey Soldatov <ss...@apache.org>
Committed: Sat Aug 6 19:44:00 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/optimize/QueryOptimizer.java | 21 +++++++++++---------
 1 file changed, 12 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bb0d89bd/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
index e4198ee..bd9c811 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
@@ -383,35 +383,38 @@ public class QueryOptimizer {
                 // For shared indexes (i.e. indexes on views and local indexes),
                 // a) add back any view constants as these won't be in the index, and
                 // b) ignore the viewIndexId which will be part of the row key columns.
-                int c = (plan2.getContext().getScanRanges().getBoundPkColumnCount() + (table2.getViewIndexId() == null ? 0 : (boundRanges - 1))) - 
+                int c = (plan2.getContext().getScanRanges().getBoundPkColumnCount() + (table2.getViewIndexId() == null ? 0 : (boundRanges - 1))) -
                         (plan1.getContext().getScanRanges().getBoundPkColumnCount() + (table1.getViewIndexId() == null ? 0 : (boundRanges - 1)));
                 if (c != 0) return c;
-                if (plan1.getGroupBy()!=null && plan2.getGroupBy()!=null) {
+                if (plan1.getGroupBy() != null && plan2.getGroupBy() != null) {
                     if (plan1.getGroupBy().isOrderPreserving() != plan2.getGroupBy().isOrderPreserving()) {
                         return plan1.getGroupBy().isOrderPreserving() ? -1 : 1;
-                    } 
+                    }
                 }
                 // Use smaller table (table with fewest kv columns)
                 c = (table1.getColumns().size() - table1.getPKColumns().size()) - (table2.getColumns().size() - table2.getPKColumns().size());
                 if (c != 0) return c;
-                
+
                 // If all things are equal, don't choose local index as it forces scan
                 // on every region (unless there's no start/stop key)
-                if (table1.getIndexType() == IndexType.LOCAL) {
+
+                if (table1.getIndexType() == IndexType.LOCAL && table2.getIndexType() !=
+                        IndexType.LOCAL) {
                     return plan1.getContext().getScanRanges().getRanges().isEmpty() ? -1 : 1;
                 }
-                if (table2.getIndexType() == IndexType.LOCAL) {
+                if (table2.getIndexType() == IndexType.LOCAL && table1.getIndexType() !=
+                        IndexType.LOCAL) {
                     return plan2.getContext().getScanRanges().getRanges().isEmpty() ? 1 : -1;
                 }
 
                 // All things being equal, just use the table based on the Hint.USE_DATA_OVER_INDEX_TABLE
-                if (table1.getType() == PTableType.INDEX) {
+
+                if (table1.getType() == PTableType.INDEX && table2.getType() != PTableType.INDEX) {
                     return comparisonOfDataVersusIndexTable;
                 }
-                if (table2.getType() == PTableType.INDEX) {
+                if (table2.getType() == PTableType.INDEX && table1.getType() != PTableType.INDEX) {
                     return -comparisonOfDataVersusIndexTable;
                 }
-                
                 return 0;
             }
             


[04/50] [abbrv] phoenix git commit: PHOENIX-3128 Remove extraneous operations during upsert with local immutable index

Posted by ma...@apache.org.
PHOENIX-3128 Remove extraneous operations during upsert with local immutable index


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

Branch: refs/heads/calcite
Commit: ca7c36d59247795d92f7fc23a4ae4038da215284
Parents: 7febdfa
Author: James Taylor <ja...@apache.org>
Authored: Tue Aug 2 12:03:33 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Tue Aug 2 16:26:40 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/DistinctPrefixFilterIT.java | 11 ++-
 .../apache/phoenix/end2end/index/IndexIT.java   | 39 +++++++++
 .../compile/PostLocalIndexDDLCompiler.java      | 15 ++--
 .../apache/phoenix/execute/MutationState.java   | 11 ++-
 .../hbase/index/covered/IndexMetaData.java      |  9 +-
 .../hbase/index/covered/NonTxIndexBuilder.java  |  6 +-
 .../phoenix/index/PhoenixIndexMetaData.java     | 12 ++-
 .../index/PhoenixTransactionalIndexer.java      | 89 +++++++++++++-------
 .../org/apache/phoenix/schema/PTableImpl.java   | 13 ++-
 .../apache/phoenix/util/TransactionUtil.java    |  7 ++
 10 files changed, 164 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
index 203d51e..9d31070 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
@@ -88,6 +88,15 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
         insertPrefixV("3", "1");
         insertPrefixV("3", "2");
         insertPrefixV("3", "3");
+        conn.commit();
+        ResultSet rs;
+        rs = conn.createStatement().executeQuery("select /*+ NO_INDEX */ count(*) from " + testTableV);
+        assertTrue(rs.next());
+        long count1 = rs.getLong(1);
+        rs = conn.createStatement().executeQuery("select count(*) from " + testTableV + "_idx");
+        assertTrue(rs.next());
+        long count2 = rs.getLong(1);
+        assertEquals(count1,count2);
 
         multiply();
         multiply();
@@ -258,7 +267,7 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
 
         testCommonDistinct(testTableF);
         testCommonDistinct(testTableV);
-}
+    }
 
     private void testCommonDistinct(String testTable) throws Exception {
         testSkipRange("SELECT %s DISTINCT prefix1 FROM " + testTable, 4);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index 35a0aad..df45ecb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -26,6 +26,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.Date;
@@ -39,7 +40,15 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.coprocessor.generated.PTableProtos.PTableType;
@@ -56,6 +65,7 @@ import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
@@ -63,6 +73,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.apache.phoenix.util.TransactionUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -871,6 +882,7 @@ public class IndexIT extends BaseHBaseManagedTimeTableReuseIT {
             conn.createStatement().execute("CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, \"V1\" VARCHAR, \"v2\" VARCHAR)"+tableDDLOptions);
             query = "SELECT * FROM "+fullTableName;
             rs = conn.createStatement().executeQuery(query);
+            long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
   	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
@@ -941,9 +953,36 @@ public class IndexIT extends BaseHBaseManagedTimeTableReuseIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
+            
+            assertNoIndexDeletes(conn, ts, fullIndexName);
         } 
     }
 
+    private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
+        if (!this.mutable) {
+            PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+            PTable index = pconn.getTable(new PTableKey(null, fullIndexName));
+            byte[] physicalIndexTable = index.getPhysicalName().getBytes();
+            try (HTableInterface hIndex = pconn.getQueryServices().getTable(physicalIndexTable)) {
+                Scan scan = new Scan();
+                scan.setRaw(true);
+                if (this.transactional) {
+                    minTimestamp = TransactionUtil.convertToNanoseconds(minTimestamp);
+                }
+                scan.setTimeRange(minTimestamp, HConstants.LATEST_TIMESTAMP);
+                ResultScanner scanner = hIndex.getScanner(scan);
+                Result result;
+                while ((result = scanner.next()) != null) {
+                    CellScanner cellScanner = result.cellScanner();
+                    while (cellScanner.advance()) {
+                        Cell current = cellScanner.current();
+                        assertEquals (KeyValue.Type.Put.getCode(), current.getTypeByte());
+                    }
+                }
+            };
+        }
+    }
+
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
         String tableName = "TBL_" + generateRandomString();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
index 079ff5c..81dbe0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.compile;
 
-import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
 
@@ -34,9 +33,9 @@ import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.collect.Lists;
 
@@ -93,10 +92,14 @@ public class PostLocalIndexDDLCompiler {
                 @Override
                 public MutationState execute() throws SQLException {
                     connection.getMutationState().commitDDLFence(dataTable);
-                    Cell kv = plan.iterator().next().getValue(0);
-                    ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
-                    // A single Cell will be returned with the count(*) - we decode that here
-                    long rowCount = PLong.INSTANCE.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
+                    Tuple tuple = plan.iterator().next();
+                    long rowCount = 0;
+                    if (tuple != null) {
+                        Cell kv = tuple.getValue(0);
+                        ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+                        // A single Cell will be returned with the count(*) - we decode that here
+                        rowCount = PLong.INSTANCE.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
+                    }
                     // The contract is to return a MutationState that contains the number of rows modified. In this
                     // case, it's the number of rows in the data table which corresponds to the number of index
                     // rows that were added.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 0a9b0ae..fb514b0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -552,12 +552,15 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-    private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values, final long timestamp, boolean includeMutableIndexes, final boolean sendAll) { 
+    private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
+            final long timestamp, boolean includeAllIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
-                (table.isImmutableRows() || includeMutableIndexes) ? 
-                        IndexMaintainer.nonDisabledIndexIterator(table.getIndexes().iterator()) : 
-                        Iterators.<PTable>emptyIterator();
+                 includeAllIndexes || table.isWALDisabled() ? // TODO: remove check for isWALDisabled once PHOENIX-3137 is fixed.
+                     IndexMaintainer.nonDisabledIndexIterator(table.getIndexes().iterator()) :
+                         table.isImmutableRows() ?
+                            IndexMaintainer.enabledGlobalIndexIterator(table.getIndexes().iterator()) :
+                                Iterators.<PTable>emptyIterator();
         final List<Mutation> mutationList = Lists.newArrayListWithExpectedSize(values.size());
         final List<Mutation> mutationsPertainingToIndex = indexes.hasNext() ? Lists.<Mutation>newArrayListWithExpectedSize(values.size()) : null;
         generateMutations(tableRef, timestamp, values, mutationList, mutationsPertainingToIndex);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexMetaData.java
index ee25a40..5420013 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/IndexMetaData.java
@@ -18,5 +18,12 @@
 package org.apache.phoenix.hbase.index.covered;
 
 public interface IndexMetaData {
-    public static final IndexMetaData NULL_INDEX_META_DATA = new IndexMetaData() {};
+    public static final IndexMetaData NULL_INDEX_META_DATA = new IndexMetaData() {
+
+        @Override
+        public boolean isImmutableRows() {
+            return false;
+        }};
+    
+    public boolean isImmutableRows();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
index 27af40f..10d164b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/NonTxIndexBuilder.java
@@ -98,7 +98,7 @@ public class NonTxIndexBuilder extends BaseIndexBuilder {
         Collection<Batch> batches = createTimestampBatchesFromMutation(m);
 
         // go through each batch of keyvalues and build separate index entries for each
-        boolean cleanupCurrentState = true;
+        boolean cleanupCurrentState = !indexMetaData.isImmutableRows();
         for (Batch batch : batches) {
             /*
              * We have to split the work between the cleanup and the update for each group because when we update the
@@ -215,7 +215,9 @@ public class NonTxIndexBuilder extends BaseIndexBuilder {
         // determine if we need to make any cleanup given the pending update.
         long batchTs = batch.getTimestamp();
         state.setPendingUpdates(batch.getKvs());
-        addCleanupForCurrentBatch(updateMap, batchTs, state, indexMetaData);
+        if (!indexMetaData.isImmutableRows()) {
+            addCleanupForCurrentBatch(updateMap, batchTs, state, indexMetaData);
+        }
 
         // A.2 do a single pass first for the updates to the current state
         state.applyPendingUpdates();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
index 7a67b9c..2679f1c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
@@ -36,13 +36,13 @@ import org.apache.phoenix.hbase.index.covered.IndexMetaData;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ServerUtil;
-
 import org.apache.tephra.Transaction;
 
 public class PhoenixIndexMetaData implements IndexMetaData {
     private final Map<String, byte[]> attributes;
     private final IndexMetaDataCache indexMetaDataCache;
     private final boolean ignoreNewerMutations;
+    private final boolean isImmutable;
     
     private static IndexMetaDataCache getIndexMetaData(RegionCoprocessorEnvironment env, Map<String, byte[]> attributes) throws IOException {
         if (attributes == null) { return IndexMetaDataCache.EMPTY_INDEX_META_DATA_CACHE; }
@@ -87,6 +87,11 @@ public class PhoenixIndexMetaData implements IndexMetaData {
 
     public PhoenixIndexMetaData(RegionCoprocessorEnvironment env, Map<String,byte[]> attributes) throws IOException {
         this.indexMetaDataCache = getIndexMetaData(env, attributes);
+        boolean isImmutable = true;
+        for (IndexMaintainer maintainer : indexMetaDataCache.getIndexMaintainers()) {
+            isImmutable &= maintainer.isImmutableRows();
+        }
+        this.isImmutable = isImmutable;
         this.attributes = attributes;
         this.ignoreNewerMutations = attributes.get(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS) != null;
     }
@@ -106,4 +111,9 @@ public class PhoenixIndexMetaData implements IndexMetaData {
     public boolean ignoreNewerMutations() {
         return ignoreNewerMutations;
     }
+
+    @Override
+    public boolean isImmutableRows() {
+        return isImmutable;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
index 3d8124c..fdf5498 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Result;
@@ -63,7 +62,6 @@ import org.apache.phoenix.hbase.index.covered.TableState;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.covered.update.ColumnTracker;
 import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
-import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.write.IndexWriter;
 import org.apache.phoenix.query.KeyRange;
@@ -71,10 +69,10 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.trace.TracingUtils;
 import org.apache.phoenix.trace.util.NullSpan;
-import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.TransactionUtil;
 import org.apache.tephra.Transaction;
 import org.apache.tephra.Transaction.VisibilityLevel;
 import org.apache.tephra.TxConstants;
@@ -177,48 +175,68 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
         }
     }
 
+    public static void addMutation(Map<ImmutableBytesPtr, MultiMutation> mutations, ImmutableBytesPtr row, Mutation m) {
+        MultiMutation stored = mutations.get(row);
+        // we haven't seen this row before, so add it
+        if (stored == null) {
+            stored = new MultiMutation(row);
+            mutations.put(row, stored);
+        }
+        stored.addAll(m);
+    }
+    
     private Collection<Pair<Mutation, byte[]>> getIndexUpdates(RegionCoprocessorEnvironment env, PhoenixIndexMetaData indexMetaData, Iterator<Mutation> mutationIterator, byte[] txRollbackAttribute) throws IOException {
+        Transaction tx = indexMetaData.getTransaction();
+        if (tx == null) {
+            throw new NullPointerException("Expected to find transaction in metadata for " + env.getRegionInfo().getTable().getNameAsString());
+        }
+        boolean isRollback = txRollbackAttribute!=null;
+        boolean isImmutable = indexMetaData.isImmutableRows();
         ResultScanner currentScanner = null;
         TransactionAwareHTable txTable = null;
         // Collect up all mutations in batch
         Map<ImmutableBytesPtr, MultiMutation> mutations =
                 new HashMap<ImmutableBytesPtr, MultiMutation>();
+        Map<ImmutableBytesPtr, MultiMutation> findPriorValueMutations;
+        if (isImmutable && !isRollback) {
+            findPriorValueMutations = new HashMap<ImmutableBytesPtr, MultiMutation>();
+        } else {
+            findPriorValueMutations = mutations;
+        }
         while(mutationIterator.hasNext()) {
             Mutation m = mutationIterator.next();
             // add the mutation to the batch set
             ImmutableBytesPtr row = new ImmutableBytesPtr(m.getRow());
-            MultiMutation stored = mutations.get(row);
-            // we haven't seen this row before, so add it
-            if (stored == null) {
-                stored = new MultiMutation(row);
-                mutations.put(row, stored);
+            if (mutations != findPriorValueMutations && isDeleteMutation(m)) {
+                addMutation(findPriorValueMutations, row, m);
             }
-            stored.addAll(m);
+            addMutation(mutations, row, m);
         }
         
         // Collect the set of mutable ColumnReferences so that we can first
         // run a scan to get the current state. We'll need this to delete
         // the existing index rows.
-        Transaction tx = indexMetaData.getTransaction();
-        if (tx == null) {
-            throw new NullPointerException("Expected to find transaction in metadata for " + env.getRegionInfo().getTable().getNameAsString());
-        }
         List<IndexMaintainer> indexMaintainers = indexMetaData.getIndexMaintainers();
-        Set<ColumnReference> mutableColumns = Sets.newHashSetWithExpectedSize(indexMaintainers.size() * 10);
+        int estimatedSize = indexMaintainers.size() * 10;
+        Set<ColumnReference> mutableColumns = Sets.newHashSetWithExpectedSize(estimatedSize);
         for (IndexMaintainer indexMaintainer : indexMaintainers) {
             // For transactional tables, we use an index maintainer
             // to aid in rollback if there's a KeyValue column in the index. The alternative would be
             // to hold on to all uncommitted index row keys (even ones already sent to HBase) on the
             // client side.
-            mutableColumns.addAll(indexMaintainer.getAllColumns());
+            Set<ColumnReference> allColumns = indexMaintainer.getAllColumns();
+            mutableColumns.addAll(allColumns);
         }
 
-        boolean isRollback = txRollbackAttribute!=null;
         Collection<Pair<Mutation, byte[]>> indexUpdates = new ArrayList<Pair<Mutation, byte[]>>(mutations.size() * 2 * indexMaintainers.size());
         try {
-            if (!mutableColumns.isEmpty()) {
+            // Track if we have row keys with Delete mutations (or Puts that are
+            // Tephra's Delete marker). If there are none, we don't need to do the scan for
+            // prior versions, if there are, we do. Since rollbacks always have delete mutations,
+            // this logic will work there too.
+            if (!findPriorValueMutations.isEmpty()) {
                 List<KeyRange> keys = Lists.newArrayListWithExpectedSize(mutations.size());
-                for (ImmutableBytesPtr ptr : mutations.keySet()) {
+                for (ImmutableBytesPtr ptr : findPriorValueMutations.keySet()) {
                     keys.add(PVarbinary.INSTANCE.getKeyRange(ptr.copyBytesIfNecessary()));
                 }
                 Scan scan = new Scan();
@@ -246,9 +264,9 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
                 currentScanner = txTable.getScanner(scan);
             }
             if (isRollback) {
-                processRollback(env, indexMetaData, txRollbackAttribute, currentScanner, mutations, tx, mutableColumns, indexUpdates);
+                processRollback(env, indexMetaData, txRollbackAttribute, currentScanner, tx, mutableColumns, indexUpdates, mutations);
             } else {
-                processMutation(env, indexMetaData, txRollbackAttribute, currentScanner, mutations, tx, mutableColumns, indexUpdates);
+                processMutation(env, indexMetaData, txRollbackAttribute, currentScanner, tx, mutableColumns, indexUpdates, mutations, findPriorValueMutations);
             }
         } finally {
             if (txTable != null) txTable.close();
@@ -257,26 +275,39 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
         return indexUpdates;
     }
 
+    private static boolean isDeleteMutation(Mutation m) {
+        for (Map.Entry<byte[],List<Cell>> cellMap : m.getFamilyCellMap().entrySet()) {
+            for (Cell cell : cellMap.getValue()) {
+                if (cell.getTypeByte() != KeyValue.Type.Put.getCode() || TransactionUtil.isDelete(cell)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
     private void processMutation(RegionCoprocessorEnvironment env,
             PhoenixIndexMetaData indexMetaData, byte[] txRollbackAttribute,
             ResultScanner scanner,
-            Map<ImmutableBytesPtr, MultiMutation> mutations, Transaction tx,
-            Set<ColumnReference> mutableColumns,
-            Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException {
+            Transaction tx, 
+            Set<ColumnReference> upsertColumns, 
+            Collection<Pair<Mutation, byte[]>> indexUpdates,
+            Map<ImmutableBytesPtr, MultiMutation> mutations,
+            Map<ImmutableBytesPtr, MultiMutation> mutationsToFindPreviousValue) throws IOException {
         if (scanner != null) {
             Result result;
             ColumnReference emptyColRef = new ColumnReference(indexMetaData.getIndexMaintainers().get(0).getDataEmptyKeyValueCF(), QueryConstants.EMPTY_COLUMN_BYTES);
             // Process existing data table rows by removing the old index row and adding the new index row
             while ((result = scanner.next()) != null) {
-                Mutation m = mutations.remove(new ImmutableBytesPtr(result.getRow()));
-                TxTableState state = new TxTableState(env, mutableColumns, indexMetaData.getAttributes(), tx.getWritePointer(), m, emptyColRef, result);
+                Mutation m = mutationsToFindPreviousValue.remove(new ImmutableBytesPtr(result.getRow()));
+                TxTableState state = new TxTableState(env, upsertColumns, indexMetaData.getAttributes(), tx.getWritePointer(), m, emptyColRef, result);
                 generateDeletes(indexMetaData, indexUpdates, txRollbackAttribute, state);
                 generatePuts(indexMetaData, indexUpdates, state);
             }
         }
         // Process new data table by adding new index rows
         for (Mutation m : mutations.values()) {
-            TxTableState state = new TxTableState(env, mutableColumns, indexMetaData.getAttributes(), tx.getWritePointer(), m);
+            TxTableState state = new TxTableState(env, upsertColumns, indexMetaData.getAttributes(), tx.getWritePointer(), m);
             generatePuts(indexMetaData, indexUpdates, state);
         }
     }
@@ -284,9 +315,9 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
     private void processRollback(RegionCoprocessorEnvironment env,
             PhoenixIndexMetaData indexMetaData, byte[] txRollbackAttribute,
             ResultScanner scanner,
-            Map<ImmutableBytesPtr, MultiMutation> mutations, Transaction tx,
-            Set<ColumnReference> mutableColumns,
-            Collection<Pair<Mutation, byte[]>> indexUpdates) throws IOException {
+            Transaction tx, Set<ColumnReference> mutableColumns,
+            Collection<Pair<Mutation, byte[]>> indexUpdates,
+            Map<ImmutableBytesPtr, MultiMutation> mutations) throws IOException {
         if (scanner != null) {
             Result result;
             // Loop through last committed row state plus all new rows associated with current transaction

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index ec09992..847979a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -834,12 +834,17 @@ public class PTableImpl implements PTable {
             // we're using the Tephra column family delete marker here to prevent the translation 
             // of deletes to puts by the Tephra's TransactionProcessor
             if (PTableImpl.this.isTransactional()) {
-                Put delete = new Put(key);
-                for (PColumnFamily colFamily : families) {
-                    delete.add(colFamily.getName().getBytes(), TxConstants.FAMILY_DELETE_QUALIFIER, ts,
+                Put put = new Put(key);
+                if (families.isEmpty()) {
+                    put.add(SchemaUtil.getEmptyColumnFamily(PTableImpl.this), TxConstants.FAMILY_DELETE_QUALIFIER, ts,
                             HConstants.EMPTY_BYTE_ARRAY);
+                } else {
+                    for (PColumnFamily colFamily : families) {
+                        put.add(colFamily.getName().getBytes(), TxConstants.FAMILY_DELETE_QUALIFIER, ts,
+                                HConstants.EMPTY_BYTE_ARRAY);
+                    }
                 }
-                deleteRow = delete;                
+                deleteRow = put;                
             } else {
                 Delete delete = new Delete(key);
                 for (PColumnFamily colFamily : families) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca7c36d5/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index 1dcf9d3..0e044b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -19,6 +19,9 @@ package org.apache.phoenix.util;
 
 import java.sql.SQLException;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -35,6 +38,10 @@ public class TransactionUtil {
     private TransactionUtil() {
     }
     
+    public static boolean isDelete(Cell cell) {
+        return (CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY));
+    }
+    
     public static long convertToNanoseconds(long serverTimeStamp) {
         return serverTimeStamp * TxConstants.MAX_TX_PER_MS;
     }


[33/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 6722b67..b47738d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -51,6 +51,7 @@ import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.UpgradeUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class UpgradeIT extends BaseHBaseManagedTimeIT {
@@ -59,64 +60,74 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testUpgradeForTenantViewWithSameColumnsAsBaseTable() throws Exception {
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW1", null, "VIEW1", ColumnDiff.EQUAL);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW", null, "VIEW2",
+        String tableWithViewName = generateRandomString();
+        String viewTableName = generateRandomString();
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.EQUAL);
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
             ColumnDiff.EQUAL);
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW3", "VIEWSCHEMA", "VIEW3",
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "3", viewTableName + "SCHEMA", viewTableName + "3",
             ColumnDiff.EQUAL);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW4", "VIEWSCHEMA", "VIEW4",
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "4", viewTableName + "SCHEMA", viewTableName + "4",
             ColumnDiff.EQUAL);
-        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", "TABLEWITHVIEW5", "SAMESCHEMA", "VIEW5",
+        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
             ColumnDiff.EQUAL);
     }
 
     @Test
     public void testUpgradeForTenantViewWithMoreColumnsThanBaseTable() throws Exception {
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW1", null, "VIEW1", ColumnDiff.MORE);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW", null, "VIEW2",
+        String tableWithViewName = generateRandomString();
+        String viewTableName = generateRandomString();
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
             ColumnDiff.MORE);
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW3", "VIEWSCHEMA", "VIEW3",
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
             ColumnDiff.MORE);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW4", "VIEWSCHEMA", "VIEW4",
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
             ColumnDiff.MORE);
-        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", "TABLEWITHVIEW5", "SAMESCHEMA", "VIEW5",
+        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
             ColumnDiff.MORE);
     }
 
     @Test
     public void testUpgradeForViewWithSameColumnsAsBaseTable() throws Exception {
-        testViewUpgrade(false, null, null, "TABLEWITHVIEW1", null, "VIEW1", ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "TABLESCHEMA", "TABLEWITHVIEW", null, "VIEW2",
+        String tableWithViewName = generateRandomString();
+        String viewTableName = generateRandomString();
+        testViewUpgrade(false, null, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.EQUAL);
+        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
             ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, null, "TABLEWITHVIEW3", "VIEWSCHEMA", "VIEW3",
+        testViewUpgrade(false, null, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
             ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "TABLESCHEMA", "TABLEWITHVIEW4", "VIEWSCHEMA", "VIEW4",
+        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
             ColumnDiff.EQUAL);
-        testViewUpgrade(false, null, "SAMESCHEMA", "TABLEWITHVIEW5", "SAMESCHEMA", "VIEW5",
+        testViewUpgrade(false, null, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
             ColumnDiff.EQUAL);
     }
 
     @Test
     public void testUpgradeForViewWithMoreColumnsThanBaseTable() throws Exception {
-        testViewUpgrade(false, null, null, "TABLEWITHVIEW1", null, "VIEW1", ColumnDiff.MORE);
-        testViewUpgrade(false, null, "TABLESCHEMA", "TABLEWITHVIEW", null, "VIEW2", ColumnDiff.MORE);
-        testViewUpgrade(false, null, null, "TABLEWITHVIEW3", "VIEWSCHEMA", "VIEW3", ColumnDiff.MORE);
-        testViewUpgrade(false, null, "TABLESCHEMA", "TABLEWITHVIEW4", "VIEWSCHEMA", "VIEW4",
+        String tableWithViewName = generateRandomString();
+        String viewTableName = generateRandomString();
+        testViewUpgrade(false, null, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
+        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2", ColumnDiff.MORE);
+        testViewUpgrade(false, null, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3", ColumnDiff.MORE);
+        testViewUpgrade(false, null, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
             ColumnDiff.MORE);
-        testViewUpgrade(false, null, "SAMESCHEMA", "TABLEWITHVIEW5", "SAMESCHEMA", "VIEW5",
+        testViewUpgrade(false, null, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
             ColumnDiff.MORE);
     }
 
     @Test
     public void testSettingBaseColumnCountWhenBaseTableColumnDropped() throws Exception {
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW1", null, "VIEW1", ColumnDiff.MORE);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW", null, "VIEW2",
+        String tableWithViewName = generateRandomString();
+        String viewTableName = generateRandomString();
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "1", null, viewTableName + "1", ColumnDiff.MORE);
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "", null, viewTableName + "2",
             ColumnDiff.LESS);
-        testViewUpgrade(true, TENANT_ID, null, "TABLEWITHVIEW3", "VIEWSCHEMA", "VIEW3",
+        testViewUpgrade(true, TENANT_ID, null, tableWithViewName + "3", "VIEWSCHEMA", viewTableName + "3",
             ColumnDiff.LESS);
-        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", "TABLEWITHVIEW4", "VIEWSCHEMA", "VIEW4",
+        testViewUpgrade(true, TENANT_ID, "TABLESCHEMA", tableWithViewName + "4", "VIEWSCHEMA", viewTableName + "4",
             ColumnDiff.LESS);
-        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", "TABLEWITHVIEW5", "SAMESCHEMA", "VIEW5",
+        testViewUpgrade(true, TENANT_ID, "SAMESCHEMA", tableWithViewName + "5", "SAMESCHEMA", viewTableName + "5",
             ColumnDiff.LESS);
     }
 
@@ -127,12 +138,16 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
 
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String schemaName = "TEST";
-            String phoenixFullTableName = schemaName + ".S_NEW";
-            String indexName = "IDX";
-            String localIndexName = "LIDX";
+            String phoenixFullTableName = schemaName + "." + generateRandomString();
+            String indexName = "IDX_" + generateRandomString();
+            String localIndexName = "LIDX_" + generateRandomString();
+
+            String viewName = "VIEW_" + generateRandomString();
+            String viewIndexName = "VIDX_" + generateRandomString();
+
             String[] tableNames = new String[] { phoenixFullTableName, schemaName + "." + indexName,
-                    schemaName + "." + localIndexName, "diff.v", "test.v","v"};
-            String[] viewIndexes = new String[] { "diff.v_idx", "test.v_idx" };
+                    schemaName + "." + localIndexName, "diff." + viewName, "test." + viewName, viewName};
+            String[] viewIndexes = new String[] { "diff." + viewIndexName, "test." + viewIndexName };
             conn.createStatement().execute("CREATE TABLE " + phoenixFullTableName
                     + "(k VARCHAR PRIMARY KEY, v INTEGER, f INTEGER, g INTEGER NULL, h INTEGER NULL)");
             PreparedStatement upsertStmt = conn
@@ -150,13 +165,13 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             // creating global index
             conn.createStatement().execute("create index " + indexName + " on " + phoenixFullTableName + "(k)");
             // creating view in schema 'diff'
-            conn.createStatement().execute("CREATE VIEW diff.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW diff." + viewName + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // creating view in schema 'test'
-            conn.createStatement().execute("CREATE VIEW test.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
-            conn.createStatement().execute("CREATE VIEW v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW test." + viewName + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW " + viewName + "(col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // Creating index on views
-            conn.createStatement().execute("create index v_idx on diff.v(col)");
-            conn.createStatement().execute("create index v_idx on test.v(col)");
+            conn.createStatement().execute("create index " + viewIndexName + "  on diff." + viewName + "(col)");
+            conn.createStatement().execute("create index " + viewIndexName + " on test." + viewName + "(col)");
 
             // validate data
             for (String tableName : tableNames) {
@@ -243,13 +258,18 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             IllegalArgumentException, IOException, InterruptedException {
         String[] strings = new String[] { "a", "b", "c", "d" };
         String schemaName = "TEST";
-        String phoenixFullTableName = schemaName + ".S_NEW1";
+        String phoenixFullTableName = schemaName + "." + generateRandomString();
         String hbaseTableName = SchemaUtil.getPhysicalTableName(Bytes.toBytes(phoenixFullTableName), true)
                 .getNameAsString();
-        String indexName = "IDX";
-        String[] tableNames = new String[] { phoenixFullTableName, "diff.v1", "test.v1", "v1" };
-        String[] viewIndexes = new String[] { "test.v_idx", "diff.v_idx" };
-        String[] tenantViewIndexes = new String[] { "test.v1_idx", "diff.v1_idx" };
+        String indexName = "IDX_" + generateRandomString();
+        String viewName = "V_" + generateRandomString();
+        String viewName1 = "V1_" + generateRandomString();
+        String viewIndexName = "V_IDX_" + generateRandomString();
+        String tenantViewIndexName = "V1_IDX_" + generateRandomString();
+
+        String[] tableNames = new String[] { phoenixFullTableName, "diff." + viewName1, "test." + viewName1, viewName1 };
+        String[] viewIndexes = new String[] { "test." + viewIndexName, "diff." + viewIndexName };
+        String[] tenantViewIndexes = new String[] { "test." + tenantViewIndexName, "diff." + tenantViewIndexName };
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE TABLE " + phoenixFullTableName
                     + "(k VARCHAR not null, v INTEGER not null, f INTEGER, g INTEGER NULL, h INTEGER NULL CONSTRAINT pk PRIMARY KEY(k,v)) MULTI_TENANT=true");
@@ -266,13 +286,13 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             // creating global index
             conn.createStatement().execute("create index " + indexName + " on " + phoenixFullTableName + "(f)");
             // creating view in schema 'diff'
-            conn.createStatement().execute("CREATE VIEW diff.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW diff." + viewName + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // creating view in schema 'test'
-            conn.createStatement().execute("CREATE VIEW test.v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
-            conn.createStatement().execute("CREATE VIEW v (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW test." + viewName + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW " + viewName + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // Creating index on views
-            conn.createStatement().execute("create local index v_idx on diff.v(col)");
-            conn.createStatement().execute("create local index v_idx on test.v(col)");
+            conn.createStatement().execute("create local index " + viewIndexName + " on diff." + viewName + "(col)");
+            conn.createStatement().execute("create local index " + viewIndexName + " on test." + viewName + "(col)");
         }
         Properties props = new Properties();
         String tenantId = "a";
@@ -289,14 +309,14 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
             conn.commit();
             // creating view in schema 'diff'
             conn.createStatement()
-                    .execute("CREATE VIEW diff.v1 (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+                    .execute("CREATE VIEW diff." + viewName1 + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // creating view in schema 'test'
             conn.createStatement()
-                    .execute("CREATE VIEW test.v1 (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
-            conn.createStatement().execute("CREATE VIEW v1 (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+                    .execute("CREATE VIEW test." + viewName1 + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            conn.createStatement().execute("CREATE VIEW " + viewName1 + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
             // Creating index on views
-            conn.createStatement().execute("create index v1_idx on diff.v1(col)");
-            conn.createStatement().execute("create index v1_idx on test.v1(col)");
+            conn.createStatement().execute("create index " + tenantViewIndexName + " on diff." + viewName1 + "(col)");
+            conn.createStatement().execute("create index " + tenantViewIndexName + " on test." + viewName1 + "(col)");
         }
 
         props = new Properties();
@@ -352,7 +372,7 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
         assertTrue(rs.getString(1).contains(hbaseTableName));
     }
     
-    
+
     @Test
     public void testSettingBaseColumnCountForMultipleViewsOnTable() throws Exception {
         String baseSchema = "XYZ";
@@ -361,13 +381,13 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String baseTableDDL = "CREATE TABLE " + fullBaseTableName + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 INTEGER, V2 INTEGER CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true";
             conn.createStatement().execute(baseTableDDL);
-            
+
             for (int i = 1; i <=2; i++) {
                 // Create views for tenants;
                 String tenant = "tenant" + i;
                 try (Connection tenantConn = createTenantConnection(tenant)) {
                     String view = "TENANT_VIEW1";
-                    
+
                     // view with its own column
                     String viewDDL = "CREATE VIEW " + view + " AS SELECT * FROM " + fullBaseTableName;
                     tenantConn.createStatement().execute(viewDDL);
@@ -392,11 +412,11 @@ public class UpgradeIT extends BaseHBaseManagedTimeIT {
                     removeBaseColumnCountKV(tenant, null, view);
                 }
             }
-            
+
             // create global views
             try (Connection globalConn = DriverManager.getConnection(getUrl())) {
                 String view = "GLOBAL_VIEW1";
-                
+
                 // view with its own column
                 String viewDDL = "CREATE VIEW " + view + " AS SELECT * FROM " + fullBaseTableName;
                 globalConn.createStatement().execute(viewDDL);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
index 3d1bb4a..564fdd9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
@@ -30,7 +30,7 @@ import java.util.Properties;
 import org.junit.Test;
 
 
-public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
+public class UpsertBigValuesIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final long INTEGER_MIN_MINUS_ONE = (long)Integer.MIN_VALUE - 1;
     private static final long INTEGER_MAX_PLUS_ONE = (long)Integer.MAX_VALUE + 1;
@@ -39,10 +39,11 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
     public void testIntegerPK() throws Exception {
         int[] testNumbers = {Integer.MIN_VALUE, Integer.MIN_VALUE + 1,
                 -2, -1, 0, 1, 2, Integer.MAX_VALUE - 1, Integer.MAX_VALUE};
-        ensureTableCreated(getUrl(),"PKIntValueTest");
+        String tableName = generateRandomString();
+        ensureTableCreated(getUrl(), tableName,"PKIntValueTest");
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String upsert = "UPSERT INTO PKIntValueTest VALUES(?)";
+        String upsert = "UPSERT INTO " + tableName + " VALUES(?)";
         PreparedStatement stmt = conn.prepareStatement(upsert);
         for (int i = 0; i < testNumbers.length; i++) {
             stmt.setInt(1, testNumbers[i]);
@@ -51,18 +52,18 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         conn.commit();
         conn.close();
         
-        String select = "SELECT COUNT(*) from PKIntValueTest";
+        String select = "SELECT COUNT(*) from " + tableName ;
         ResultSet rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM PKIntValueTest where pk >= " + Integer.MIN_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where pk >= " + Integer.MIN_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKIntValueTest where pk >= " + Integer.MIN_VALUE + 
+        select = "SELECT pk FROM " + tableName + " where pk >= " + Integer.MIN_VALUE +
                 " GROUP BY pk ORDER BY pk ASC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = 0; i < testNumbers.length; i++) {
@@ -73,12 +74,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         
         // NOTE: This case currently fails with an error message:
         // "Overflow trying to get next key for [-1, -1, -1, -1]"
-        select = "SELECT count(*) FROM PKIntValueTest where pk <= " + Integer.MAX_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where pk <= " + Integer.MAX_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKIntValueTest where pk <= " + Integer.MAX_VALUE + 
+        select = "SELECT pk FROM " + tableName + " where pk <= " + Integer.MAX_VALUE +
                 " GROUP BY pk ORDER BY pk DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = testNumbers.length - 1; i >= 0; i--) {
@@ -88,12 +89,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         
         // NOTE: This case currently fails since it is not retrieving the negative values.
-        select = "SELECT count(*) FROM PKIntValueTest where pk >= " + INTEGER_MIN_MINUS_ONE;
+        select = "SELECT count(*) FROM " + tableName + " where pk >= " + INTEGER_MIN_MINUS_ONE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKIntValueTest where pk >= " + INTEGER_MIN_MINUS_ONE + 
+        select = "SELECT pk FROM " + tableName + " where pk >= " + INTEGER_MIN_MINUS_ONE +
                 " GROUP BY pk ORDER BY pk ASC NULLS LAST ";
         rs = conn.createStatement().executeQuery(select);
         for (int i = 0; i < testNumbers.length; i++) {
@@ -103,12 +104,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         
         // NOTE: This test case fails because it is not retrieving positive values.
-        select = "SELECT count(*) FROM PKIntValueTest where pk <= " + INTEGER_MAX_PLUS_ONE;
+        select = "SELECT count(*) FROM " + tableName + " where pk <= " + INTEGER_MAX_PLUS_ONE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKIntValueTest where pk <= " + INTEGER_MAX_PLUS_ONE + 
+        select = "SELECT pk FROM " + tableName + " where pk <= " + INTEGER_MAX_PLUS_ONE +
                 " GROUP BY pk ORDER BY pk DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = testNumbers.length - 1; i >= 0; i--) {
@@ -123,12 +124,13 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
       // NOTE: Due to how we parse negative long, -9223372036854775808L, the minimum value of 
       // bigint is not recognizable in the current version. As a result, we start with 
       // Long.MIN_VALUE+1 as the smallest value.
+        String tableName = generateRandomString();
         long[] testNumbers = {Long.MIN_VALUE+1 , Long.MIN_VALUE+2 , 
                 -2L, -1L, 0L, 1L, 2L, Long.MAX_VALUE-1, Long.MAX_VALUE};
-        ensureTableCreated(getUrl(),"PKBigIntValueTest");
+        ensureTableCreated(getUrl(), tableName, "PKBigIntValueTest" );
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String upsert = "UPSERT INTO PKBigIntValueTest VALUES(?)";
+        String upsert = "UPSERT INTO " + tableName + " VALUES(?)";
         PreparedStatement stmt = conn.prepareStatement(upsert);
         for (int i=0; i<testNumbers.length; i++) {
             stmt.setLong(1, testNumbers[i]);
@@ -137,18 +139,18 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         conn.commit();
         conn.close();
         
-        String select = "SELECT COUNT(*) from PKBigIntValueTest";
+        String select = "SELECT COUNT(*) from " + tableName ;
         ResultSet rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM PKBigIntValueTest where pk >= " + (Long.MIN_VALUE + 1);
+        select = "SELECT count(*) FROM " + tableName + " where pk >= " + (Long.MIN_VALUE + 1);
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKBigIntValueTest WHERE pk >= " + (Long.MIN_VALUE + 1) +
+        select = "SELECT pk FROM " + tableName + " WHERE pk >= " + (Long.MIN_VALUE + 1) +
                 " GROUP BY pk ORDER BY pk ASC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = 0; i < testNumbers.length; i++) {
@@ -157,12 +159,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         }
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM PKBigIntValueTest where pk <= " + Long.MAX_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where pk <= " + Long.MAX_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT pk FROM PKBigIntValueTest WHERE pk <= " + Long.MAX_VALUE + 
+        select = "SELECT pk FROM " + tableName + " WHERE pk <= " + Long.MAX_VALUE +
                 " GROUP BY pk ORDER BY pk DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = testNumbers.length - 1; i >= 0; i--) {
@@ -206,12 +208,13 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testIntegerKV() throws Exception {
+        String tableName = generateRandomString();
         int[] testNumbers = {Integer.MIN_VALUE, Integer.MIN_VALUE + 1, 
                 -2, -1, 0, 1, 2, Integer.MAX_VALUE - 1, Integer.MAX_VALUE};
-        ensureTableCreated(getUrl(),"KVIntValueTest");
+        ensureTableCreated(getUrl(), tableName, "KVIntValueTest" );
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String upsert = "UPSERT INTO KVIntValueTest VALUES(?, ?)";
+        String upsert = "UPSERT INTO " + tableName + " VALUES(?, ?)";
         PreparedStatement stmt = conn.prepareStatement(upsert);
         for (int i=0; i<testNumbers.length; i++) {
             stmt.setInt(1, i);
@@ -221,18 +224,18 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         conn.commit();
         conn.close();
         
-        String select = "SELECT COUNT(*) from KVIntValueTest";
+        String select = "SELECT COUNT(*) from " + tableName ;
         ResultSet rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVIntValueTest where kv >= " + Integer.MIN_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where kv >= " + Integer.MIN_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVIntValueTest WHERE kv >= " + Integer.MIN_VALUE +
+        select = "SELECT kv FROM " + tableName + " WHERE kv >= " + Integer.MIN_VALUE +
                 " GROUP BY kv ORDER BY kv ASC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i=0; i<testNumbers.length; i++) {
@@ -241,12 +244,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         }
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVIntValueTest where kv <= " + Integer.MAX_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where kv <= " + Integer.MAX_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVIntValueTest WHERE kv <= " + Integer.MAX_VALUE +
+        select = "SELECT kv FROM " + tableName + " WHERE kv <= " + Integer.MAX_VALUE +
                 " GROUP BY kv ORDER BY kv DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i=testNumbers.length-1; i>=0; i--) {
@@ -255,12 +258,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         }
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVIntValueTest where kv >= " + INTEGER_MIN_MINUS_ONE;
+        select = "SELECT count(*) FROM " + tableName + " where kv >= " + INTEGER_MIN_MINUS_ONE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVIntValueTest WHERE kv >= " + INTEGER_MIN_MINUS_ONE +
+        select = "SELECT kv FROM " + tableName + " WHERE kv >= " + INTEGER_MIN_MINUS_ONE +
                 " GROUP BY kv ORDER BY kv ASC NULLS LAST ";
         rs = conn.createStatement().executeQuery(select);
         for (int i=0; i<testNumbers.length; i++) {
@@ -269,12 +272,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         }
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVIntValueTest where kv <= " + INTEGER_MAX_PLUS_ONE;
+        select = "SELECT count(*) FROM " + tableName + " where kv <= " + INTEGER_MAX_PLUS_ONE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVIntValueTest WHERE kv <= " + INTEGER_MAX_PLUS_ONE +
+        select = "SELECT kv FROM " + tableName + " WHERE kv <= " + INTEGER_MAX_PLUS_ONE +
                 " GROUP BY kv ORDER BY kv DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i=testNumbers.length-1; i>=0; i--) {
@@ -289,12 +292,13 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         // NOTE: Due to how we parse negative long, -9223372036854775808L, the minimum value of 
         // bigint is not recognizable in the current version. As a result, we start with 
         // Long.MIN_VALUE+1 as the smallest value.
+        String tableName = generateRandomString();
         long[] testNumbers = {Long.MIN_VALUE+1, Long.MIN_VALUE+2, 
                 -2L, -1L, 0L, 1L, 2L, Long.MAX_VALUE-1, Long.MAX_VALUE};
-        ensureTableCreated(getUrl(),"KVBigIntValueTest");
+        ensureTableCreated(getUrl(), tableName, "KVBigIntValueTest" );
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String upsert = "UPSERT INTO KVBigIntValueTest VALUES(?,?)";
+        String upsert = "UPSERT INTO " + tableName + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(upsert);
         for (int i = 0; i < testNumbers.length; i++) {
             stmt.setLong(1, i);
@@ -304,18 +308,18 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         conn.commit();
         conn.close();
         
-        String select = "SELECT COUNT(*) from KVBigIntValueTest";
+        String select = "SELECT COUNT(*) from " + tableName ;
         ResultSet rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVBigIntValueTest where kv >= " + (Long.MIN_VALUE+1);
+        select = "SELECT count(*) FROM " + tableName + " where kv >= " + (Long.MIN_VALUE+1);
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVBigIntValueTest WHERE kv >= " + (Long.MIN_VALUE+1) + 
+        select = "SELECT kv FROM " + tableName + " WHERE kv >= " + (Long.MIN_VALUE+1) + 
                 " GROUP BY kv ORDER BY kv ASC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = 0; i < testNumbers.length; i++) {
@@ -324,12 +328,12 @@ public class UpsertBigValuesIT extends BaseHBaseManagedTimeIT {
         }
         assertFalse(rs.next());
         
-        select = "SELECT count(*) FROM KVBigIntValueTest where kv <= " + Long.MAX_VALUE;
+        select = "SELECT count(*) FROM " + tableName + " where kv <= " + Long.MAX_VALUE;
         rs = conn.createStatement().executeQuery(select);
         assertTrue(rs.next());
         assertEquals(testNumbers.length, rs.getInt(1));
         assertFalse(rs.next());
-        select = "SELECT kv FROM KVBigIntValueTest WHERE kv <= " + Long.MAX_VALUE +
+        select = "SELECT kv FROM " + tableName + " WHERE kv <= " + Long.MAX_VALUE +
                 " GROUP BY kv ORDER BY kv DESC NULLS LAST";
         rs = conn.createStatement().executeQuery(select);
         for (int i = testNumbers.length-1; i >= 0; i--) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 4d811a4..8c9c8eb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -29,6 +29,7 @@ import static org.apache.phoenix.util.TestUtil.ROW7;
 import static org.apache.phoenix.util.TestUtil.ROW8;
 import static org.apache.phoenix.util.TestUtil.ROW9;
 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.assertNotNull;
@@ -88,8 +89,9 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     private void testUpsertSelect(boolean createIndex) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts-1);
-        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, ts-1);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+
+        ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts-1);
         String indexName = "IDX1";
         if (createIndex) {
             Properties props = new Properties();
@@ -208,8 +210,8 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     public void testUpsertSelectEmptyPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts-1);
-        ensureTableCreated(getUrl(), PTSDB_NAME, ts-1);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-1);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -384,8 +386,8 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     private void testUpsertSelectForAgg(boolean autoCommit) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts-1);
-        ensureTableCreated(getUrl(), PTSDB_NAME, ts-1);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-1);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -460,7 +462,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         byte[][] splits = new byte[][] { PInteger.INSTANCE.toBytes(1), PInteger.INSTANCE.toBytes(2),
                 PInteger.INSTANCE.toBytes(3), PInteger.INSTANCE.toBytes(4)};
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest",splits, ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits, ts-2);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -600,7 +602,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         byte[][] splits = new byte[][] { PInteger.INSTANCE.toBytes(1), PInteger.INSTANCE.toBytes(2),
                 PInteger.INSTANCE.toBytes(3), PInteger.INSTANCE.toBytes(4)};
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest",splits,ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits,ts-2);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index 9bbe23e..9725175 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -52,7 +52,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     @Test
     public void testGroupByWithLimitOverRowKey() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME, null, ts-2);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -85,7 +85,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     public void testUpsertDateValues() throws Exception {
         long ts = nextTimestamp();
         Date now = new Date(System.currentTimeMillis());
-        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME,null, ts-2);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -114,7 +114,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     @Test
     public void testUpsertValuesWithExpression() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest",null, ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest","IntKeyTest", null, ts-2);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/UseSchemaIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UseSchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UseSchemaIT.java
index cec23ed..c42c287 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UseSchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UseSchemaIT.java
@@ -41,16 +41,18 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
-public class UseSchemaIT extends BaseHBaseManagedTimeIT {
+public class UseSchemaIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testUseSchemaCaseInsensitive() throws Exception {
-        testUseSchema("TEST_SCHEMA");
+        String schemaName = generateRandomString();
+        testUseSchema(schemaName);
     }
 
     @Test
     public void testUseSchemaCaseSensitive() throws Exception {
-        testUseSchema("\"test_schema\"");
+        String schemaName = generateRandomString();
+        testUseSchema("\"" + schemaName + "\"");
     }
 
     public void testUseSchema(String schema) throws Exception {
@@ -59,25 +61,26 @@ public class UseSchemaIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String ddl = "CREATE SCHEMA IF NOT EXISTS "+schema;
         conn.createStatement().execute(ddl);
-        ddl = "create table "+schema+".TEST(id varchar primary key)";
+        String testTable = generateRandomString();
+        ddl = "create table "+schema+"." + testTable + "(id varchar primary key)";
         conn.createStatement().execute(ddl);
         conn.createStatement().execute("use "+schema);
-        String query = "select count(*) from TEST";
+        String query = "select count(*) from " + testTable;
         ResultSet rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals(0, rs.getInt(1));
         try {
-            conn.createStatement().execute("use test");
+            conn.createStatement().execute("use " + testTable);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.SCHEMA_NOT_FOUND.getErrorCode(), e.getErrorCode());
         }
         conn.createStatement().execute("use default");
-        ddl = "create table IF NOT EXISTS TEST(schema_name varchar primary key)";
+        ddl = "create table IF NOT EXISTS " + testTable + "(schema_name varchar primary key)";
         conn.createStatement().execute(ddl);
-        conn.createStatement().executeUpdate("upsert into test values('"+SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE+"')");
+        conn.createStatement().executeUpdate("upsert into " + testTable + " values('"+SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE+"')");
         conn.commit();
-        rs = conn.createStatement().executeQuery("select schema_name from TEST");
+        rs = conn.createStatement().executeQuery("select schema_name from " + testTable);
         assertTrue(rs.next());
         assertEquals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE, rs.getString(1));
         conn.close();
@@ -86,29 +89,30 @@ public class UseSchemaIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSchemaInJdbcUrl() throws Exception {
         Properties props = new Properties();
-        String schema = "TEST_SCHEMA";
+        String schema = generateRandomString();
         props.setProperty(QueryServices.SCHEMA_ATTRIB, schema);
         props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         String ddl = "CREATE SCHEMA IF NOT EXISTS " + schema;
         conn.createStatement().execute(ddl);
-        ddl = "create table IF NOT EXISTS " + schema + ".test(schema_name varchar primary key)";
+        String testTable = generateRandomString();
+        ddl = "create table IF NOT EXISTS " + schema + "." + testTable + " (schema_name varchar primary key)";
         conn.createStatement().execute(ddl);
-        conn.createStatement().executeUpdate("upsert into " + schema + ".test values('" + schema + "')");
-        String query = "select schema_name from test";
+        conn.createStatement().executeUpdate("upsert into " + schema + "." + testTable + " values('" + schema + "')");
+        String query = "select schema_name from " + testTable;
         ResultSet rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals(schema, rs.getString(1));
 
-        schema = "test";
+        schema = generateRandomString();
         ddl = "CREATE SCHEMA " + schema;
         conn.createStatement().execute(ddl);
         conn.createStatement().execute("use " + schema);
-        ddl = "create table test(schema_name varchar primary key)";
+        ddl = "create table " + testTable + "(schema_name varchar primary key)";
         conn.createStatement().execute(ddl);
-        conn.createStatement().executeUpdate("upsert into test values('" + schema + "')");
-        rs = conn.createStatement().executeQuery("select schema_name from test");
+        conn.createStatement().executeUpdate("upsert into " + testTable + " values('" + schema + "')");
+        rs = conn.createStatement().executeQuery("select schema_name from " + testTable );
         assertTrue(rs.next());
         assertEquals(schema, rs.getString(1));
         conn.close();
@@ -117,8 +121,8 @@ public class UseSchemaIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testMappedView() throws Exception {
         Properties props = new Properties();
-        String schema = "TEST_SCHEMA_V";
-        String tableName = "TEST";
+        String schema = generateRandomString();
+        String tableName = generateRandomString();
         String fullTablename = schema + QueryConstants.NAME_SEPARATOR + tableName;
         props.setProperty(QueryServices.SCHEMA_ATTRIB, schema);
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index c0b557c..4bc0f43 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -58,7 +58,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     protected static void initGroupByRowKeyColumns(long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -85,7 +85,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, splits, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -106,7 +106,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
         stmt.setBigDecimal(4, new BigDecimal(.5));
         stmt.execute();
 
-        ensureTableCreated(getUrl(),BTABLE_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, splits, ts-2);
         conn.setAutoCommit(false);
 
         // Insert all rows at ts
@@ -431,7 +431,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testNullValueEqualityScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -459,7 +459,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testVarLengthPKColScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -489,7 +489,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testEscapedQuoteScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -527,7 +527,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     private static void initPtsdbTableValues(long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -560,7 +560,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     private static void initPtsdbTableValues2(long ts, Date d) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB2_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -696,7 +696,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testBatchUpsert() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB2_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2);
         Date d = new Date(ts);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -874,7 +874,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testMissingPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -894,7 +894,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testNoKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -914,7 +914,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     // Broken, since we don't know if insert vs update. @Test
     public void testMissingKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -942,7 +942,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooShortKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -978,7 +978,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooShortPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1014,7 +1014,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooLongPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1051,7 +1051,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooLongKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1481,7 +1481,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testLikeOnColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -1598,7 +1598,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testILikeOnColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -1730,7 +1730,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testIsNullInPK() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
index ec360ad..93f8e27 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
@@ -18,9 +18,7 @@
 package org.apache.phoenix.end2end.index;
 
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
-import static org.apache.phoenix.util.TestUtil.HBASE_NATIVE;
-import static org.apache.phoenix.util.TestUtil.HBASE_NATIVE_SCHEMA_NAME;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -41,7 +39,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
@@ -61,17 +59,12 @@ import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class DropMetadataIT extends BaseHBaseManagedTimeIT {
-    private static final byte[] HBASE_NATIVE_BYTES = SchemaUtil.getTableNameAsBytes(HBASE_NATIVE_SCHEMA_NAME, HBASE_NATIVE);
+public class DropMetadataIT extends BaseHBaseManagedTimeTableReuseIT {
     private static final byte[] FAMILY_NAME = Bytes.toBytes(SchemaUtil.normalizeIdentifier("1"));
     public static final String SCHEMA_NAME = "";
-    public static final String DATA_TABLE_NAME = "T";
-    public static final String INDEX_TABLE_NAME = "I";
-    public static final String DATA_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "T");
-    public static final String INDEX_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, "I");
     private final String TENANT_SPECIFIC_URL = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
     
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     @BeforeClass 
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
@@ -84,14 +77,12 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testDropViewKeepsHTable() throws Exception {
         HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TEST_PROPERTIES).getAdmin();
+        String hbaseNativeViewName = generateRandomString();
+
+        byte[] hbaseNativeBytes = SchemaUtil.getTableNameAsBytes(HBASE_NATIVE_SCHEMA_NAME, hbaseNativeViewName);
         try {
-            try {
-                admin.disableTable(HBASE_NATIVE_BYTES);
-                admin.deleteTable(HBASE_NATIVE_BYTES);
-            } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
-            }
             @SuppressWarnings("deprecation")
-            HTableDescriptor descriptor = new HTableDescriptor(HBASE_NATIVE_BYTES);
+            HTableDescriptor descriptor = new HTableDescriptor(hbaseNativeBytes);
             HColumnDescriptor columnDescriptor =  new HColumnDescriptor(FAMILY_NAME);
             columnDescriptor.setKeepDeletedCells(true);
             descriptor.addFamily(columnDescriptor);
@@ -102,7 +93,7 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("create view " + HBASE_NATIVE +
+        conn.createStatement().execute("create view " + hbaseNativeViewName+
                 "   (uint_key unsigned_int not null," +
                 "    ulong_key unsigned_long not null," +
                 "    string_key varchar not null,\n" +
@@ -110,41 +101,32 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
                 "    \"1\".ulong_col unsigned_long" +
                 "    CONSTRAINT pk PRIMARY KEY (uint_key, ulong_key, string_key))\n" +
                      HColumnDescriptor.DATA_BLOCK_ENCODING + "='" + DataBlockEncoding.NONE + "'");
-        conn.createStatement().execute("drop view " + HBASE_NATIVE);
-        
-        admin = driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)).getAdmin();
-        try {
-            try {
-                admin.disableTable(HBASE_NATIVE_BYTES);
-                admin.deleteTable(HBASE_NATIVE_BYTES);
-            } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
-                fail(); // The underlying HBase table should still exist
-            }
-        } finally {
-            admin.close();
-        }
+        conn.createStatement().execute("drop view " + hbaseNativeViewName);
+
     }
     
     @Test
     public void testDroppingIndexedColDropsIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String localIndexTableName1 = "LOCAL_" + INDEX_TABLE_NAME + "_1";
-        String localIndexTableName2 = "LOCAL_" + INDEX_TABLE_NAME + "_2";
+        String indexTableName = generateRandomString();
+        String dataTableFullName = SchemaUtil.getTableName(SCHEMA_NAME, generateRandomString());
+        String localIndexTableName1 = "LOCAL_" + indexTableName + "_1";
+        String localIndexTableName2 = "LOCAL_" + indexTableName + "_2";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
             conn.createStatement().execute(
-                "CREATE TABLE " + DATA_TABLE_FULL_NAME
+                "CREATE TABLE " + dataTableFullName
                         + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
             // create one regular and two local indexes
             conn.createStatement().execute(
-                "CREATE INDEX " + INDEX_TABLE_NAME + " ON " + DATA_TABLE_FULL_NAME + " (v2) INCLUDE (v1)");
+                "CREATE INDEX " + indexTableName + " ON " + dataTableFullName + " (v2) INCLUDE (v1)");
             conn.createStatement().execute(
-                "CREATE LOCAL INDEX " + localIndexTableName1 + " ON " + DATA_TABLE_FULL_NAME + " (v2) INCLUDE (v1)");
+                "CREATE LOCAL INDEX " + localIndexTableName1 + " ON " + dataTableFullName + " (v2) INCLUDE (v1)");
             conn.createStatement().execute(
-                "CREATE LOCAL INDEX " + localIndexTableName2 + " ON " + DATA_TABLE_FULL_NAME + " (k) INCLUDE (v1)");
+                "CREATE LOCAL INDEX " + localIndexTableName2 + " ON " + dataTableFullName + " (k) INCLUDE (v1)");
             
             // upsert a single row
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + DATA_TABLE_FULL_NAME + " VALUES(?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableFullName + " VALUES(?,?,?)");
             stmt.setString(1, "a");
             stmt.setString(2, "x");
             stmt.setString(3, "1");
@@ -153,7 +135,7 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
             
             // verify the indexes were created
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-            PTable dataTable = pconn.getTable(new PTableKey(null, DATA_TABLE_FULL_NAME));
+            PTable dataTable = pconn.getTable(new PTableKey(null, dataTableFullName));
             assertEquals("Unexpected number of indexes ", 3, dataTable.getIndexes().size());
             PTable indexTable = dataTable.getIndexes().get(0);
             byte[] indexTablePhysicalName = indexTable.getPhysicalName().getBytes();
@@ -161,19 +143,19 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
             
             // drop v2 which causes the regular index and first local index to be dropped
             conn.createStatement().execute(
-                "ALTER TABLE " + DATA_TABLE_FULL_NAME + " DROP COLUMN v2 ");
+                "ALTER TABLE " + dataTableFullName + " DROP COLUMN v2 ");
 
             // verify the both of the indexes' metadata were dropped
-            conn.createStatement().execute("SELECT * FROM "+DATA_TABLE_FULL_NAME);
+            conn.createStatement().execute("SELECT * FROM "+dataTableFullName);
             try {
-                conn.createStatement().execute("SELECT * FROM "+INDEX_TABLE_NAME);
+                conn.createStatement().execute("SELECT * FROM "+indexTableName);
                 fail("Index should have been dropped");
             } catch (TableNotFoundException e) {
             }
             pconn = conn.unwrap(PhoenixConnection.class);
-            dataTable = pconn.getTable(new PTableKey(null, DATA_TABLE_FULL_NAME));
+            dataTable = pconn.getTable(new PTableKey(null, dataTableFullName));
             try {
-                pconn.getTable(new PTableKey(null, INDEX_TABLE_NAME));
+                pconn.getTable(new PTableKey(null, indexTableName));
                 fail("index should have been dropped");
             } catch (TableNotFoundException e) {
             }
@@ -220,28 +202,33 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
     public void helpTestDroppingIndexedColDropsViewIndex(boolean isMultiTenant) throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : conn ) {
+            String tableWithView = generateRandomString();
+            String viewOfTable = generateRandomString();
+            String viewIndex1 = generateRandomString();
+            String viewIndex2 = generateRandomString();
+            
             conn.setAutoCommit(false);
             viewConn.setAutoCommit(false);
-            String ddlFormat = "CREATE TABLE TABLEWITHVIEW (%s k VARCHAR NOT NULL, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR, v4 VARCHAR CONSTRAINT PK PRIMARY KEY(%s k))%s";
+            String ddlFormat = "CREATE TABLE " + tableWithView + " (%s k VARCHAR NOT NULL, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR, v4 VARCHAR CONSTRAINT PK PRIMARY KEY(%s k))%s";
             String ddl = String.format(ddlFormat, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
                     isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
             conn.createStatement().execute(ddl);
             viewConn.createStatement()
                     .execute(
-                        "CREATE VIEW VIEWOFTABLE ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM TABLEWITHVIEW");
+                        "CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableWithView );
             // create an index with the column that will be dropped
-            viewConn.createStatement().execute("CREATE INDEX VIEWINDEX1 ON VIEWOFTABLE(v2) INCLUDE (v4)");
+            viewConn.createStatement().execute("CREATE INDEX " + viewIndex1 + " ON " + viewOfTable + "(v2) INCLUDE (v4)");
             // create an index without the column that will be dropped
-            viewConn.createStatement().execute("CREATE INDEX VIEWINDEX2 ON VIEWOFTABLE(v1) INCLUDE (v4)");
+            viewConn.createStatement().execute("CREATE INDEX " + viewIndex2 + " ON " + viewOfTable + "(v1) INCLUDE (v4)");
             // verify index was created
             try {
-                viewConn.createStatement().execute("SELECT * FROM VIEWINDEX1");
+                viewConn.createStatement().execute("SELECT * FROM " + viewIndex1 );
             } catch (TableNotFoundException e) {
                 fail("Index on view was not created");
             }
             
             // upsert a single row
-            PreparedStatement stmt = viewConn.prepareStatement("UPSERT INTO VIEWOFTABLE VALUES(?,?,?,?,?,?,?)");
+            PreparedStatement stmt = viewConn.prepareStatement("UPSERT INTO " + viewOfTable + " VALUES(?,?,?,?,?,?,?)");
             stmt.setString(1, "a");
             stmt.setString(2, "b");
             stmt.setString(3, "c");
@@ -255,47 +242,47 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
             // verify the index was created
             PhoenixConnection pconn = viewConn.unwrap(PhoenixConnection.class);
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null; 
-            PTable view = pconn.getTable(new PTableKey(tenantId, "VIEWOFTABLE"));
-            PTable viewIndex = pconn.getTable(new PTableKey(tenantId, "VIEWINDEX1"));
+            PTable view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
+            PTable viewIndex = pconn.getTable(new PTableKey(tenantId,  viewIndex1 ));
             byte[] viewIndexPhysicalTable = viewIndex.getPhysicalName().getBytes();
             assertNotNull("Can't find view index", viewIndex);
             assertEquals("Unexpected number of indexes ", 2, view.getIndexes().size());
-            assertEquals("Unexpected index ", "VIEWINDEX1", view.getIndexes().get(0).getName()
+            assertEquals("Unexpected index ",  viewIndex1 , view.getIndexes().get(0).getName()
                     .getString());
-            assertEquals("Unexpected index ", "VIEWINDEX2", view.getIndexes().get(1).getName()
+            assertEquals("Unexpected index ",  viewIndex2 , view.getIndexes().get(1).getName()
                 .getString());
             
             // drop two columns
-            conn.createStatement().execute("ALTER TABLE TABLEWITHVIEW DROP COLUMN v2, v3 ");
+            conn.createStatement().execute("ALTER TABLE " + tableWithView + " DROP COLUMN v2, v3 ");
             
             // verify columns were dropped
             try {
-                conn.createStatement().execute("SELECT v2 FROM TABLEWITHVIEW");
+                conn.createStatement().execute("SELECT v2 FROM " + tableWithView );
                 fail("Column should have been dropped");
             } catch (ColumnNotFoundException e) {
             }
             try {
-                conn.createStatement().execute("SELECT v3 FROM TABLEWITHVIEW");
+                conn.createStatement().execute("SELECT v3 FROM " + tableWithView );
                 fail("Column should have been dropped");
             } catch (ColumnNotFoundException e) {
             }
             
             // verify index metadata was dropped
             try {
-                viewConn.createStatement().execute("SELECT * FROM VIEWINDEX1");
+                viewConn.createStatement().execute("SELECT * FROM " + viewIndex1 );
                 fail("Index metadata should have been dropped");
             } catch (TableNotFoundException e) {
             }
             
             pconn = viewConn.unwrap(PhoenixConnection.class);
-            view = pconn.getTable(new PTableKey(tenantId, "VIEWOFTABLE"));
+            view = pconn.getTable(new PTableKey(tenantId,  viewOfTable ));
             try {
-                viewIndex = pconn.getTable(new PTableKey(tenantId, "VIEWINDEX1"));
+                viewIndex = pconn.getTable(new PTableKey(tenantId,  viewIndex1 ));
                 fail("View index should have been dropped");
             } catch (TableNotFoundException e) {
             }
             assertEquals("Unexpected number of indexes ", 1, view.getIndexes().size());
-            assertEquals("Unexpected index ", "VIEWINDEX2", view.getIndexes().get(0).getName().getString());
+            assertEquals("Unexpected index ",  viewIndex2 , view.getIndexes().get(0).getName().getString());
             
             // verify that the physical index view table is *not* dropped
             conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(viewIndexPhysicalTable);
@@ -306,8 +293,8 @@ public class DropMetadataIT extends BaseHBaseManagedTimeIT {
             ResultScanner results = table.getScanner(scan);
             Result result = results.next();
             assertNotNull(result);
-            // there should be a single row belonging to VIEWINDEX2 
-            assertNotNull("VIEWINDEX2 row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
+            // there should be a single row belonging to " + viewIndex2 + " 
+            assertNotNull( viewIndex2 + " row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
                 IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4").getBytes()));
             assertNull(results.next());
         }


[50/50] [abbrv] phoenix git commit: Fix compilation errors from merge

Posted by ma...@apache.org.
Fix compilation errors from merge


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

Branch: refs/heads/calcite
Commit: faff90afdebfa007f7f82e65678cdb90a36b6a43
Parents: a9526a9 6861a5e
Author: maryannxue <ma...@gmail.com>
Authored: Wed Sep 14 14:42:36 2016 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Wed Sep 14 14:42:36 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/calcite/BaseCalciteIT.java   |  51 +++---
 .../phoenix/calcite/BaseCalciteIndexIT.java     |   9 +-
 .../apache/phoenix/calcite/CalciteDMLIT.java    |   2 +-
 .../org/apache/phoenix/calcite/CalciteIT.java   |  11 +-
 .../end2end/BaseClientManagedTimeIT.java        |   2 +-
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java |   2 +-
 .../BaseOwnClusterClientManagedTimeIT.java      |   2 +-
 .../BaseOwnClusterHBaseManagedTimeIT.java       |   2 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  |  36 ++--
 .../calcite/jdbc/PhoenixCalciteFactory.java     |  94 +++++++++++
 .../calcite/jdbc/PhoenixCalciteDriver.java      | 101 +++---------
 .../jdbc/PhoenixCalciteEmbeddedDriver.java      | 116 +++++++++++++
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  10 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |   4 +
 .../java/org/apache/phoenix/util/QueryUtil.java |  30 ++--
 .../phoenix/jdbc/PhoenixCalciteTestDriver.java  |  43 +++++
 .../apache/phoenix/jdbc/PhoenixTestDriver.java  |   3 -
 .../query/BaseConnectionlessQueryTest.java      |  36 ++--
 .../java/org/apache/phoenix/query/BaseTest.java | 164 +++++++++++++++----
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +
 21 files changed, 524 insertions(+), 198 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
index 7a16ed1,3ea8db7..5f98f8d
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
@@@ -43,7 -43,7 +43,7 @@@ import java.util.Properties
  import org.apache.calcite.avatica.util.ArrayImpl;
  import org.apache.calcite.config.CalciteConnectionProperty;
  import org.apache.phoenix.calcite.rel.PhoenixRel;
--import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
++import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
  import org.apache.phoenix.query.QueryServices;
  import org.apache.phoenix.schema.TableAlreadyExistsException;
  import org.apache.phoenix.util.PhoenixRuntime;
@@@ -53,12 -54,12 +54,14 @@@ import org.junit.Assert
  import org.junit.BeforeClass;
  
  import com.google.common.collect.Lists;
++import com.google.common.collect.Maps;
  
--public class BaseCalciteIT extends BaseClientManagedTimeIT {
++public class BaseCalciteIT extends BaseHBaseManagedTimeIT {
      
      @BeforeClass
      public static void doSetup() throws Exception {
--        Map<String,String> props = getDefaultProps();
++        Map<String,String> props = Maps.newHashMapWithExpectedSize(5);
++        props.put(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB, Boolean.FALSE.toString());
          props.put(QueryServices.RUN_UPDATE_STATS_ASYNC, Boolean.FALSE.toString());
          props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(1000));
          props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(200));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIndexIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIndexIT.java
index 62943aa,1f32591..b7e5955
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIndexIT.java
@@@ -21,6 -21,6 +21,7 @@@ import java.sql.Connection
  import java.sql.DriverManager;
  import java.util.Properties;
  
++import org.apache.phoenix.util.TestUtil;
  import org.junit.Before;
  
  public class BaseCalciteIndexIT extends BaseCalciteIT {
@@@ -33,11 -33,11 +34,11 @@@
      
      @Before
      public void initTable() throws Exception {
-         final String url = getUrl();
+         final String url = getOldUrl();
          final String index = localIndex ? "LOCAL INDEX" : "INDEX";
--        initATableValues(getOrganizationId(), null, url);
-         initSaltedTables(index);
-         initMultiTenantTables(index);
++        initATableValues(TestUtil.ATABLE_NAME, getOrganizationId(), null, null, null, url);
+         initSaltedTables(url, index);
+         initMultiTenantTables(url, index);
          Connection connection = DriverManager.getConnection(url);
          connection.createStatement().execute("CREATE " + index + " IF NOT EXISTS IDX1 ON aTable (a_string) INCLUDE (b_string, x_integer)");
          connection.createStatement().execute("CREATE " + index + " IF NOT EXISTS IDX2 ON aTable (b_string) INCLUDE (a_string, y_integer)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDMLIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDMLIT.java
index da60b15,da60b15..6705042
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteDMLIT.java
@@@ -21,7 -21,7 +21,7 @@@ public class CalciteDMLIT extends BaseC
      
      @Before
      public void initTable() throws Exception {
--        final String url = getUrl();
++        final String url = getOldUrl();
          ensureTableCreated(url, ATABLE_NAME);
      }
  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
index e65d386,0ecf330..da6303b
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
@@@ -30,6 -30,6 +30,7 @@@ import java.sql.Timestamp
  import java.text.SimpleDateFormat;
  import java.util.Properties;
  
++import org.apache.phoenix.util.TestUtil;
  import org.junit.Before;
  import org.junit.Ignore;
  import org.junit.Test;
@@@ -41,12 -41,12 +42,12 @@@ public class CalciteIT extends BaseCalc
      
      @Before
      public void initTable() throws Exception {
-         final String url = getUrl();
-         initATableValues(getOrganizationId(), null, url);
+         final String url = getOldUrl();
 -        initATableValues(getOrganizationId(), null, url);
++        initATableValues(TestUtil.ATABLE_NAME, getOrganizationId(), null, null, null, url);
          initJoinTableValues(url, null, null);
-         initArrayTable();
-         initSaltedTables(null);
-         initKeyOrderingTable();
+         initArrayTable(url);
+         initSaltedTables(url, null);
+         initKeyOrderingTable(url);
          final Connection connection = DriverManager.getConnection(url);
          connection.createStatement().execute("CREATE VIEW IF NOT EXISTS v AS SELECT * from aTable where a_string = 'a'");
          connection.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS seq0 START WITH 1 INCREMENT BY 1");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
index dee7200,0dada7a..879d9fd
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
@@@ -56,7 -56,7 +56,7 @@@ public abstract class BaseClientManaged
      @After
      public void cleanUpAfterTest() throws Exception {
          long ts = nextTimestamp();
-         deletePriorMetaData(ts - 1, getUrl());
 -        deletePriorTables(ts - 1, getOldUrl());    
++        deletePriorMetaData(ts - 1, getOldUrl());
      }
      
      public static Map<String,String> getDefaultProps() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
index 7439b1d,6aeafbf..8da0998
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
@@@ -64,6 -61,6 +64,6 @@@ public abstract class BaseHBaseManagedT
      
      @After
      public void cleanUpAfterTest() throws Exception {
-         deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
 -        deletePriorTables(HConstants.LATEST_TIMESTAMP, getOldUrl());    
++        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getOldUrl());
      }
  }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterClientManagedTimeIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterClientManagedTimeIT.java
index 6ece674,25b3128..5631e19
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterClientManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterClientManagedTimeIT.java
@@@ -24,6 -26,6 +24,6 @@@ public class BaseOwnClusterClientManage
      @After
      public void cleanUpAfterTest() throws Exception {
          long ts = nextTimestamp();
-         deletePriorMetaData(ts - 1, getUrl());
 -        deletePriorTables(ts - 1, getOldUrl());    
++        deletePriorMetaData(ts - 1, getOldUrl());
      }    
  }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterHBaseManagedTimeIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterHBaseManagedTimeIT.java
index 63a4300,a7458f2..3569a6e
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterHBaseManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseOwnClusterHBaseManagedTimeIT.java
@@@ -24,6 -26,6 +24,6 @@@ import org.junit.After
  public class BaseOwnClusterHBaseManagedTimeIT extends BaseOwnClusterIT {
      @After
      public void cleanUpAfterTest() throws Exception {
-         deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getUrl());
 -        deletePriorTables(HConstants.LATEST_TIMESTAMP, getOldUrl());    
++        deletePriorMetaData(HConstants.LATEST_TIMESTAMP, getOldUrl());
      }
  }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index 272fb22,566d8d8..047cc24
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@@ -416,23 -329,20 +416,29 @@@ public abstract class PhoenixEmbeddedDr
          private final boolean isConnectionless;
          private final String principal;
          private final String keytab;
 +        private final User user;
+         private final boolean isCalciteEnabled;
          
          public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode, String principal, String keytab) {
+          this(zookeeperQuorum, port, rootNode, principal, keytab, false);
+         }
+         
+         public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode, String principal, String keytab, boolean isCalciteEnabled) {
              this.zookeeperQuorum = zookeeperQuorum;
              this.port = port;
              this.rootNode = rootNode;
              this.isConnectionless = PhoenixRuntime.CONNECTIONLESS.equals(zookeeperQuorum);
              this.principal = principal;
              this.keytab = keytab;
 +            try {
 +                this.user = User.getCurrent();
 +            } catch (IOException e) {
 +                throw new RuntimeException("Couldn't get the current user!!");
 +            }
 +            if (null == this.user) {
 +                throw new RuntimeException("Acquired null user which should never happen");
 +            }
+             this.isCalciteEnabled = isCalciteEnabled;
          }
          
          public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode) {
@@@ -544,26 -437,9 +550,26 @@@
  		}
  
          public String toUrl() {
-             return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR
-                     + toString();
+             return (isCalciteEnabled? PhoenixRuntime.JDBC_PROTOCOL_CALCITE : PhoenixRuntime.JDBC_PROTOCOL)
+                     + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + toString();
          }
 +
 +        private static ConnectionInfo defaultConnectionInfo(String url) throws SQLException {
 +            Configuration config =
 +                    HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
 +            String quorum = config.get(HConstants.ZOOKEEPER_QUORUM);
 +            if (quorum == null || quorum.isEmpty()) {
 +                throw getMalFormedUrlException(url);
 +            }
 +            String clientPort = config.get(HConstants.ZOOKEEPER_CLIENT_PORT);
 +            Integer port = clientPort==null ? null : Integer.parseInt(clientPort);
 +            if (port == null || port < 0) {
 +                throw getMalFormedUrlException(url);
 +            }
 +            String znodeParent = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 +            LOG.debug("Getting default jdbc connection url " + quorum + ":" + port + ":" + znodeParent);
 +            return new ConnectionInfo(quorum, port, znodeParent);
 +        }
      }
  
      public static boolean isTestUrl(String url) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index 6d8e00d,ba6227a..69690a6
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@@ -233,53 -229,46 +233,53 @@@ public final class QueryUtil 
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
      public static String getUrl(String zkQuorum) {
-         return getUrlInternal(zkQuorum, null, null, null);
 -        return getUrlInternal(zkQuorum, null, null, false);
++        return getUrlInternal(zkQuorum, null, null, null, false);
      }
  
      /**
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
      public static String getUrl(String zkQuorum, int clientPort) {
-         return getUrlInternal(zkQuorum, clientPort, null, null);
 -        return getUrlInternal(zkQuorum, clientPort, null, false);
++        return getUrlInternal(zkQuorum, clientPort, null, null, false);
      }
  
      /**
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
      public static String getUrl(String zkQuorum, String znodeParent) {
-         return getUrlInternal(zkQuorum, null, znodeParent, null);
 -        return getUrlInternal(zkQuorum, null, znodeParent, false);
++        return getUrlInternal(zkQuorum, null, znodeParent, null, false);
 +    }
 +
 +    /**
 +     * Create the Phoenix JDBC connection URL from the provided cluster connection details.
 +     */
 +    public static String getUrl(String zkQuorum, int port, String znodeParent, String principal) {
-         return getUrlInternal(zkQuorum, port, znodeParent, principal);
++        return getUrlInternal(zkQuorum, port, znodeParent, principal, false);
      }
  
      /**
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
      public static String getUrl(String zkQuorum, int port, String znodeParent) {
-         return getUrlInternal(zkQuorum, port, znodeParent, null);
 -        return getUrlInternal(zkQuorum, port, znodeParent, false);
++        return getUrlInternal(zkQuorum, port, znodeParent, null, false);
      }
  
      /**
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
      public static String getUrl(String zkQuorum, Integer port, String znodeParent) {
-         return getUrlInternal(zkQuorum, port, znodeParent, null);
 -        return getUrlInternal(zkQuorum, port, znodeParent, false);
++        return getUrlInternal(zkQuorum, port, znodeParent, null, false);
      }
  
      /**
       * Create the Phoenix JDBC connection URL from the provided cluster connection details.
       */
-     public static String getUrl(String zkQuorum, Integer port, String znodeParent, String principal) {
-         return getUrlInternal(zkQuorum, port, znodeParent, principal);
 -    public static String getUrl(String zkQuorum, Integer port, String znodeParent, boolean isCalciteEnabled) {
 -        return getUrlInternal(zkQuorum, port, znodeParent, isCalciteEnabled);
++    public static String getUrl(String zkQuorum, Integer port, String znodeParent, String principal, boolean isCalciteEnabled) {
++        return getUrlInternal(zkQuorum, port, znodeParent, principal, isCalciteEnabled);
      }
  
-     private static String getUrlInternal(String zkQuorum, Integer port, String znodeParent, String principal) {
-         return new PhoenixEmbeddedDriver.ConnectionInfo(zkQuorum, port, znodeParent, principal, null).toUrl()
 -    private static String getUrlInternal(String zkQuorum, Integer port, String znodeParent, boolean isCalciteEnabled) {
 -        return new PhoenixEmbeddedDriver.ConnectionInfo(zkQuorum, port, znodeParent, null, null, isCalciteEnabled).toUrl()
++    private static String getUrlInternal(String zkQuorum, Integer port, String znodeParent, String principal, boolean isCalciteEnabled) {
++        return new PhoenixEmbeddedDriver.ConnectionInfo(zkQuorum, port, znodeParent, principal, null, isCalciteEnabled).toUrl()
                  + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
      }
  
@@@ -342,10 -314,12 +342,16 @@@
  
      public static String getConnectionUrl(Properties props, Configuration conf)
              throws ClassNotFoundException, SQLException {
-         return getConnectionUrl(props, conf, null);
 -        return getConnectionUrl(props, conf, false);
++        return getConnectionUrl(props, conf, null, false);
      }
 -    
 -    public static String getConnectionUrl(Properties props, Configuration conf,
 -            boolean isCalciteEnabled)
++
 +    public static String getConnectionUrl(Properties props, Configuration conf, String principal)
 +            throws ClassNotFoundException, SQLException {
++        return getConnectionUrl(props, conf, principal, false);
++    }
++
++    public static String getConnectionUrl(Properties props, Configuration conf, String principal, boolean isCalciteEnabled)
+             throws ClassNotFoundException, SQLException {
          // TODO: props is ignored!
          // read the hbase properties from the configuration
          String server = ZKConfig.getZKQuorumServersString(conf);
@@@ -380,7 -354,7 +386,7 @@@
          server = Joiner.on(',').join(servers);
          String znodeParent = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
                  HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
-         String url = getUrl(server, port, znodeParent, principal);
 -        String url = getUrl(server, port, znodeParent, isCalciteEnabled);
++        String url = getUrl(server, port, znodeParent, principal, isCalciteEnabled);
          // Mainly for testing to tack on the test=true part to ensure driver is found on server
          String extraArgs = conf.get(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
          if (extraArgs.length() > 0) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index cb0bceb,d511b36..e064dc9
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@@ -134,13 -133,17 +135,13 @@@ import org.apache.hadoop.hbase.io.encod
  import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
  import org.apache.hadoop.hbase.ipc.controller.ServerRpcControllerFactory;
 -import org.apache.hadoop.hbase.master.LoadBalancer;
 -import org.apache.hadoop.hbase.regionserver.LocalIndexMerger;
  import org.apache.hadoop.hbase.regionserver.RSRpcServices;
--import org.apache.hadoop.hbase.util.Bytes;
  import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
  import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
  import org.apache.phoenix.exception.SQLExceptionCode;
  import org.apache.phoenix.exception.SQLExceptionInfo;
 -import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
 -import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
 +import org.apache.phoenix.jdbc.PhoenixConnection;
+ import org.apache.phoenix.jdbc.PhoenixCalciteTestDriver;
  import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
  import org.apache.phoenix.jdbc.PhoenixDriver;
  import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
@@@ -645,7 -656,8 +655,8 @@@ public abstract class BaseTest 
              try {
                  utility.startMiniCluster(NUM_SLAVES_BASE);
                  utility.startMiniMapReduceCluster();
 -                url = QueryUtil.getConnectionUrl(new Properties(), utility.getConfiguration(), false);
 +                url = QueryUtil.getConnectionUrl(new Properties(), utility.getConfiguration());
+                 calciteUrl = url.replaceFirst(PhoenixRuntime.JDBC_PROTOCOL, PhoenixRuntime.JDBC_PROTOCOL_CALCITE);
              } catch (Throwable t) {
                  throw new RuntimeException(t);
              }
@@@ -893,61 -884,16 +906,63 @@@
      }
      
      protected static byte[][] getDefaultSplits(String tenantId) {
-         return new byte[][] { 
-             Bytes.toBytes(tenantId + "00A"),
-             Bytes.toBytes(tenantId + "00B"),
-             Bytes.toBytes(tenantId + "00C"),
-             };
+     	return null;
+     	// TODO CALCITE-1319
+ //        return new byte[][] { 
+ //            Bytes.toBytes(tenantId + "00A"),
+ //            Bytes.toBytes(tenantId + "00B"),
+ //            Bytes.toBytes(tenantId + "00C"),
+ //            };
      }
 -    
 -    protected static void deletePriorTables(long ts, String url) throws Exception {
 +
 +    private static void deletePriorSchemas(long ts, String url) throws Exception {
 +        Properties props = new Properties();
 +        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1024));
 +        if (ts != HConstants.LATEST_TIMESTAMP) {
 +            props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts));
 +        }
 +        try (Connection conn = DriverManager.getConnection(url, props)) {
 +            DatabaseMetaData dbmd = conn.getMetaData();
 +            ResultSet rs = dbmd.getSchemas();
 +            while (rs.next()) {
 +                String schemaName = rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM);
 +                if (schemaName.equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME)) {
 +                    continue;
 +                }
 +                schemaName = SchemaUtil.getEscapedArgument(schemaName);
 +
 +                String ddl = "DROP SCHEMA " + schemaName;
 +                conn.createStatement().executeUpdate(ddl);
 +            }
 +            rs.close();
 +        }
 +        // Make sure all schemas have been dropped
 +        props.remove(CURRENT_SCN_ATTRIB);
 +        try (Connection seeLatestConn = DriverManager.getConnection(url, props)) {
 +            DatabaseMetaData dbmd = seeLatestConn.getMetaData();
 +            ResultSet rs = dbmd.getSchemas();
 +            boolean hasSchemas = rs.next();
 +            if (hasSchemas) {
 +                String schemaName = rs.getString(PhoenixDatabaseMetaData.TABLE_SCHEM);
 +                if (schemaName.equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME)) {
 +                    hasSchemas = rs.next();
 +                }
 +            }
 +            if (hasSchemas) {
 +                fail("The following schemas are not dropped that should be:" + getSchemaNames(rs));
 +            }
 +        }
 +    }
 +
 +    protected static void deletePriorMetaData(long ts, String url) throws Exception {
 +        deletePriorTables(ts, url);
 +        if (ts != HConstants.LATEST_TIMESTAMP) {
 +            ts = nextTimestamp() - 1;
 +        }
 +        deletePriorSchemas(ts, url);
 +    }
 +
 +    private static void deletePriorTables(long ts, String url) throws Exception {
          deletePriorTables(ts, (String)null, url);
      }
      
@@@ -1345,15 -1248,17 +1360,14 @@@
              stmt.execute();
                  
              conn.commit();
++            return tableName;
          } finally {
              conn.close();
-             return tableName;
          }
      }
- 
      
 -    protected static void initATableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
 -        initATableValues(tenantId, splits, date, ts, getOldUrl());
 -    }
 -    
      protected static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
-         initEntityHistoryTableValues(tenantId, splits, date, ts, getUrl());
+         initEntityHistoryTableValues(tenantId, splits, date, ts, getOldUrl());
      }
      
      protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts) throws Exception {
@@@ -1594,9 -1499,9 +1608,9 @@@
          if (ts != null) {
              props.setProperty(CURRENT_SCN_ATTRIB, ts.toString());
          }
 -        Connection conn = DriverManager.getConnection(getOldUrl(), props);
 +        Connection conn = DriverManager.getConnection(url, props);
          try {
-             conn.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS my.seq");
+             //conn.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS my.seq");
              // Insert into customer table
              PreparedStatement stmt = conn.prepareStatement(
                      "upsert into " + JOIN_CUSTOMER_TABLE_FULL_NAME +
@@@ -2092,29 -1997,90 +2106,113 @@@
          } finally {
              conn.close();
          }
 -    }  
 -    
 -    
 -    
 +    }
 +
 +    protected static void verifySequence(String tenantID, String sequenceName, String sequenceSchemaName, boolean exists) throws SQLException {
 +
 +        PhoenixConnection phxConn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
 +        String ddl = "SELECT "
 +                + PhoenixDatabaseMetaData.TENANT_ID + ","
 +                + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA + ","
 +                + PhoenixDatabaseMetaData.SEQUENCE_NAME
 +                + " FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE
 +                + " WHERE ";
 +
 +        ddl += " TENANT_ID  " + ((tenantID == null ) ? "IS NULL " : " = '" + tenantID + "'");
 +        ddl += " AND SEQUENCE_NAME " + ((sequenceName == null) ? "IS NULL " : " = '" +  sequenceName + "'");
 +        ddl += " AND SEQUENCE_SCHEMA " + ((sequenceSchemaName == null) ? "IS NULL " : " = '" + sequenceSchemaName + "'" );
 +
 +        ResultSet rs = phxConn.createStatement().executeQuery(ddl);
 +
 +        if(exists) {
 +            assertTrue(rs.next());
 +        } else {
 +            assertFalse(rs.next());
 +        }
 +        phxConn.close();
++    }        
++
++
+     //=============================================================================================
+     // Overriden assertEquals to mask EXPLAIN PLAN check.
+     // TODO to be removed later.
+     // ============================================================================================
+     
+     protected static void assertEquals(String message, String expected, String actual) {
+         if ((expected != null && expected.contains("PhoenixToEnumerableConverter"))
+                 || (actual != null && actual.contains("PhoenixToEnumerableConverter"))) {
+             return;
+         }
+         org.junit.Assert.assertEquals(message, expected, actual);
+     }
+ 
+     protected static void assertEquals(String expected, String actual) {
+         if ((expected != null && expected.contains("PhoenixToEnumerableConverter"))
+                 || (actual != null && actual.contains("PhoenixToEnumerableConverter"))) {
+             return;
+         }
+         org.junit.Assert.assertEquals(expected, actual);        
+     }
+     
+     protected static void assertEquals(String message, float expected, float actual, float delta) {
+         org.junit.Assert.assertEquals(message, expected, actual, delta);
+     }
+     
+     protected static void assertEquals(float expected, float actual, float delta) {
+         org.junit.Assert.assertEquals(expected, actual, delta);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(String message, float expected, float actual) {
+         org.junit.Assert.assertEquals(message, expected, actual);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(float expected, float actual) {
+         org.junit.Assert.assertEquals(expected, actual);
+     }
+     
+     protected static void assertEquals(String message, double expected, double actual, double delta) {
+         org.junit.Assert.assertEquals(message, expected, actual, delta);        
+     }
+     
+     protected static void assertEquals(double expected, double actual, double delta) {
+         org.junit.Assert.assertEquals(expected, actual, delta);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(String message, double expected, double actual) {
+         org.junit.Assert.assertEquals(message, expected, actual);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(double expected, double actual) {
+         org.junit.Assert.assertEquals(expected, actual);
+     }
+     
+     protected static void assertEquals(String message, long expected, long actual) {
+         org.junit.Assert.assertEquals(message, expected, actual);
+     }
+     
+     protected static void assertEquals(long expected, long actual) {
+         org.junit.Assert.assertEquals(expected, actual);
+     }
+     
+     protected static void assertEquals(String message, Object expected, Object actual) {
+         org.junit.Assert.assertEquals(message, expected, actual);
+     }
+     
+     protected static void assertEquals(Object expected, Object actual) {
+         org.junit.Assert.assertEquals(expected, actual);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(String message, Object[] expecteds, Object[] actuals) {
+         org.junit.Assert.assertEquals(message, expecteds, actuals);
+     }
+     
+     @SuppressWarnings("deprecation")
+     protected static void assertEquals(Object[] expecteds, Object[] actuals) {
+         org.junit.Assert.assertEquals(expecteds, actuals);
      }
 -}
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/faff90af/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 68e7b9b,212a7cc..8629535
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@@ -17,12 -17,10 +17,13 @@@
   */
  package org.apache.phoenix.util;
  
 +import static org.apache.phoenix.query.BaseTest.generateRandomString;
  import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 +import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY_NAME;
 +import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_NAME;
  import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
  import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+ import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_CALCITE;
  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;


[12/50] [abbrv] phoenix git commit: PHOENIX-3158 COUNT(DISTINCT) may return null instead of 0 after PHOENIX-258.

Posted by ma...@apache.org.
PHOENIX-3158 COUNT(DISTINCT) may return null instead of 0 after PHOENIX-258.


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

Branch: refs/heads/calcite
Commit: 67b8b73630676b2f75ff05eed194faf9b7e856ca
Parents: 60b95f9
Author: Lars Hofhansl <la...@apache.org>
Authored: Sat Aug 6 16:27:32 2016 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Sat Aug 6 16:29:46 2016 -0700

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java | 2 ++
 .../src/main/java/org/apache/phoenix/execute/AggregatePlan.java    | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/67b8b736/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
index 1a0e4e1..e8e9e07 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctPrefixFilterIT.java
@@ -290,6 +290,8 @@ public class DistinctPrefixFilterIT extends BaseHBaseManagedTimeTableReuseIT {
 
         testCount("SELECT %s COUNT(DISTINCT col1) FROM " + testTable, -1);
         testCount("SELECT %s COUNT(DISTINCT col2) FROM " + testTable, -1);
+
+        testCount("SELECT %s COUNT(DISTINCT prefix1) FROM " + testTable + " WHERE col1 < 0", -1);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/67b8b736/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
index 136379d..00d478a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
@@ -223,7 +223,7 @@ public class AggregatePlan extends BaseQueryPlan {
 
         AggregatingResultIterator aggResultIterator;
         // No need to merge sort for ungrouped aggregation
-        if (groupBy.isEmpty()) {
+        if (groupBy.isEmpty() || groupBy.isUngroupedAggregate()) {
             aggResultIterator = new UngroupedAggregatingResultIterator(new ConcatResultIterator(iterators), aggregators);
         // If salted or local index we still need a merge sort as we'll potentially have multiple group by keys that aren't contiguous.
         } else if (groupBy.isOrderPreserving() && !(this.getTableRef().getTable().getBucketNum() != null || this.getTableRef().getTable().getIndexType() == IndexType.LOCAL)) {


[44/50] [abbrv] phoenix git commit: PHOENIX-2645 Wildcard characters do not match newline characters

Posted by ma...@apache.org.
PHOENIX-2645 Wildcard characters do not match newline characters


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

Branch: refs/heads/calcite
Commit: ede483cf29c86c1acebc12d995966a838e4eaffb
Parents: d873c2f
Author: kliewkliew <kl...@users.noreply.github.com>
Authored: Wed Aug 24 03:06:03 2016 -0700
Committer: Thomas D'Silva <tw...@gmail.com>
Committed: Tue Aug 30 12:05:18 2016 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/LikeExpressionIT.java       | 29 ++++++++++++++++++++
 .../expression/ByteBasedLikeExpression.java     |  4 ++-
 .../expression/StringBasedLikeExpression.java   |  3 +-
 .../ByteBasedRegexpReplaceFunction.java         |  3 +-
 .../function/ByteBasedRegexpSplitFunction.java  |  3 +-
 .../function/ByteBasedRegexpSubstrFunction.java |  3 +-
 .../StringBasedRegexpReplaceFunction.java       |  3 +-
 .../StringBasedRegexpSubstrFunction.java        |  3 +-
 .../phoenix/expression/LikeExpressionTest.java  |  8 ++++++
 9 files changed, 52 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
index 6f056a7..6bfa358 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
@@ -153,4 +153,33 @@ public class LikeExpressionIT extends BaseHBaseManagedTimeTableReuseIT {
         assertEquals(null, rs.getString(2));
         assertFalse(rs.next());
     }
+
+    @Test
+    public void testNewLine() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String t = generateRandomString();
+        String ddl = "CREATE TABLE " + t + " (k VARCHAR NOT NULL PRIMARY KEY)";
+        conn.createStatement().execute(ddl);
+        conn.createStatement().execute("UPSERT INTO " + t + " VALUES('AA\nA')");
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery(
+                "SELECT * FROM " + t + " WHERE k like 'AA%'");
+        assertTrue(rs.next());
+        assertEquals("AA\nA", rs.getString(1));
+
+        rs = conn.createStatement().executeQuery(
+                "SELECT * FROM " + t + " WHERE k like 'AA_A'");
+        assertTrue(rs.next());
+        assertEquals("AA\nA", rs.getString(1));
+
+        rs = conn.createStatement().executeQuery(
+                "SELECT * FROM " + t + " WHERE k like 'AA%A'");
+        assertTrue(rs.next());
+        assertEquals("AA\nA", rs.getString(1));
+
+        rs = conn.createStatement().executeQuery(
+                "SELECT * FROM " + t + " WHERE k like 'AA_'");
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
index 4dd4f70..5b1dd2e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
@@ -18,10 +18,12 @@
 package org.apache.phoenix.expression;
 
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.expression.util.regex.JONIPattern;
 import org.apache.phoenix.parse.LikeParseNode.LikeType;
+import org.joni.Option;
 
 public class ByteBasedLikeExpression extends LikeExpression {
 
@@ -34,7 +36,7 @@ public class ByteBasedLikeExpression extends LikeExpression {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JONIPattern(value);
+        return new JONIPattern(value, Option.MULTILINE);
     }
 
     public static LikeExpression create(List<Expression> children, LikeType likeType) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
index e2afea2..10c5fd4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.expression;
 
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.expression.util.regex.JavaPattern;
@@ -34,7 +35,7 @@ public class StringBasedLikeExpression extends LikeExpression {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JavaPattern(value);
+        return new JavaPattern(value, Pattern.DOTALL);
     }
 
     public static LikeExpression create(List<Expression> children, LikeType likeType) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
index 0d6543c..c815190 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.expression.util.regex.JONIPattern;
+import org.joni.Option;
 
 public class ByteBasedRegexpReplaceFunction extends RegexpReplaceFunction {
 
@@ -34,7 +35,7 @@ public class ByteBasedRegexpReplaceFunction extends RegexpReplaceFunction {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JONIPattern(value);
+        return new JONIPattern(value, Option.MULTILINE);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
index 062713e..1a74975 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.util.regex.AbstractBaseSplitter;
 import org.apache.phoenix.expression.util.regex.JONIPattern;
+import org.joni.Option;
 
 public class ByteBasedRegexpSplitFunction extends RegexpSplitFunction {
     public ByteBasedRegexpSplitFunction() {
@@ -33,6 +34,6 @@ public class ByteBasedRegexpSplitFunction extends RegexpSplitFunction {
 
     @Override
     protected AbstractBaseSplitter compilePatternSpec(String value) {
-        return new JONIPattern(value);
+        return new JONIPattern(value, Option.MULTILINE);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
index 7ee99bf..96e5353 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.expression.util.regex.JONIPattern;
+import org.joni.Option;
 
 public class ByteBasedRegexpSubstrFunction extends RegexpSubstrFunction {
     public ByteBasedRegexpSubstrFunction() {
@@ -33,6 +34,6 @@ public class ByteBasedRegexpSubstrFunction extends RegexpSubstrFunction {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JONIPattern(value);
+        return new JONIPattern(value, Option.MULTILINE);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
index 9aaec70..1e4943f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.expression.function;
 
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
@@ -34,7 +35,7 @@ public class StringBasedRegexpReplaceFunction extends RegexpReplaceFunction {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JavaPattern(value);
+        return new JavaPattern(value, Pattern.DOTALL);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
index 253db36..5e82a32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.expression.function;
 
 import java.util.List;
+import java.util.regex.Pattern;
 
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
@@ -33,6 +34,6 @@ public class StringBasedRegexpSubstrFunction extends RegexpSubstrFunction {
 
     @Override
     protected AbstractBasePattern compilePatternSpec(String value) {
-        return new JavaPattern(value);
+        return new JavaPattern(value, Pattern.DOTALL);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede483cf/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
index 0bf8b06..580ac8e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/LikeExpressionTest.java
@@ -81,4 +81,12 @@ public class LikeExpressionTest {
         assertEquals(Boolean.TRUE, testExpression ("", "%"));
         assertEquals(Boolean.FALSE, testExpression ("", "_"));
     }
+
+    @Test
+    public void testNewline() throws Exception {
+        assertEquals(Boolean.TRUE, testExpression ("AA\nA", "AA%"));
+        assertEquals(Boolean.TRUE, testExpression ("AA\nA", "AA_A"));
+        assertEquals(Boolean.TRUE, testExpression ("AA\nA", "AA%A"));
+        assertEquals(Boolean.FALSE, testExpression ("AA\nA", "AA_"));
+    }
  }


[34/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
index 31994bb..1021bdd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
@@ -34,36 +34,40 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
 
 
-public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
+public class SkipScanQueryIT extends BaseHBaseManagedTimeTableReuseIT {
     
-    private void initIntInTable(Connection conn, List<Integer> data) throws SQLException {
-        String ddl = "CREATE TABLE IF NOT EXISTS inTest (" + 
+    private String initIntInTable(Connection conn, List<Integer> data) throws SQLException {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName + " (" + 
                      "  i INTEGER NOT NULL PRIMARY KEY)";
         conn.createStatement().executeUpdate(ddl);
         
         // Test upsert correct values 
-        String query = "UPSERT INTO inTest VALUES(?)";
+        String query = "UPSERT INTO " + tableName + " VALUES(?)";
         PreparedStatement stmt = conn.prepareStatement(query);
         for (Integer i : data) {
             stmt.setInt(1, i);
             stmt.execute();
         }
         conn.commit();
+        return tableName;
     }
     
-    private void initVarCharCrossProductInTable(Connection conn, List<String> c1, List<String> c2) throws SQLException {
-        String ddl = "CREATE TABLE IF NOT EXISTS inVarTest (" + 
+    private String initVarCharCrossProductInTable(Connection conn, List<String> c1, List<String> c2) throws SQLException {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName + " (" +
                      "  s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (s1,s2))";
         conn.createStatement().executeUpdate(ddl);
         
         // Test upsert correct values 
-        String query = "UPSERT INTO inVarTest VALUES(?,?)";
+        String query = "UPSERT INTO " + tableName + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(query);
         for (String s1 : c1) {
             for (String s2 : c2) {
@@ -73,15 +77,17 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             }
         }
         conn.commit();
+        return tableName;
     }
     
-    private void initVarCharParallelListInTable(Connection conn, List<String> c1, List<String> c2) throws SQLException {
-        String ddl = "CREATE TABLE IF NOT EXISTS inVarTest (" + 
+    private String initVarCharParallelListInTable(Connection conn, List<String> c1, List<String> c2) throws SQLException {
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName + " (" + 
                      "  s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (s1,s2))";
         conn.createStatement().executeUpdate(ddl);
         
         // Test upsert correct values 
-        String query = "UPSERT INTO inVarTest VALUES(?,?)";
+        String query = "UPSERT INTO " + tableName + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(query);
         for (int i = 0; i < c1.size(); i++) {
             stmt.setString(1, c1.get(i));
@@ -89,13 +95,15 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
         }
         conn.commit();
+        return tableName;
     }
     
-    private static final String UPSERT_SELECT_AFTER_UPSERT_STATEMENTS = 
-    		"upsert into table1(c1, c2, c3, c4, v1, v2) values('1001', '91', 's1', '2013-09-26', 28397, 23541);\n" + 
-    		"upsert into table1(c1, c2, c3, c4, v1, v2) values('1001', '91', 's2', '2013-09-23', 3369, null);\n";
-    private void initSelectAfterUpsertTable(Connection conn) throws Exception {
-        String ddl = "create table if not exists table1("
+    private static String UPSERT_SELECT_AFTER_UPSERT_STATEMENTS =
+    		"upsert into %s(c1, c2, c3, c4, v1, v2) values('1001', '91', 's1', '2013-09-26', 28397, 23541);\n" +
+    		"upsert into %s(c1, c2, c3, c4, v1, v2) values('1001', '91', 's2', '2013-09-23', 3369, null);\n";
+    private String initSelectAfterUpsertTable(Connection conn) throws Exception {
+        String tableName = generateRandomString();
+        String ddl = "create table if not exists  " + tableName + " ("
                 + "c1 VARCHAR NOT NULL," + "c2 VARCHAR NOT NULL,"
                 + "c3 VARCHAR NOT NULL," + "c4 VARCHAR NOT NULL,"
                 + "v1 integer," + "v2 integer "
@@ -103,18 +111,20 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
         conn.createStatement().execute(ddl);
 
         // Test upsert correct values
-        StringReader reader = new StringReader(UPSERT_SELECT_AFTER_UPSERT_STATEMENTS);
+        StringReader reader = new StringReader(String.format(UPSERT_SELECT_AFTER_UPSERT_STATEMENTS, tableName, tableName));
         PhoenixRuntime.executeStatements(conn, reader, Collections.emptyList());
         reader.close();
         conn.commit();
+        return tableName;
     }
     
     @Test
     public void testSkipScanFilterQuery() throws Exception {
-        String createTableDDL = "CREATE TABLE test" + "(col1 VARCHAR," + "col2 VARCHAR," + "col3 VARCHAR,"
+        String tableName = generateRandomString();
+        String createTableDDL = "CREATE TABLE " + tableName + "(col1 VARCHAR," + "col2 VARCHAR," + "col3 VARCHAR,"
              + "col4 VARCHAR," + "CONSTRAINT pk  " + "PRIMARY KEY (col1,col2,col3,col4))";
-        String upsertQuery = "upsert into test values(?,?,?,?)";
-        String query = "SELECT col1, col2, col3, col4 FROM test WHERE col1 IN ('a','e','f') AND col2 = 'b' AND col4 = '1' ";
+        String upsertQuery = "upsert into  " + tableName + "  values(?,?,?,?)";
+        String query = "SELECT col1, col2, col3, col4 FROM " + tableName + " WHERE col1 IN ('a','e','f') AND col2 = 'b' AND col4 = '1' ";
         String[] col1Values = { "a", "e.f", "f" };
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         createTestTable(getUrl(), createTableDDL);
@@ -143,11 +153,11 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectAfterUpsertInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initSelectAfterUpsertTable(conn);
+        String tableName = initSelectAfterUpsertTable(conn);
         try {
             String query;
-            query = "SELECT case when sum(v2)*1.0/sum(v1) is null then 0 else sum(v2)*1.0/sum(v1) END AS val FROM table1 " +
-            		"WHERE c1='1001' AND c2 = '91' " +
+            query = "SELECT case when sum(v2)*1.0/sum(v1) is null then 0 else sum(v2)*1.0/sum(v1) END AS val FROM  " + tableName +
+            		" WHERE c1='1001' AND c2 = '91' " +
             		"AND c3 IN ('s1','s2') AND c4='2013-09-24'";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
@@ -160,10 +170,10 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     public void testInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(false);
-        initIntInTable(conn,Arrays.asList(2,7,10));
+        String tableName = initIntInTable(conn,Arrays.asList(2,7,10));
         try {
             String query;
-            query = "SELECT i FROM inTest WHERE i IN (1,2,4,5,7,8,10)";
+            query = "SELECT i FROM " + tableName + " WHERE i IN (1,2,4,5,7,8,10)";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -181,10 +191,10 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     public void testVarCharParallelListInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(false);
-        initVarCharParallelListInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
+        String tableName = initVarCharParallelListInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
         try {
             String query;
-            query = "SELECT s1,s2 FROM inVarTest WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z')";
+            query = "SELECT s1,s2 FROM " + tableName + " WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z')";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("da", rs.getString(1));
@@ -199,10 +209,10 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     public void testVarCharXInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(false);
-        initVarCharCrossProductInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
+        String tableName = initVarCharCrossProductInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
         try {
             String query;
-            query = "SELECT s1,s2 FROM inVarTest WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z')";
+            query = "SELECT s1,s2 FROM " + tableName + " WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z')";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("da", rs.getString(1));
@@ -226,11 +236,11 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     public void testVarCharXIntInQuery() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         conn.setAutoCommit(false);
-        initVarCharCrossProductInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
+        String tableName = initVarCharCrossProductInTable(conn,Arrays.asList("d","da","db"),Arrays.asList("m","mc","tt"));
         try {
             String query;
-            query = "SELECT s1,s2 FROM inVarTest " +
-                    "WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z') " +
+            query = "SELECT s1,s2 FROM " + tableName +
+                    " WHERE s1 IN ('a','b','da','db') AND s2 IN ('c','ma','m','mc','ttt','z') " +
                     "AND s1 > 'd' AND s1 < 'db' AND s2 > 'm'";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
@@ -244,17 +254,18 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testPreSplitCompositeFixedKey() throws Exception {
+        String tableName = generateRandomString();
         Connection conn = DriverManager.getConnection(getUrl());
         try {
-            conn.createStatement().execute("create table test(key_1 char(3) not null, key_2 char(4) not null, v varchar(8)  CONSTRAINT pk PRIMARY KEY (key_1,key_2)) split on('000','100','200')");
+            conn.createStatement().execute("create table " + tableName + "(key_1 char(3) not null, key_2 char(4) not null, v varchar(8)  CONSTRAINT pk PRIMARY KEY (key_1,key_2)) split on('000','100','200')");
             conn.setAutoCommit(true);
-            conn.createStatement().execute("upsert into test values('000','aaaa','value_1')");
-            conn.createStatement().execute("upsert into test values('000','aabb','value_2')");
-            conn.createStatement().execute("upsert into test values('100','aacc','value_3')");
-            conn.createStatement().execute("upsert into test values('100','aadd','value_4')");
-            conn.createStatement().execute("upsert into test values('200','aaee','value_5')");
-            conn.createStatement().execute("upsert into test values('201','aaff','value_6')");
-            ResultSet rs = conn.createStatement().executeQuery("select * from test where key_1>='000' and key_1<'200' and key_2>='aabb' and key_2<'aadd'");
+            conn.createStatement().execute("upsert into " + tableName + " values('000','aaaa','value_1')");
+            conn.createStatement().execute("upsert into " + tableName + " values('000','aabb','value_2')");
+            conn.createStatement().execute("upsert into " + tableName + " values('100','aacc','value_3')");
+            conn.createStatement().execute("upsert into " + tableName + " values('100','aadd','value_4')");
+            conn.createStatement().execute("upsert into " + tableName + " values('200','aaee','value_5')");
+            conn.createStatement().execute("upsert into " + tableName + " values('201','aaff','value_6')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName + " where key_1>='000' and key_1<'200' and key_2>='aabb' and key_2<'aadd'");
             assertTrue(rs.next());
             assertEquals("000", rs.getString(1));
             assertEquals("aabb", rs.getString(2));
@@ -273,16 +284,17 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testInWithDescKey() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
         try {
-            conn.createStatement().execute("create table test(key_1 char(3) not null, key_2 char(4) not null, v varchar(8)  CONSTRAINT pk PRIMARY KEY (key_1,key_2 desc))");
+            conn.createStatement().execute("create table " + tableName + "(key_1 char(3) not null, key_2 char(4) not null, v varchar(8)  CONSTRAINT pk PRIMARY KEY (key_1,key_2 desc))");
             conn.setAutoCommit(true);
-            conn.createStatement().execute("upsert into test values('000','aaaa','value_1')");
-            conn.createStatement().execute("upsert into test values('000','aabb','value_2')");
-            conn.createStatement().execute("upsert into test values('100','aacc','value_3')");
-            conn.createStatement().execute("upsert into test values('100','aadd','value_4')");
-            conn.createStatement().execute("upsert into test values('200','aaee','value_5')");
-            conn.createStatement().execute("upsert into test values('201','aaff','value_6')");
-            ResultSet rs = conn.createStatement().executeQuery("select * from test where key_1>='000' and key_1<'200' and key_2>='aabb' and key_2<'aadd'");
+            conn.createStatement().execute("upsert into " + tableName + " values('000','aaaa','value_1')");
+            conn.createStatement().execute("upsert into " + tableName + " values('000','aabb','value_2')");
+            conn.createStatement().execute("upsert into " + tableName + " values('100','aacc','value_3')");
+            conn.createStatement().execute("upsert into " + tableName + " values('100','aadd','value_4')");
+            conn.createStatement().execute("upsert into " + tableName + " values('200','aaee','value_5')");
+            conn.createStatement().execute("upsert into " + tableName + " values('201','aaff','value_6')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName + " where key_1>='000' and key_1<'200' and key_2>='aabb' and key_2<'aadd'");
             assertTrue(rs.next());
             assertEquals("000", rs.getString(1));
             assertEquals("aabb", rs.getString(2));
@@ -293,7 +305,7 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             assertEquals("value_3", rs.getString(3));
             assertFalse(rs.next());
 
-            rs = conn.createStatement().executeQuery("select * from test where (key_1,key_2) in (('100','aacc'),('100','aadd'))");
+            rs = conn.createStatement().executeQuery("select * from " + tableName + " where (key_1,key_2) in (('100','aacc'),('100','aadd'))");
             assertTrue(rs.next());
             assertEquals("100", rs.getString(1));
             assertEquals("aadd", rs.getString(2));
@@ -312,8 +324,8 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSkipScanIntersectionAtEnd() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        
-        PreparedStatement stmt = conn.prepareStatement("create table splits_test "
+        String tableName = generateRandomString();
+        PreparedStatement stmt = conn.prepareStatement("create table " + tableName 
             + "(pk1 UNSIGNED_TINYINT NOT NULL, pk2 UNSIGNED_TINYINT NOT NULL, pk3 UNSIGNED_TINYINT NOT NULL, kv VARCHAR "
             + "CONSTRAINT pk PRIMARY KEY (pk1, pk2, pk3)) SPLIT ON (?, ?, ?)");
         stmt.setBytes(1, new byte[] {1, 1});
@@ -321,13 +333,13 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
         stmt.setBytes(3, new byte[] {3, 1});
         stmt.execute();
         
-        conn.createStatement().execute("upsert into splits_test values (0, 1, 1, 'a')");
-        conn.createStatement().execute("upsert into splits_test values (1, 1, 1, 'a')");
-        conn.createStatement().execute("upsert into splits_test values (2, 1, 1, 'a')");
-        conn.createStatement().execute("upsert into splits_test values (3, 1, 1, 'a')");
+        conn.createStatement().execute("upsert into " + tableName + " values (0, 1, 1, 'a')");
+        conn.createStatement().execute("upsert into " + tableName + " values (1, 1, 1, 'a')");
+        conn.createStatement().execute("upsert into " + tableName + " values (2, 1, 1, 'a')");
+        conn.createStatement().execute("upsert into " + tableName + " values (3, 1, 1, 'a')");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("select count(kv) from splits_test where pk1 in (0, 1, 2, 3) AND pk2 = 1");
+        ResultSet rs = conn.createStatement().executeQuery("select count(kv) from " + tableName + " where pk1 in (0, 1, 2, 3) AND pk2 = 1");
         assertTrue(rs.next());
         assertEquals(4, rs.getInt(1));
         assertFalse(rs.next());
@@ -338,10 +350,12 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String tableName = generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try {
-            createMultiCFTestTable(TestUtil.DEFAULT_DATA_TABLE_FULL_NAME, null);
-            populateMultiCFTestTable(TestUtil.DEFAULT_DATA_TABLE_FULL_NAME);
-            String upsert = "UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME
+            createMultiCFTestTable(fullTableName , null);
+            populateMultiCFTestTable(fullTableName);
+            String upsert = "UPSERT INTO " + fullTableName
                     + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
             PreparedStatement stmt = conn.prepareStatement(upsert);
             stmt.setString(1, "varchar4");
@@ -398,7 +412,7 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             stmt.setDate(16, null);
             stmt.executeUpdate();
             conn.commit();
-            String query = "SELECT char_col1, int_col1, long_col2 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " where varchar_pk in ('varchar3','varchar6')";
+            String query = "SELECT char_col1, int_col1, long_col2 from " + fullTableName + " where varchar_pk in ('varchar3','varchar6')";
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("chara", rs.getString(1));
@@ -418,13 +432,14 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testOrPKWithAndNonPK() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
         try {
-            conn.createStatement().execute("create table bugTable(ID varchar primary key,company varchar)");
+            conn.createStatement().execute("create table " + tableName + "(ID varchar primary key,company varchar)");
             conn.setAutoCommit(true);
-            conn.createStatement().execute("upsert into bugTable values('i1','c1')");
-            conn.createStatement().execute("upsert into bugTable values('i2','c2')");
-            conn.createStatement().execute("upsert into bugTable values('i3','c3')");
-            ResultSet rs = conn.createStatement().executeQuery("select * from bugTable where ID = 'i1' or (ID = 'i2' and company = 'c3')");
+            conn.createStatement().execute("upsert into " + tableName + " values('i1','c1')");
+            conn.createStatement().execute("upsert into " + tableName + " values('i2','c2')");
+            conn.createStatement().execute("upsert into " + tableName + " values('i3','c3')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName + " where ID = 'i1' or (ID = 'i2' and company = 'c3')");
             assertTrue(rs.next());
             assertEquals("i1", rs.getString(1));
             assertEquals("c1", rs.getString(2));
@@ -438,9 +453,10 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testNullInfiniteLoop() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String tableName = generateRandomString();
             conn.setAutoCommit(true);
             conn.createStatement().execute(
-              "create table NULL_TEST"+
+              "create table " + tableName +
               "("+
                      "CREATETIME VARCHAR,"+
                      "ACCOUNTID VARCHAR,"+
@@ -449,9 +465,9 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
                      "CONSTRAINT pk PRIMARY KEY(CREATETIME,ACCOUNTID,SERVICENAME)"+
               ")");
 
-            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,SERVICENAME,SPAN.APPID) values('20160116141006','servlet','android')");
-            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,ACCOUNTID,SERVICENAME,SPAN.APPID) values('20160116151006','2404787','jdbc','ios')");
-            ResultSet rs = conn.createStatement().executeQuery("select * from NULL_TEST where CREATETIME>='20160116121006' and  CREATETIME<='20160116181006' and ACCOUNTID='2404787'");
+            conn.createStatement().execute("upsert into " + tableName + "(CREATETIME,SERVICENAME,SPAN.APPID) values('20160116141006','servlet','android')");
+            conn.createStatement().execute("upsert into " + tableName + "(CREATETIME,ACCOUNTID,SERVICENAME,SPAN.APPID) values('20160116151006','2404787','jdbc','ios')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName + " where CREATETIME>='20160116121006' and  CREATETIME<='20160116181006' and ACCOUNTID='2404787'");
             assertTrue(rs.next());
             assertFalse(rs.next());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index 8ede6ae..f1be09f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -39,10 +39,10 @@ import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
+public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeTableReuseIT {
     
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Forces server cache to be used
@@ -53,8 +53,8 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testJoinOverSaltedTables() throws Exception {
-        String tempTableNoSalting = "TEMP_TABLE_NO_SALTING";
-        String tempTableWithSalting = "TEMP_TABLE_WITH_SALTING";
+        String tempTableNoSalting = "TEMP_TABLE_NO_SALTING"  + generateRandomString();
+        String tempTableWithSalting = "TEMP_TABLE_WITH_SALTING" + generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -191,8 +191,8 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testJoinOnDynamicColumns() throws Exception {
-        String tableA = "tableA";
-        String tableB = "tableB";
+        String tableA =  generateRandomString();
+        String tableB =  generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = null;
         PreparedStatement stmt = null;
@@ -210,7 +210,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             stmt.close();
 
-            String upsertA = "UPSERT INTO TABLEA (pkA, colA1, colA2) VALUES(?, ?, ?)";
+            String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
             stmt = conn.prepareStatement(upsertA);
             int i = 0;
             for (i = 0; i < 5; i++) {
@@ -221,20 +221,21 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
             }
             conn.commit();
             stmt.close();
+            String seqBName = generateRandomString();
 
             // upsert select dynamic columns in tableB
-            conn.createStatement().execute("CREATE SEQUENCE SEQB");
-            String upsertBSelectA = "UPSERT INTO TABLEB (pkB, pkA INTEGER)"
-                    + "SELECT NEXT VALUE FOR SEQB, pkA FROM TABLEA";
+            conn.createStatement().execute("CREATE SEQUENCE " + seqBName);
+            String upsertBSelectA = "UPSERT INTO " + tableB + " (pkB, pkA INTEGER)"
+                    + "SELECT NEXT VALUE FOR " + seqBName + ", pkA FROM " + tableA ;
             stmt = conn.prepareStatement(upsertBSelectA);
             stmt.executeUpdate();
             stmt.close();
             conn.commit();
-            conn.createStatement().execute("DROP SEQUENCE SEQB");
+            conn.createStatement().execute("DROP SEQUENCE " + seqBName );
 
             // perform a join between tableB and tableA by joining on the dynamic column that we upserted in
             // tableB. This join should return all the rows from table A.
-            String joinSql = "SELECT /*+ USE_SORT_MERGE_JOIN*/ A.pkA, A.COLA1, A.colA2 FROM TABLEB B(pkA INTEGER) JOIN TABLEA A ON a.pkA = b.pkA";
+            String joinSql = "SELECT /*+ USE_SORT_MERGE_JOIN*/ A.pkA, A.COLA1, A.colA2 FROM " + tableB + " B(pkA INTEGER) JOIN " + tableA + " A ON a.pkA = b.pkA";
             stmt = conn.prepareStatement(joinSql);
             ResultSet rs = stmt.executeQuery();
             i = 0;
@@ -263,9 +264,13 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
+        String gTableName = generateRandomString();
+        String lTableName = generateRandomString();
+        String slTableName = generateRandomString();
+
 
         try {
-            String GRAMMAR_TABLE = "CREATE TABLE IF NOT EXISTS GRAMMAR_TABLE (ID INTEGER PRIMARY KEY, " +
+            String GRAMMAR_TABLE = "CREATE TABLE IF NOT EXISTS " + gTableName + "  (ID INTEGER PRIMARY KEY, " +
                     "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
                     "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
                     "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
@@ -273,7 +278,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                     "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
                     "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
 
-            String LARGE_TABLE = "CREATE TABLE IF NOT EXISTS LARGE_TABLE (ID INTEGER PRIMARY KEY, " +
+            String LARGE_TABLE = "CREATE TABLE IF NOT EXISTS " + lTableName + " (ID INTEGER PRIMARY KEY, " +
                     "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
                     "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
                     "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
@@ -281,7 +286,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                     "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
                     "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
 
-            String SECONDARY_LARGE_TABLE = "CREATE TABLE IF NOT EXISTS SECONDARY_LARGE_TABLE (SEC_ID INTEGER PRIMARY KEY," +
+            String SECONDARY_LARGE_TABLE = "CREATE TABLE IF NOT EXISTS " + slTableName + " (SEC_ID INTEGER PRIMARY KEY," +
                     "sec_unsig_id UNSIGNED_INT, sec_big_id BIGINT, sec_usnig_long_id UNSIGNED_LONG, sec_tiny_id TINYINT," + 
                     "sec_unsig_tiny_id UNSIGNED_TINYINT, sec_small_id SMALLINT, sec_unsig_small_id UNSIGNED_SMALLINT," + 
                     "sec_float_id FLOAT, sec_unsig_float_id UNSIGNED_FLOAT, sec_double_id DOUBLE, sec_unsig_double_id UNSIGNED_DOUBLE," +
@@ -292,22 +297,22 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
             createTestTable(getUrl(), LARGE_TABLE);
             createTestTable(getUrl(), SECONDARY_LARGE_TABLE);
 
-            String ddl = "SELECT /*+USE_SORT_MERGE_JOIN*/ * FROM (SELECT ID, BIG_ID, DATE_ID FROM LARGE_TABLE AS A WHERE (A.ID % 5) = 0) AS A " +
-                    "INNER JOIN (SELECT SEC_ID, SEC_TINY_ID, SEC_UNSIG_FLOAT_ID FROM SECONDARY_LARGE_TABLE AS B WHERE (B.SEC_ID % 5) = 0) AS B " +     
-                    "ON A.ID=B.SEC_ID WHERE A.DATE_ID > ALL (SELECT SEC_DATE_ID FROM SECONDARY_LARGE_TABLE LIMIT 100) " +      
-                    "AND B.SEC_UNSIG_FLOAT_ID = ANY (SELECT sec_unsig_float_id FROM SECONDARY_LARGE_TABLE " +                                       
-                    "WHERE SEC_ID > ALL (SELECT MIN (ID) FROM GRAMMAR_TABLE WHERE UNSIG_ID IS NULL) AND " +
-                    "SEC_UNSIG_ID < ANY (SELECT DISTINCT(UNSIG_ID) FROM LARGE_TABLE WHERE UNSIG_ID<2500) LIMIT 1000) " +
+            String ddl = "SELECT /*+USE_SORT_MERGE_JOIN*/ * FROM (SELECT ID, BIG_ID, DATE_ID FROM " + lTableName + " AS A WHERE (A.ID % 5) = 0) AS A " +
+                    "INNER JOIN (SELECT SEC_ID, SEC_TINY_ID, SEC_UNSIG_FLOAT_ID FROM " + slTableName + " AS B WHERE (B.SEC_ID % 5) = 0) AS B " +
+                    "ON A.ID=B.SEC_ID WHERE A.DATE_ID > ALL (SELECT SEC_DATE_ID FROM " + slTableName + " LIMIT 100) " +
+                    "AND B.SEC_UNSIG_FLOAT_ID = ANY (SELECT sec_unsig_float_id FROM " + slTableName +
+                    " WHERE SEC_ID > ALL (SELECT MIN (ID) FROM " + gTableName + "  WHERE UNSIG_ID IS NULL) AND " +
+                    "SEC_UNSIG_ID < ANY (SELECT DISTINCT(UNSIG_ID) FROM " + lTableName + " WHERE UNSIG_ID<2500) LIMIT 1000) " +
                     "AND A.ID < 10000";
             ResultSet rs = conn.createStatement().executeQuery(ddl);
             assertFalse(rs.next());  
         } finally {
             Statement statement = conn.createStatement();
-            String query = "drop table GRAMMAR_TABLE";
+            String query = "drop table " + gTableName;
             statement.executeUpdate(query);
-            query = "drop table LARGE_TABLE";
+            query = "drop table " + lTableName ;
             statement.executeUpdate(query);
-            query = "drop table SECONDARY_LARGE_TABLE";
+            query = "drop table " + slTableName ;
             statement.executeUpdate(query);
             conn.close();
         }
@@ -318,9 +323,10 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
+        String eventCountTableName = generateRandomString();
         try {
             conn.createStatement().execute(
-                    "CREATE TABLE IF NOT EXISTS EVENT_COUNT (\n" +
+                    "CREATE TABLE IF NOT EXISTS " + eventCountTableName + " (\n" +
                     "        BUCKET VARCHAR,\n" +
                     "        TIMESTAMP_DATE TIMESTAMP,\n" +
                     "        TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
@@ -332,7 +338,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                     "        E FLOAT\n" +
                     "    CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP DESC, LOCATION, A, B, C)\n" +
                     ") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO EVENT_COUNT(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + eventCountTableName + "(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
             stmt.setString(1, "5SEC");
             stmt.setString(3, "Tr/Bal");
             stmt.setString(4, "A1");
@@ -378,7 +384,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
             // We'll test the original version of the user table as well as a slightly modified
             // version, in order to verify that sort-merge join works for columns both having
             // DESC sort order as well as one having ASC order and the other having DESC order.
-            String[] t = new String[] {"EVENT_LATENCY", "EVENT_LATENCY_2"};
+            String[] t = new String[] {"EVENT_LATENCY" + generateRandomString(), "EVENT_LATENCY_2" + generateRandomString()};
             for (int i = 0; i < 2; i++) {
                 conn.createStatement().execute(
                         "CREATE TABLE IF NOT EXISTS " + t[i] + " (\n" +
@@ -425,7 +431,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                 String q =
                         "SELECT C.BUCKET, C.TIMESTAMP FROM (\n" +
                         "     SELECT E.BUCKET as BUCKET, L.BUCKET as LBUCKET, E.TIMESTAMP as TIMESTAMP, L.TIMESTAMP as LTIMESTAMP FROM\n" +
-                        "        (SELECT BUCKET, TIMESTAMP FROM EVENT_COUNT\n" +
+                        "        (SELECT BUCKET, TIMESTAMP FROM " + eventCountTableName + "\n" +
                         "             WHERE BUCKET = '5SEC' AND LOCATION = 'Tr/Bal'\n" +
                         "                 AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
                         "             GROUP BY BUCKET, TIMESTAMP, LOCATION\n" +
@@ -442,7 +448,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                 
                 String p = i == 0 ?
                         "SORT-MERGE-JOIN (INNER) TABLES\n" +
-                        "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
+                        "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + eventCountTableName + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
                         "        SERVER FILTER BY FIRST KEY ONLY\n" +
                         "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, LOCATION]\n" +
                         "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, LOCATION]\n" +
@@ -459,7 +465,7 @@ public class SortMergeJoinMoreIT extends BaseHBaseManagedTimeIT {
                         "CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]"
                         :
                         "SORT-MERGE-JOIN (INNER) TABLES\n" +
-                        "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
+                        "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + eventCountTableName + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
                         "        SERVER FILTER BY FIRST KEY ONLY\n" +
                         "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, LOCATION]\n" +
                         "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, LOCATION]\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
index 70d5f1d..51dcd21 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
@@ -74,7 +74,7 @@ public class SpillableGroupByIT extends BaseOwnClusterHBaseManagedTimeIT {
 
     private long createTable() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), GROUPBYTEST_NAME, null, ts - 2);
+        ensureTableCreated(getUrl(), GROUPBYTEST_NAME, GROUPBYTEST_NAME, null, ts - 2);
         return ts;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledTmpFileDeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledTmpFileDeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledTmpFileDeleteIT.java
index e23378e..728aa6f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledTmpFileDeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledTmpFileDeleteIT.java
@@ -38,11 +38,12 @@ import com.google.common.io.Files;
 
 
 
-public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeIT {
+public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeTableReuseIT {
 	
     private Connection conn = null;
     private Properties props = null;
     private File spoolDir;
+	private String tableName;
 
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
@@ -56,16 +57,17 @@ public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeIT {
 	
 	@Before 
 	public void setup() throws SQLException {
+		tableName = generateRandomString();
 		props = new Properties();
 		spoolDir =  Files.createTempDir();
 		props.put(QueryServices.SPOOL_DIRECTORY, spoolDir.getPath());
         props.setProperty(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(1));
         conn = DriverManager.getConnection(getUrl(), props);
 		Statement stmt = conn.createStatement();
-		stmt.execute("CREATE TABLE test (ID varchar NOT NULL PRIMARY KEY) SPLIT ON ('EA','EZ')");
-		stmt.execute("UPSERT INTO test VALUES ('AA')");
-		stmt.execute("UPSERT INTO test VALUES ('EB')");    
-		stmt.execute("UPSERT INTO test VALUES ('FA')");    
+		stmt.execute("CREATE TABLE " + tableName + " (ID varchar NOT NULL PRIMARY KEY) SPLIT ON ('EA','EZ')");
+		stmt.execute("UPSERT INTO " + tableName + " VALUES ('AA')");
+		stmt.execute("UPSERT INTO " + tableName + " VALUES ('EB')");
+		stmt.execute("UPSERT INTO " + tableName + " VALUES ('FA')");
 		stmt.close();
 		conn.commit();
 	}
@@ -98,7 +100,7 @@ public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeIT {
 			file.delete();
 		}
 
-		String query = "select * from TEST";
+		String query = "select * from " + tableName + "";
 		Statement statement = conn.createStatement();
 		ResultSet rs = statement.executeQuery(query);
 		assertTrue(rs.next());
@@ -109,7 +111,7 @@ public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeIT {
 			fileNames.add(file.getName());
 		}
 
-		String preparedQuery = "select * from test where id = ?";
+		String preparedQuery = "select * from " + tableName + " where id = ?";
 		PreparedStatement pstmt = conn.prepareStatement(preparedQuery);
 		pstmt.setString(1, "EB");
 		ResultSet prs = pstmt.executeQuery(preparedQuery);
@@ -121,14 +123,14 @@ public class SpooledTmpFileDeleteIT extends BaseHBaseManagedTimeIT {
 		}
 
 		Connection conn2 = DriverManager.getConnection(getUrl(), props);
-		String query2 = "select * from TEST";
+		String query2 = "select * from " + tableName + "";
 		Statement statement2 = conn2.createStatement();
 		ResultSet rs2 = statement2.executeQuery(query2);
 		assertTrue(rs2.next());
 		files = dir.listFiles(fnameFilter);
 		assertTrue(files.length > 0);
 
-		String preparedQuery2 = "select * from test where id = ?";
+		String preparedQuery2 = "select * from " + tableName + " where id = ?";
 		PreparedStatement pstmt2 = conn2.prepareStatement(preparedQuery2);
 		pstmt2.setString(1, "EB");
 		ResultSet prs2 = pstmt2.executeQuery(preparedQuery2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SqrtFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SqrtFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SqrtFunctionEnd2EndIT.java
index 50fdd4f..877b46d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SqrtFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SqrtFunctionEnd2EndIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.closeStmtAndConn;
+import static org.apache.phoenix.util.TestUtil.getTableName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -33,21 +34,25 @@ import org.junit.Test;
 /**
  * End to end tests for {@link SqrtFunction}
  */
-public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
+public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final String KEY = "key";
     private static final double ZERO = 1e-8;
-
+    String testUnsignedTable;
+    String testSignedTable;
+    
     @Before
     public void initTable() throws Exception {
         Connection conn = null;
         PreparedStatement stmt = null;
+        testUnsignedTable = generateRandomString();
+        testSignedTable = generateRandomString();
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE testSigned (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
+            ddl = "CREATE TABLE " + testSignedTable + " (k VARCHAR NOT NULL PRIMARY KEY, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
             conn.createStatement().execute(ddl);
-            ddl = "CREATE TABLE testUnsigned (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
+            ddl = "CREATE TABLE " + testUnsignedTable + " (k VARCHAR NOT NULL PRIMARY KEY, doub UNSIGNED_DOUBLE, fl UNSIGNED_FLOAT, inte UNSIGNED_INT, lon UNSIGNED_LONG, smalli UNSIGNED_SMALLINT, tinyi UNSIGNED_TINYINT)";
             conn.createStatement().execute(ddl);
             conn.commit();
         } finally {
@@ -56,7 +61,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     }
 
     private void updateSignedTable(Connection conn, double data) throws Exception {
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO testSigned VALUES (?, ?, ?, ?, ?, ?, ?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + testSignedTable + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -70,7 +75,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
     }
 
     private void updateUnsignedTable(Connection conn, double data) throws Exception {
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO testUnsigned VALUES (?, ?, ?, ?, ?, ?, ?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + testUnsignedTable + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -85,7 +90,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
 
     private void testSignedNumberSpec(Connection conn, double data) throws Exception {
         updateSignedTable(conn, data);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT SQRT(doub),SQRT(fl),SQRT(inte),SQRT(lon),SQRT(smalli),SQRT(tinyi) FROM testSigned");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT SQRT(doub),SQRT(fl),SQRT(inte),SQRT(lon),SQRT(smalli),SQRT(tinyi) FROM " + testSignedTable );
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.sqrt(d.doubleValue())) < ZERO);
@@ -95,7 +100,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
         assertTrue(Math.abs(rs.getDouble(5) - Math.sqrt(d.shortValue())) < ZERO);
         assertTrue(Math.abs(rs.getDouble(6) - Math.sqrt(d.byteValue())) < ZERO);
         assertTrue(!rs.next());
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM testSigned WHERE SQRT(doub)>0 AND SQRT(fl)>0 AND SQRT(inte)>0 AND SQRT(lon)>0 AND SQRT(smalli)>0 AND SQRT(tinyi)>0");
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + testSignedTable + " WHERE SQRT(doub)>0 AND SQRT(fl)>0 AND SQRT(inte)>0 AND SQRT(lon)>0 AND SQRT(smalli)>0 AND SQRT(tinyi)>0");
         rs = stmt.executeQuery();
         if (data > 0) {
             assertTrue(rs.next());
@@ -106,7 +111,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
 
     private void testUnsignedNumberSpec(Connection conn, double data) throws Exception {
         updateUnsignedTable(conn, data);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT SQRT(doub),SQRT(fl),SQRT(inte),SQRT(lon),SQRT(smalli),SQRT(tinyi) FROM testUnsigned");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT SQRT(doub),SQRT(fl),SQRT(inte),SQRT(lon),SQRT(smalli),SQRT(tinyi) FROM " + testUnsignedTable );
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.sqrt(d.doubleValue())) < ZERO);
@@ -116,7 +121,7 @@ public class SqrtFunctionEnd2EndIT extends BaseHBaseManagedTimeIT {
         assertTrue(Math.abs(rs.getDouble(5) - Math.sqrt(d.shortValue())) < ZERO);
         assertTrue(Math.abs(rs.getDouble(6) - Math.sqrt(d.byteValue())) < ZERO);
         assertTrue(!rs.next());
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM testUnsigned WHERE SQRT(doub)>0 AND SQRT(fl)>0 AND SQRT(inte)>0 AND SQRT(lon)>0 AND SQRT(smalli)>0 AND SQRT(tinyi)>0");
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + testUnsignedTable + " WHERE SQRT(doub)>0 AND SQRT(fl)>0 AND SQRT(inte)>0 AND SQRT(lon)>0 AND SQRT(smalli)>0 AND SQRT(tinyi)>0");
         rs = stmt.executeQuery();
         if (data > 0) {
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
index c00b770..280b0aa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StddevIT.java
@@ -30,14 +30,14 @@ import java.sql.ResultSet;
 
 import org.junit.Test;
 
-public class StddevIT extends BaseHBaseManagedTimeIT {
+public class StddevIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testSTDDEV_POP() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
 
-        String query = "SELECT STDDEV_POP(A_INTEGER) FROM aTable";
+        String query = "SELECT STDDEV_POP(A_INTEGER) FROM " + tableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -56,9 +56,9 @@ public class StddevIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSTDDEV_SAMP() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
 
-        String query = "SELECT STDDEV_SAMP(x_decimal) FROM aTable";
+        String query = "SELECT STDDEV_SAMP(x_decimal) FROM " + tableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -77,9 +77,9 @@ public class StddevIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSTDDEV_POPOnDecimalColType() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
 
-        String query = "SELECT STDDEV_POP(x_decimal) FROM aTable";
+        String query = "SELECT STDDEV_POP(x_decimal) FROM " + tableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {
@@ -98,9 +98,9 @@ public class StddevIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSTDDEV_SAMPOnDecimalColType() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
 
-        String query = "SELECT STDDEV_SAMP(x_decimal) FROM aTable";
+        String query = "SELECT STDDEV_SAMP(x_decimal) FROM " + tableName;
 
         Connection conn = DriverManager.getConnection(getUrl());
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringIT.java
index b315d7d..d5aad53 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringIT.java
@@ -40,7 +40,7 @@ import com.google.common.collect.Lists;
  * Tests for the LPAD built-in function.
  */
 
-public class StringIT extends BaseHBaseManagedTimeIT {
+public class StringIT extends BaseHBaseManagedTimeTableReuseIT {
     
     /**
      * Helper to test LPAD function
@@ -51,21 +51,23 @@ public class StringIT extends BaseHBaseManagedTimeIT {
      *            name of column to query
      * @param length
      *            length of the output string
-     * @param fillString
+     * @param fillStringList
      *            fill characters to be used while prepending
      * @param sortOrder
      *            sort order of the pk column
-     * @param expectedOutput
+     * @param expectedOutputList
      *            expected output of LPAD function
+     * @param tableName
+     *            base name of the table
      */
     private void testLpadHelper(Connection conn, String colName, int length, List<String> fillStringList,
-        List<String> expectedOutputList, String sortOrder) throws Exception {
+        List<String> expectedOutputList, String tableName, String sortOrder) throws Exception {
         assertEquals("fillStringList and expectedOutputList should be of equal size", fillStringList.size(),
             expectedOutputList.size());
         for (int id = 0; id < fillStringList.size(); ++id) {
             String fillString = fillStringList.get(id);
             String lPadExpr = fillString != null ? "LPAD(%s,?,?)" : "LPAD(%s,?)";
-            String sql = String.format("SELECT " + lPadExpr + " FROM TEST_TABLE_%s WHERE id=?", colName, sortOrder);
+            String sql = String.format("SELECT " + lPadExpr + " FROM " + tableName + "_%s WHERE id=?", colName, sortOrder);
             PreparedStatement stmt = conn.prepareStatement(sql);
             int index = 1;
             stmt.setInt(index++, length);
@@ -93,16 +95,16 @@ public class StringIT extends BaseHBaseManagedTimeIT {
      *            list of fill string to be used while testing
      * @param colName
      *            name of column to be used as function input
-     * @param expecetedOutputList
+     * @param expectedOutputList
      *            list of expected output values
      * @param expectedOutputList
      *            expected output of lpad function
      */
     private void testLpad(Connection conn, List<String> inputList, int length, List<String> fillStringList,
         String colName, List<String> expectedOutputList) throws Exception {
-        TestUtil.initTables(conn, "VARCHAR", new ArrayList<Object>(inputList));
-        testLpadHelper(conn, colName, length, fillStringList, expectedOutputList, "ASC");
-        testLpadHelper(conn, colName, length, fillStringList, expectedOutputList, "DESC");
+        String tableName = TestUtil.initTables(conn, "VARCHAR", new ArrayList<Object>(inputList));
+        testLpadHelper(conn, colName, length, fillStringList, expectedOutputList, tableName, "ASC");
+        testLpadHelper(conn, colName, length, fillStringList, expectedOutputList, tableName, "DESC");
     }
 
     private void testLpad(Connection conn, List<String> inputList, int length, List<String> fillStringList,
@@ -114,23 +116,23 @@ public class StringIT extends BaseHBaseManagedTimeIT {
     public void testCharPadding() throws Exception {
         ResultSet rs;
         Connection conn = DriverManager.getConnection(getUrl());
-        
-        conn.createStatement().execute("CREATE TABLE t (k CHAR(3) PRIMARY KEY)");
-        conn.createStatement().execute("UPSERT INTO t VALUES('a')");
-        conn.createStatement().execute("UPSERT INTO t VALUES('ab')");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k CHAR(3) PRIMARY KEY)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('ab')");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT * FROM t ORDER BY k");
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName + " ORDER BY k");
         assertTrue(rs.next());
         assertEquals("a", rs.getString(1));
         assertTrue(rs.next());
         assertEquals("ab", rs.getString(1));
         assertFalse(rs.next());
-
-        conn.createStatement().execute("CREATE TABLE tdesc (k CHAR(3) PRIMARY KEY DESC)");
-        conn.createStatement().execute("UPSERT INTO tdesc VALUES('a')");
-        conn.createStatement().execute("UPSERT INTO tdesc VALUES('ab')");
+        String tableNameDesc = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableNameDesc + " (k CHAR(3) PRIMARY KEY DESC)");
+        conn.createStatement().execute("UPSERT INTO " + tableNameDesc + " VALUES('a')");
+        conn.createStatement().execute("UPSERT INTO " + tableNameDesc + " VALUES('ab')");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT * FROM tdesc ORDER BY k DESC");
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + tableNameDesc + " ORDER BY k DESC");
         assertTrue(rs.next());
         assertEquals("ab", rs.getString(1));
         assertTrue(rs.next());
@@ -142,23 +144,24 @@ public class StringIT extends BaseHBaseManagedTimeIT {
     public void testBinaryPadding() throws Exception {
         ResultSet rs;
         Connection conn = DriverManager.getConnection(getUrl());
-        
-        conn.createStatement().execute("CREATE TABLE t (k BINARY(3) PRIMARY KEY)");
-        conn.createStatement().execute("UPSERT INTO t VALUES('a')");
-        conn.createStatement().execute("UPSERT INTO t VALUES('ab')");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k BINARY(3) PRIMARY KEY)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('a')");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('ab')");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT * FROM t ORDER BY k");
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName + " ORDER BY k");
         assertTrue(rs.next());
         assertArrayEquals(ByteUtil.concat(Bytes.toBytes("a"), QueryConstants.SEPARATOR_BYTE_ARRAY, QueryConstants.SEPARATOR_BYTE_ARRAY), rs.getBytes(1));
         assertTrue(rs.next());
         assertArrayEquals(ByteUtil.concat(Bytes.toBytes("ab"), QueryConstants.SEPARATOR_BYTE_ARRAY), rs.getBytes(1));
         assertFalse(rs.next());
 
-        conn.createStatement().execute("CREATE TABLE tdesc (k BINARY(3) PRIMARY KEY DESC)");
-        conn.createStatement().execute("UPSERT INTO tdesc VALUES('a')");
-        conn.createStatement().execute("UPSERT INTO tdesc VALUES('ab')");
+        String tableNameDesc = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " +  tableNameDesc + " (k BINARY(3) PRIMARY KEY DESC)");
+        conn.createStatement().execute("UPSERT INTO " + tableNameDesc + " VALUES('a')");
+        conn.createStatement().execute("UPSERT INTO " + tableNameDesc + " VALUES('ab')");
         conn.commit();
-        rs = conn.createStatement().executeQuery("SELECT * FROM tdesc ORDER BY k DESC");
+        rs = conn.createStatement().executeQuery("SELECT * FROM " + tableNameDesc + " ORDER BY k DESC");
         assertTrue(rs.next());
         assertArrayEquals(ByteUtil.concat(Bytes.toBytes("ab"), QueryConstants.SEPARATOR_BYTE_ARRAY), rs.getBytes(1));
         assertTrue(rs.next());
@@ -242,11 +245,12 @@ public class StringIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testStrConcat() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("create table T (PK1 integer, F1 varchar, F2 varchar, F3 varchar, F4 varchar, constraint PK primary key (PK1))");
-        conn.createStatement().execute("upsert into T(PK1, F1,F3) values(0, 'tortilla', 'chip')");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("create table " + tableName + " (PK1 integer, F1 varchar, F2 varchar, F3 varchar, F4 varchar, constraint PK primary key (PK1))");
+        conn.createStatement().execute("upsert into " + tableName + "(PK1, F1,F3) values(0, 'tortilla', 'chip')");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("select * from T where (F1||F2||F3||F4)='tortillachip'");
+        ResultSet rs = conn.createStatement().executeQuery("select * from " + tableName + " where (F1||F2||F3||F4)='tortillachip'");
         assertTrue(rs.next());
         assertEquals(0, rs.getInt(1));
         assertFalse(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
index 2ced621..d94d4e3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SubqueryIT.java
@@ -67,7 +67,8 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
     
     private String[] indexDDL;
     private String[] plans;
-    
+    private String itemTableFullName;
+
     public SubqueryIT(String[] indexDDL, String[] plans) {
         this.indexDDL = indexDDL;
         this.plans = plans;
@@ -322,7 +323,7 @@ public class SubqueryIT extends BaseHBaseManagedTimeIT {
     
     
     protected void initCoItemTableValues() throws Exception {
-        ensureTableCreated(getUrl(), JOIN_COITEM_TABLE_FULL_NAME);
+        ensureTableCreated(getUrl(), JOIN_COITEM_TABLE_FULL_NAME, JOIN_COITEM_TABLE_FULL_NAME);
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
index f28436e..c7b5716 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantIdTypeIT.java
@@ -38,7 +38,7 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
-public class TenantIdTypeIT extends BaseHBaseManagedTimeIT {
+public class TenantIdTypeIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private Connection regularConnection(String url) throws SQLException {
         Properties props = new Properties();
@@ -74,7 +74,7 @@ public class TenantIdTypeIT extends BaseHBaseManagedTimeIT {
         this.dataType = dataType;
         this.tenantId = tenantId;
         this.otherTenantId = otherTenantId;
-        String tbl = "foo" + dataType;
+        String tbl = generateRandomString();
         if(tbl.contains("(")){
             tbl = tbl.substring(0, tbl.indexOf("("));
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index 9970ce7..c99c95f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
+import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -34,11 +36,13 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 
 
@@ -69,16 +73,19 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         testUpdatableViewsWithSameNameDifferentTenants(null, true);
     }
 
+
     @Test
     public void testMultiCFViewIndex() throws Exception {
         testMultiCFViewIndex(false, false);
     }
 
+
     @Test
     public void testMultiCFViewIndexWithNamespaceMapping() throws Exception {
         testMultiCFViewIndex(false, true);
     }
 
+
     @Test
     public void testMultiCFViewLocalIndex() throws Exception {
         testMultiCFViewIndex(true, false);
@@ -110,12 +117,18 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
     }
 
     private void testMultiCFViewIndex(boolean localIndex, boolean isNamespaceEnabled) throws Exception {
-        String tableName = "A.MT_BASE";
-        String baseViewName = "acme";
+        String tableName = "XYZ." + generateRandomString();
+        String baseViewName = generateRandomString() ;
         createTableAndValidate(tableName, isNamespaceEnabled);
         createViewAndIndexesWithTenantId(tableName, baseViewName, localIndex, "b", isNamespaceEnabled);
         createViewAndIndexesWithTenantId(tableName, baseViewName, localIndex, "a", isNamespaceEnabled);
-        validateSequence(tableName, isNamespaceEnabled, "-32767,-32767");
+
+        String sequenceNameA = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("a"), isNamespaceEnabled);
+        String sequenceNameB = getViewIndexSequenceName(PNameFactory.newName(tableName), PNameFactory.newName("b"), isNamespaceEnabled);
+        String sequenceSchemaName = getViewIndexSequenceSchemaName(PNameFactory.newName(tableName), isNamespaceEnabled);
+        verifySequence(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, true);
+        verifySequence(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, true);
+
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "a");
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -126,24 +139,9 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
             conn.createStatement().execute("DROP VIEW  " + baseViewName + "_b");
         }
         DriverManager.getConnection(getUrl()).createStatement().execute("DROP TABLE " + tableName + " CASCADE");
-        validateSequence(tableName, isNamespaceEnabled, null);
-    }
 
-    private void validateSequence(String tableName, boolean isNamespaceEnabled, String expectedResult)
-            throws SQLException {
-        PhoenixConnection phxConn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class);
-        ResultSet rs = phxConn.createStatement().executeQuery("SELECT " + PhoenixDatabaseMetaData.CURRENT_VALUE
-                + "  FROM " + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE);
-        if (expectedResult != null) {
-            String[] splits = expectedResult.split(",");
-            for (String seq : splits) {
-                assertTrue(rs.next());
-                assertEquals(seq, rs.getString(1));
-            }
-        } else {
-            assertFalse(rs.next());
-        }
-        phxConn.close();
+        verifySequence(isNamespaceEnabled? "a" : null, sequenceNameA, sequenceSchemaName, false);
+        verifySequence(isNamespaceEnabled? "b" : null, sequenceNameB, sequenceSchemaName, false);
     }
 
     private void createViewAndIndexesWithTenantId(String tableName,String baseViewName, boolean localIndex, String tenantId,
@@ -250,10 +248,12 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
     public void testNonPaddedTenantId() throws Exception {
         String tenantId1 = "org1";
         String tenantId2 = "org2";
-        String ddl = "CREATE TABLE T (tenantId char(15) NOT NULL, pk1 varchar NOT NULL, pk2 INTEGER NOT NULL, val1 VARCHAR CONSTRAINT pk primary key (tenantId,pk1,pk2)) MULTI_TENANT = true";
+        String tableName = generateRandomString();
+        String viewName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + " (tenantId char(15) NOT NULL, pk1 varchar NOT NULL, pk2 INTEGER NOT NULL, val1 VARCHAR CONSTRAINT pk primary key (tenantId,pk1,pk2)) MULTI_TENANT = true";
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T (tenantId, pk1, pk2, val1) VALUES (?, ?, ?, ?)";
+        String dml = "UPSERT INTO " + tableName + " (tenantId, pk1, pk2, val1) VALUES (?, ?, ?, ?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         
         String pk = "pk1b";
@@ -277,8 +277,8 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         Connection tenantConn = DriverManager.getConnection(tenantUrl);
         
         // create a tenant specific view.
-        tenantConn.createStatement().execute("CREATE VIEW V AS select * from T");
-        String query = "SELECT val1 FROM V WHERE pk1 = ?";
+        tenantConn.createStatement().execute("CREATE VIEW " + viewName + " AS select * from " + tableName);
+        String query = "SELECT val1 FROM " + viewName + " WHERE pk1 = ?";
         
         // using the tenant connection query the view.
         PreparedStatement stmt2 = tenantConn.prepareStatement(query);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
index 72544f2..e976517 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToCharFunctionIT.java
@@ -245,12 +245,13 @@ public class ToCharFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
     @Test
     public void testToCharWithCloneMethod() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-    	String ddl = "create table t (k varchar primary key, v integer[])";
+        String tableName = generateRandomString();
+    	String ddl = "create table " + tableName + " (k varchar primary key, v integer[])";
         conn.createStatement().execute(ddl);
-        conn.createStatement().execute("UPSERT INTO T VALUES('x',ARRAY[1234])");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('x',ARRAY[1234])");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("select to_char(v[1],'000') from t");
+        ResultSet rs = conn.createStatement().executeQuery("select to_char(v[1],'000') from " + tableName);
         assertTrue(rs.next());
         assertEquals("Unexpected value for date ", String.valueOf(1234), rs.getString(1));
         assertFalse(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
index acb29df..b530d2a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ToDateFunctionIT.java
@@ -30,7 +30,7 @@ import java.util.Properties;
 import static org.junit.Assert.*;
 
 
-public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
+public class ToDateFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final long ONE_HOUR_IN_MILLIS = 1000L * 60L * 60L;
 
@@ -255,12 +255,13 @@ public class ToDateFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testToDateWithCloneMethod() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-    	String ddl = "create table t (k varchar primary key, v varchar[])";
+        String tableName = generateRandomString();
+    	String ddl = "create table " + tableName + " (k varchar primary key, v varchar[])";
         conn.createStatement().execute(ddl);
         String dateStr = "2100-01-01";
-        conn.createStatement().execute("UPSERT INTO T VALUES('x',ARRAY['"+dateStr+"'])");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES('x',ARRAY['"+dateStr+"'])");
         conn.commit();
-        ResultSet rs = conn.createStatement().executeQuery("select to_date(v[1], 'yyyy-MM-dd', 'local') from t");
+        ResultSet rs = conn.createStatement().executeQuery("select to_date(v[1], 'yyyy-MM-dd', 'local') from " + tableName);
         
         assertTrue(rs.next());
         assertEquals("Unexpected value for date ", Date.valueOf(dateStr), rs.getDate(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
index 618680d..ca1cd86 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
@@ -27,6 +27,7 @@ import static org.apache.phoenix.util.TestUtil.ROW7;
 import static org.apache.phoenix.util.TestUtil.ROW8;
 import static org.apache.phoenix.util.TestUtil.ROW9;
 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;
@@ -48,7 +49,8 @@ public class TopNIT extends BaseClientManagedTimeIT {
     public void testMultiOrderByExpr() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT entity_id FROM aTable ORDER BY b_string, entity_id LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -78,7 +80,7 @@ public class TopNIT extends BaseClientManagedTimeIT {
     public void testDescMultiOrderByExpr() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "SELECT entity_id FROM aTable ORDER BY b_string || entity_id desc LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -117,7 +119,7 @@ public class TopNIT extends BaseClientManagedTimeIT {
     private void testTopNDelete(boolean autoCommit) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
         String query = "DELETE FROM aTable ORDER BY b_string, entity_id LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
index 9e8f2c0..983d086 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TruncateFunctionIT.java
@@ -57,7 +57,7 @@ public class TruncateFunctionIT extends BaseClientManagedTimeIT {
     public void testTruncate() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        ensureTableCreated(getUrl(), ATABLE_NAME, ts-5);
+        ensureTableCreated(getUrl(), ATABLE_NAME,ATABLE_NAME, ts-5);
         Properties props = new Properties();
         props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts-3));
         Connection conn = DriverManager.getConnection(getUrl(), props);


[32/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
index 10634b8..0ab171a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/GlobalIndexOptimizationIT.java
@@ -28,22 +28,19 @@ import java.sql.SQLException;
 import java.util.Map;
 import java.util.regex.Pattern;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.TestUtil;
+import org.apache.phoenix.util.*;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
 
-public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
+public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @BeforeClass 
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Drop the HBase table metadata for this test
@@ -76,44 +73,58 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testGlobalIndexOptimization() throws Exception {
-        testOptimization(null);
+        String dataTableName = generateRandomString();
+        String indexTableName = generateRandomString();
+        String dataTableFullName = SchemaUtil.getTableName("", dataTableName);
+        testOptimization(dataTableName, dataTableFullName, indexTableName, 4);
     }
     
     @Test
     public void testGlobalIndexOptimizationWithSalting() throws Exception {
-        testOptimization(4);
+        String dataTableName = generateRandomString();
+        String indexTableName = generateRandomString();
+        String dataTableFullName = SchemaUtil.getTableName("", dataTableName);
+        testOptimization(dataTableName, dataTableFullName, indexTableName, 4);
+
     }
     
     @Test
     public void testGlobalIndexOptimizationTenantSpecific() throws Exception {
-        testOptimizationTenantSpecific(null);
+        String dataTableName = generateRandomString();
+        String indexTableName = generateRandomString();
+        testOptimizationTenantSpecific(dataTableName, indexTableName, null);
     }
     
     @Test
     public void testGlobalIndexOptimizationWithSaltingTenantSpecific() throws Exception {
-        testOptimizationTenantSpecific(4);
+        String dataTableName = generateRandomString();
+        String indexTableName = generateRandomString();
+        testOptimizationTenantSpecific(dataTableName, indexTableName, 4);
     }
 
-    private void testOptimization(Integer saltBuckets) throws Exception {
-        createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, saltBuckets, "('e','i','o')", false);
+    private void testOptimization(String dataTableName, String dataTableFullName, String indexTableName, Integer saltBuckets) throws Exception {
+        
+        createBaseTable(dataTableName, saltBuckets, "('e','i','o')", false);
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('b',1,2,4,'z')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('f',1,2,3,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('j',2,4,2,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('q',3,1,1,'c')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('b',1,2,4,'z')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('f',1,2,3,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('j',2,4,2,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('q',3,1,1,'c')");
             conn1.commit();
-            createIndex(TestUtil.DEFAULT_INDEX_TABLE_NAME, TestUtil.DEFAULT_DATA_TABLE_NAME, "v1");
+            createIndex(indexTableName, dataTableName, "v1");
             
-            String query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ * FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a'";
+            String query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ * FROM " + dataTableName +" where v1='a'";
             ResultSet rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
-            
+
+
+
             String expected = 
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " \\['a'\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\['a'\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                    "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                    "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             String actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
             
@@ -130,15 +141,15 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertEquals(2, rs.getInt("k3"));
             assertFalse(rs.next());
             
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ * FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a'";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ * FROM " + dataTableName +" where v1='a'";
             rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             expected = 
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " \\['a'\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\['a'\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                    "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                    "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
             
@@ -157,16 +168,16 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertEquals("a", rs.getString("v1"));
             assertFalse(rs.next());
             
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ * FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a' limit 1";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ * FROM " + dataTableName +" where v1='a' limit 1";
             rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             expected = 
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "CLIENT 1 ROW LIMIT\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " \\['a'\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\['a'\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                    "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)\n" +
+                    "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)\n" +
                     "    JOIN-SCANNER 1 ROW LIMIT";
             actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
@@ -180,18 +191,18 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertEquals("a", rs.getString("v1"));
             assertFalse(rs.next());
             
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ t_id, k1, k2, k3, V1 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "  where v1<='z' and k3 > 1 order by V1,t_id";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ t_id, k1, k2, k3, V1 from " + dataTableFullName + "  where v1<='z' and k3 > 1 order by V1,t_id";
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             
             expected = 
-                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SERVER FILTER BY K3 > 1\n" +
-                    "    SERVER SORTED BY \\[T.V1, T.T_ID\\]\n" +
+                    "    SERVER SORTED BY \\[" + dataTableName + ".V1, " + dataTableName + ".T_ID\\]\n" +
                     "CLIENT MERGE SORT\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " \\[\\*\\] - \\['z'\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\[\\*\\] - \\['z'\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                    "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                    "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
             
@@ -216,17 +227,17 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertEquals("z", rs.getString("V1"));
             assertFalse(rs.next());
             
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ t_id, V1, k3 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "  where v1 <='z' group by v1,t_id, k3";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ t_id, V1, k3 from " + dataTableFullName + "  where v1 <='z' group by v1,t_id, k3";
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             
             expected = 
-                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
-                            "    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[T.V1, T.T_ID, T.K3\\]\n" +
+                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER " + dataTableName + "\n" +
+                            "    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[" + dataTableName + ".V1, " + dataTableName + ".T_ID, " + dataTableName + ".K3\\]\n" +
                             "CLIENT MERGE SORT\n" +
                             "    SKIP-SCAN-JOIN TABLE 0\n" +
-                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " \\[\\*\\] - \\['z'\\]\n" +
+                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\[\\*\\] - \\['z'\\]\n" +
                             "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                            "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                            "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
             
@@ -249,17 +260,17 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertEquals("z", rs.getString("V1"));
             assertFalse(rs.next());
             
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ v1,sum(k3) from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " where v1 <='z'  group by v1 order by v1";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ v1,sum(k3) from " + dataTableFullName + " where v1 <='z'  group by v1 order by v1";
             
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             expected = 
-                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER T\n" +
-                            "    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[T.V1\\]\n" +
+                    "CLIENT PARALLEL \\d-WAY FULL SCAN OVER " + dataTableName + "\n" +
+                            "    SERVER AGGREGATE INTO DISTINCT ROWS BY \\[" + dataTableName + ".V1\\]\n" +
                             "CLIENT MERGE SORT\n" +
                             "    SKIP-SCAN-JOIN TABLE 0\n" +
-                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER I \\[\\*\\] - \\['z'\\]\n" +
+                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\[\\*\\] - \\['z'\\]\n" +
                             "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                            "    DYNAMIC SERVER FILTER BY \\(\"T.T_ID\", \"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                            "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".T_ID\", \"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             actual = QueryUtil.getExplainPlan(rs);
             assertTrue("Expected:\n" + expected + "\nbut got\n" + actual, Pattern.matches(expected, actual));
             
@@ -278,26 +289,26 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private void testOptimizationTenantSpecific(Integer saltBuckets) throws Exception {
-        createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, saltBuckets, "('e','i','o')", true);
+    private void testOptimizationTenantSpecific(String dataTableName, String indexTableName, Integer saltBuckets) throws Exception {
+        createBaseTable(dataTableName, saltBuckets, "('e','i','o')", true);
         Connection conn1 = DriverManager.getConnection(getUrl() + ';' + PhoenixRuntime.TENANT_ID_ATTRIB + "=tid1");
         try{
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values(1,2,4,'z')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values(1,2,3,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values(2,4,2,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values(3,1,1,'c')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values(1,2,4,'z')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values(1,2,3,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values(2,4,2,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values(3,1,1,'c')");
             conn1.commit();
-            createIndex(TestUtil.DEFAULT_INDEX_TABLE_NAME, TestUtil.DEFAULT_DATA_TABLE_NAME, "v1");
+            createIndex(indexTableName, dataTableName, "v1");
             
-            String query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ k1,k2,k3,v1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a'";
+            String query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ k1,k2,k3,v1 FROM " + dataTableName +" where v1='a'";
             ResultSet rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             String actual = QueryUtil.getExplainPlan(rs);
-            String expected = "CLIENT PARALLEL 1-WAY RANGE SCAN OVER T \\['tid1'\\]\n" +
+            String expected = "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " \\['tid1'\\]\n" +
                             "    SKIP-SCAN-JOIN TABLE 0\n" +
-                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER I \\['tid1','a'\\]\n" +
+                            "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " \\['tid1','a'\\]\n" +
                             "            SERVER FILTER BY FIRST KEY ONLY\n" +
-                            "    DYNAMIC SERVER FILTER BY \\(\"T.K1\", \"T.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
+                            "    DYNAMIC SERVER FILTER BY \\(\"" + dataTableName + ".K1\", \"" + dataTableName + ".K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             assertTrue("Expected:\n" + expected + "\ndid not match\n" + actual, Pattern.matches(expected, actual));
             
             rs = conn1.createStatement().executeQuery(query);
@@ -319,15 +330,16 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testGlobalIndexOptimizationOnSharedIndex() throws Exception {
-        createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, null, "('e','i','o')", false);
+        String dataTableName = generateRandomString();
+        createBaseTable(dataTableName, null, "('e','i','o')", false);
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
-            conn1.createStatement().execute("CREATE INDEX i1 ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(k2,k1) INCLUDE (v1)");
-            conn1.createStatement().execute("CREATE VIEW v AS SELECT * FROM t WHERE v1 = 'a'");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('b',1,2,4,'z')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('f',1,2,3,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('j',2,4,2,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('q',3,1,1,'c')");
+            conn1.createStatement().execute("CREATE INDEX i1 ON " + dataTableName + "(k2,k1) INCLUDE (v1)");
+            conn1.createStatement().execute("CREATE VIEW v AS SELECT * FROM " + dataTableName + " WHERE v1 = 'a'");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('b',1,2,4,'z')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('f',1,2,3,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('j',2,4,2,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('q',3,1,1,'c')");
             conn1.commit();
             ResultSet rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM v");
             assertTrue(rs.next());
@@ -340,10 +352,10 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             
             String actual = QueryUtil.getExplainPlan(rs);
             String expected = 
-                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER T\n" +
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                     "    SERVER FILTER BY V1 = 'a'\n" +
                     "    SKIP-SCAN-JOIN TABLE 0\n" +
-                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_T \\[-32768,1\\] - \\[-32768,2\\]\n" +
+                    "        CLIENT PARALLEL 1-WAY SKIP SCAN ON 2 KEYS OVER _IDX_" + dataTableName + " \\[-32768,1\\] - \\[-32768,2\\]\n" +
                     "            SERVER FILTER BY FIRST KEY ONLY AND \"K2\" IN \\(3,4\\)\n" +
                     "    DYNAMIC SERVER FILTER BY \\(\"V.T_ID\", \"V.K1\", \"V.K2\"\\) IN \\(\\(\\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+, \\$\\d+.\\$\\d+\\)\\)";
             assertTrue("Expected:\n" + expected + "\ndid not match\n" + actual, Pattern.matches(expected,actual));
@@ -363,22 +375,25 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testNoGlobalIndexOptimization() throws Exception {
-        createBaseTable(TestUtil.DEFAULT_DATA_TABLE_NAME, null, "('e','i','o')", false);
+        String dataTableName = generateRandomString();
+        String indexTableName = generateRandomString();
+        String dataTableFullName = SchemaUtil.getTableName("", dataTableName);
+        createBaseTable(dataTableName, null, "('e','i','o')", false);
         Connection conn1 = DriverManager.getConnection(getUrl());
         try{
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('b',1,2,4,'z')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('f',1,2,3,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('j',2,4,2,'a')");
-            conn1.createStatement().execute("UPSERT INTO " + TestUtil.DEFAULT_DATA_TABLE_NAME + " values('q',3,1,1,'c')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('b',1,2,4,'z')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('f',1,2,3,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('j',2,4,2,'a')");
+            conn1.createStatement().execute("UPSERT INTO " + dataTableName + " values('q',3,1,1,'c')");
             conn1.commit();
-            conn1.createStatement().execute("CREATE INDEX " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ON " + TestUtil.DEFAULT_DATA_TABLE_NAME + "(v1)");
+            conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableName + "(v1)");
             
             // All columns available in index
-            String query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ t_id, k1, k2, V1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a'";
+            String query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ t_id, k1, k2, V1 FROM " + dataTableName +" where v1='a'";
             ResultSet rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             assertEquals(
-                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + TestUtil.DEFAULT_INDEX_TABLE_NAME + " ['a']\n"
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexTableName + " ['a']\n"
                                 + "    SERVER FILTER BY FIRST KEY ONLY",
                         QueryUtil.getExplainPlan(rs));
             
@@ -394,11 +409,11 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
 
             // No INDEX hint specified
-            query = "SELECT t_id, k1, k2, k3, V1 FROM " + TestUtil.DEFAULT_DATA_TABLE_NAME +" where v1='a'";
+            query = "SELECT t_id, k1, k2, k3, V1 FROM " + dataTableName +" where v1='a'";
             rs = conn1.createStatement().executeQuery("EXPLAIN "+ query);
             
             assertEquals(
-                        "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                        "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + dataTableName + "\n" +
                         "    SERVER FILTER BY V1 = 'a'",
                         QueryUtil.getExplainPlan(rs));
             
@@ -416,11 +431,11 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             // No where clause
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ t_id, k1, k2, k3, V1 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + " order by V1,t_id";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ t_id, k1, k2, k3, V1 from " + dataTableFullName + " order by V1,t_id";
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             
             assertEquals(
-                        "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                        "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + dataTableName + "\n" +
                         "    SERVER SORTED BY [V1, T_ID]\n" +
                         "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
@@ -453,11 +468,11 @@ public class GlobalIndexOptimizationIT extends BaseHBaseManagedTimeIT {
             assertFalse(rs.next());
             
             // No where clause in index scan
-            query = "SELECT /*+ INDEX(" + TestUtil.DEFAULT_DATA_TABLE_NAME + " " + TestUtil.DEFAULT_INDEX_TABLE_NAME + ")*/ t_id, k1, k2, k3, V1 from " + TestUtil.DEFAULT_DATA_TABLE_FULL_NAME + "  where k3 > 1 order by V1,t_id";
+            query = "SELECT /*+ INDEX(" + dataTableName + " " + indexTableName + ")*/ t_id, k1, k2, k3, V1 from " + dataTableFullName + "  where k3 > 1 order by V1,t_id";
             rs = conn1.createStatement().executeQuery("EXPLAIN " + query);
             
             assertEquals(
-                        "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + TestUtil.DEFAULT_DATA_TABLE_NAME + "\n" +
+                        "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + dataTableName + "\n" +
                         "    SERVER FILTER BY K3 > 1\n" +
                         "    SERVER SORTED BY [V1, T_ID]\n" +
                         "CLIENT MERGE SORT",

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/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 ced3ac8..0c7a356 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.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
@@ -69,14 +69,12 @@ import com.google.common.collect.Maps;
 
 
 @RunWith(Parameterized.class)
-public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
+public class ImmutableIndexIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private final boolean localIndex;
+    private final boolean transactional;
     private final String tableDDLOptions;
-    private final String tableName;
-    private final String indexName;
-    private final String fullTableName;
-    private final String fullIndexName;
+
     private volatile boolean stopThreads = false;
 
     private static String TABLE_NAME;
@@ -85,19 +83,17 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
 
     public ImmutableIndexIT(boolean localIndex, boolean transactional) {
         this.localIndex = localIndex;
+        this.transactional = transactional;
         StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
         if (transactional) {
             optionBuilder.append(", TRANSACTIONAL=true");
         }
         this.tableDDLOptions = optionBuilder.toString();
-        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
-        this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
-        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+
     }
 
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put("hbase.coprocessor.region.classes", CreateIndexRegionObserver.class.getName());
@@ -118,6 +114,10 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     @Ignore
     public void testDropIfImmutableKeyValueColumn() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
             String ddl =
@@ -157,7 +157,11 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     public void testCreateIndexDuringUpsertSelect() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(100));
-        TABLE_NAME = fullTableName + "_testCreateIndexDuringUpsertSelect";
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        TABLE_NAME = fullTableName;
         String ddl ="CREATE TABLE " + TABLE_NAME + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         INDEX_DDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IF NOT EXISTS " + indexName + " ON " + TABLE_NAME
                 + " (long_pk, varchar_pk)"
@@ -173,8 +177,8 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
 
             // run the upsert select and also create an index
             conn.setAutoCommit(true);
-            String upsertSelect = "UPSERT INTO " + TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) " + 
-                    "SELECT varchar_pk||'_upsert_select', char_pk, int_pk, long_pk, decimal_pk, date_pk FROM "+ TABLE_NAME;    
+            String upsertSelect = "UPSERT INTO " + TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) " +
+                    "SELECT varchar_pk||'_upsert_select', char_pk, int_pk, long_pk, decimal_pk, date_pk FROM "+ TABLE_NAME;
             conn.createStatement().execute(upsertSelect);
             ResultSet rs;
             rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + TABLE_NAME);
@@ -241,6 +245,10 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testCreateIndexWhileUpsertingData() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         String indexDDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                 + " (long_pk, varchar_pk)"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 21da43a..65f9bac 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -30,7 +30,7 @@ import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.execute.CommitException;
 import org.apache.phoenix.query.QueryConstants;
@@ -40,7 +40,7 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.junit.Test;
 
-public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
+public class IndexExpressionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testImmutableIndexCreateAndUpdate() throws Exception {
@@ -112,18 +112,19 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestCreateAndUpdate(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
 
             // create an expression index
             String ddl = "CREATE "
                     + (localIndex ? "LOCAL" : "")
-                    + " INDEX IDX ON "
+                    + " INDEX " + indexName + " ON "
                     + fullDataTableName
                     + " ((UPPER(varchar_pk) || '_' || UPPER(char_pk) || '_' || UPPER(varchar_col1) || '_' || UPPER(b.char_col2)),"
                     + " (decimal_pk+int_pk+decimal_col2+int_col1)," + " date_pk+1, date1+1, date2+1 )"
@@ -153,7 +154,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST."
                             + dataTableName
                             + " [1,'VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]\nCLIENT MERGE SORT"
-                            : "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST.IDX ['VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]",
+                            : "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST." + indexName + " ['VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]",
                     QueryUtil.getExplainPlan(rs));
 
             // verify that the correct results are returned
@@ -173,7 +174,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     + fullDataTableName;
             rs = conn.createStatement().executeQuery("EXPLAIN " + indexSelectSql);
             assertEquals(localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullDataTableName
-                    + " [1]\nCLIENT MERGE SORT" : "CLIENT PARALLEL 1-WAY FULL SCAN OVER INDEX_TEST.IDX",
+                    + " [1]\nCLIENT MERGE SORT" : "CLIENT PARALLEL 1-WAY FULL SCAN OVER INDEX_TEST." + indexName,
                     QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(indexSelectSql);
             verifyResult(rs, 1);
@@ -195,7 +196,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             verifyResult(rs, 3);
             verifyResult(rs, 4);
 
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -203,27 +204,31 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testMutableIndexUpdate() throws Exception {
-    	helpTestUpdate(false);
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+    	helpTestUpdate(dataTableName, fullDataTableName, indexName, false);
     }
 
     @Test
     public void testMutableLocalIndexUpdate() throws Exception {
-    	helpTestUpdate(true);
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+        helpTestUpdate(dataTableName, fullDataTableName, indexName, true);
     }
     
-    protected void helpTestUpdate(boolean localIndex) throws Exception {
-        String dataTableName = MUTABLE_INDEX_DATA_TABLE;
-        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+    protected void helpTestUpdate(String dataTableName, String fullDataTableName, String indexName, boolean localIndex) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
+            populateDataTable(conn, dataTableName, MUTABLE_INDEX_DATA_TABLE);
 
             // create an expression index
             String ddl = "CREATE "
                     + (localIndex ? "LOCAL" : "")
-                    + " INDEX IDX ON "
+                    + " INDEX " + indexName + " ON "
                     + fullDataTableName
                     + " ((UPPER(varchar_pk) || '_' || UPPER(char_pk) || '_' || UPPER(varchar_col1) || '_' || UPPER(char_col2)),"
                     + " (decimal_pk+int_pk+decimal_col2+int_col1)," + " date_pk+1, date1+1, date2+1 )"
@@ -269,14 +274,14 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals("VARCHAR2_CHAR2     _A.VARCHAR2_B.CHAR2   ", rs.getString(1));
             assertEquals(2, rs.getLong(2));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
     }
 
-    private void populateDataTable(Connection conn, String dataTable) throws SQLException {
-        ensureTableCreated(getUrl(), dataTable);
+    private void populateDataTable(Connection conn, String dataTable, String tableType) throws SQLException {
+        ensureTableCreated(getUrl(), dataTable, tableType);
         String upsert = "UPSERT INTO " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTable
                 + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
         PreparedStatement stmt1 = conn.prepareStatement(upsert);
@@ -307,16 +312,17 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestDeleteIndexedExpression(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
-        String fullIndexTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + "IDX";
+        String indexName = generateRandomString();
+        String fullIndexTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexName;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            ensureTableCreated(getUrl(), dataTableName);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            ensureTableCreated(getUrl(), dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (2*long_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
@@ -353,7 +359,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexTableName);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -380,16 +386,17 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestDeleteCoveredCol(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
-        String fullIndexTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + "IDX";
+        String indexName = generateRandomString();
+        String fullIndexTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexName;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            ensureTableCreated(getUrl(), dataTableName);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            ensureTableCreated(getUrl(), dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (long_pk, varchar_pk, 1+long_pk, UPPER(varchar_pk) )" + " INCLUDE (long_col1, long_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
@@ -434,7 +441,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals(2L, rs.getLong(3));
             assertEquals("VARCHAR1", rs.getString(4));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -461,14 +468,15 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestGroupByCount(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (int_col1+int_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
@@ -478,7 +486,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + groupBySql);
             String expectedPlan = "CLIENT PARALLEL 1-WAY "
                     + (localIndex ? "RANGE SCAN OVER " + fullDataTableName + " [1]"
-                            : "FULL SCAN OVER INDEX_TEST.IDX")
+                            : "FULL SCAN OVER INDEX_TEST." + indexName)
                     + "\n    SERVER FILTER BY FIRST KEY ONLY\n    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [TO_BIGINT(\"(A.INT_COL1 + B.INT_COL2)\")]" 
                     + (localIndex ? "\nCLIENT MERGE SORT" : "");
             assertEquals(expectedPlan, QueryUtil.getExplainPlan(rs));
@@ -488,7 +496,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(2));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -515,14 +523,15 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestSelectDistinct(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (int_col1+1)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
@@ -530,7 +539,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             String expectedPlan = "CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
                     + (localIndex ? fullDataTableName + " [1,0] - [1,*]"
-                            : "INDEX_TEST.IDX [0] - [*]")
+                            : "INDEX_TEST." + indexName + " [0] - [*]")
                     + "\n    SERVER FILTER BY FIRST KEY ONLY\n    SERVER DISTINCT PREFIX FILTER OVER [TO_BIGINT(\"(A.INT_COL1 + 1)\")]\n    SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [TO_BIGINT(\"(A.INT_COL1 + 1)\")]"
                     + (localIndex ? "\nCLIENT MERGE SORT" : "");
             assertEquals(expectedPlan, QueryUtil.getExplainPlan(rs));
@@ -540,7 +549,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -567,14 +576,16 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestInClauseWithIndex(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (int_col1+1)";
 
             PreparedStatement stmt = conn.prepareStatement(ddl);
@@ -583,12 +594,12 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + sql);
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
                     + (localIndex ? fullDataTableName + " [1,2]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
-                            : "INDEX_TEST.IDX [2]\n    SERVER FILTER BY FIRST KEY ONLY"), QueryUtil.getExplainPlan(rs));
+                            : "INDEX_TEST." + indexName + " [2]\n    SERVER FILTER BY FIRST KEY ONLY"), QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(sql);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -615,14 +626,16 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestSelectAliasAndOrderByWithIndex(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (int_col1+1)";
 
             PreparedStatement stmt = conn.prepareStatement(ddl);
@@ -632,7 +645,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals("CLIENT PARALLEL 1-WAY "
                     + (localIndex ? "RANGE SCAN OVER " + fullDataTableName
                             + " [1]\n    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT"
-                            : "FULL SCAN OVER INDEX_TEST.IDX\n    SERVER FILTER BY FIRST KEY ONLY"),
+                            : "FULL SCAN OVER INDEX_TEST." + indexName + "\n    SERVER FILTER BY FIRST KEY ONLY"),
                     QueryUtil.getExplainPlan(rs));
             rs = conn.createStatement().executeQuery(sql);
             assertTrue(rs.next());
@@ -640,7 +653,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -667,21 +680,24 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
     
     protected void helpTestIndexWithCaseSensitiveCols(boolean mutable, boolean localIndex) throws Exception {
+        String dataTableName = generateRandomString();
+        String indexName = generateRandomString();
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
-            conn.createStatement().execute("CREATE TABLE cs (k VARCHAR NOT NULL PRIMARY KEY, \"cf1\".\"V1\" VARCHAR, \"CF2\".\"v2\" VARCHAR) "+ (mutable ? "IMMUTABLE_ROWS=true" : ""));
-            String query = "SELECT * FROM cs";
+            conn.createStatement().execute("CREATE TABLE " + dataTableName + " (k VARCHAR NOT NULL PRIMARY KEY, \"cf1\".\"V1\" VARCHAR, \"CF2\".\"v2\" VARCHAR) "+ (mutable ? "IMMUTABLE_ROWS=true" : ""));
+            String query = "SELECT * FROM " + dataTableName;
             ResultSet rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX ics ON cs (\"cf1\".\"V1\" || '_' || \"CF2\".\"v2\") INCLUDE (\"V1\",\"v2\")";
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + dataTableName + " (\"cf1\".\"V1\" || '_' || \"CF2\".\"v2\") INCLUDE (\"V1\",\"v2\")";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            query = "SELECT * FROM ics";
+            query = "SELECT * FROM " + indexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
 
-            stmt = conn.prepareStatement("UPSERT INTO cs VALUES(?,?,?)");
+            stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?,?)");
             stmt.setString(1,"a");
             stmt.setString(2, "x");
             stmt.setString(3, "1");
@@ -692,13 +708,13 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
             conn.commit();
 
-            query = "SELECT (\"V1\" || '_' || \"v2\"), k, \"V1\", \"v2\"  FROM cs WHERE (\"V1\" || '_' || \"v2\") = 'x_1'";
+            query = "SELECT (\"V1\" || '_' || \"v2\"), k, \"V1\", \"v2\"  FROM " + dataTableName + " WHERE (\"V1\" || '_' || \"v2\") = 'x_1'";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER CS [1,'x_1']\n"
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1,'x_1']\n"
                            + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER ICS ['x_1']", QueryUtil.getExplainPlan(rs));
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexName + " ['x_1']", QueryUtil.getExplainPlan(rs));
             }
 
             rs = conn.createStatement().executeQuery(query);
@@ -714,13 +730,13 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals("1",rs.getString("v2"));
             assertFalse(rs.next());
 
-            query = "SELECT \"V1\", \"V1\" as foo1, (\"V1\" || '_' || \"v2\") as foo, (\"V1\" || '_' || \"v2\") as \"Foo1\", (\"V1\" || '_' || \"v2\") FROM cs ORDER BY foo";
+            query = "SELECT \"V1\", \"V1\" as foo1, (\"V1\" || '_' || \"v2\") as foo, (\"V1\" || '_' || \"v2\") as \"Foo1\", (\"V1\" || '_' || \"v2\") FROM " + dataTableName + " ORDER BY foo";
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
-                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER CS [1]\nCLIENT MERGE SORT",
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1]\nCLIENT MERGE SORT",
                     QueryUtil.getExplainPlan(rs));
             } else {
-                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER ICS", QueryUtil.getExplainPlan(rs));
+                assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + indexName, QueryUtil.getExplainPlan(rs));
             }
 
             rs = conn.createStatement().executeQuery(query);
@@ -747,7 +763,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals("y_2",rs.getString(5));
             assertEquals("y_2",rs.getString("\"('cf1'.'V1' || '_' || 'CF2'.'v2')\""));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX ICS ON CS");
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + dataTableName);
         } finally {
             conn.close();
         }
@@ -774,14 +790,16 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     }
 
     protected void helpTestSelectColOnlyInDataTable(boolean mutable, boolean localIndex) throws Exception {
-        String dataTableName = mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE;
+        String dataTableName = generateRandomString();
         String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
             conn.setAutoCommit(false);
-            populateDataTable(conn, dataTableName);
-            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IDX ON " + fullDataTableName
+            populateDataTable(conn, dataTableName, mutable ? MUTABLE_INDEX_DATA_TABLE : INDEX_DATA_TABLE);
+            String ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullDataTableName
                     + " (int_col1+1)";
 
             conn = DriverManager.getConnection(getUrl(), props);
@@ -800,7 +818,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals(2, rs.getInt(1));
             assertEquals(1, rs.getInt(2));
             assertFalse(rs.next());
-            conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullDataTableName);
         } finally {
             conn.close();
         }
@@ -833,35 +851,39 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
+
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+
         try {
 	        conn.setAutoCommit(false);
 	
 	        // make sure that the tables are empty, but reachable
 	        conn.createStatement().execute(
-	          "CREATE TABLE t (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-	        query = "SELECT * FROM t" ;
+	          "CREATE TABLE " + dataTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+	        query = "SELECT * FROM " + dataTableName ;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertFalse(rs.next());
-	        String indexName = "it_" + (mutable ? "m" : "im") + "_" + (local ? "l" : "h");
-	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON t (v1 || '_' || v2)");
+	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON " + dataTableName + " (v1 || '_' || v2)");
 	
-	        query = "SELECT * FROM t";
+	        query = "SELECT * FROM " + dataTableName;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertFalse(rs.next());
 	
 	        // load some data into the table
-	        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?,?)");
 	        stmt.setString(1, "a");
 	        stmt.setString(2, "x");
 	        stmt.setString(3, "1");
 	        stmt.execute();
 	        conn.commit();
 	
-	        assertIndexExists(conn,true);
-	        conn.createStatement().execute("ALTER TABLE t DROP COLUMN v1");
-	        assertIndexExists(conn,false);
+	        assertIndexExists(conn, dataTableName, true);
+	        conn.createStatement().execute("ALTER TABLE " + dataTableName + " DROP COLUMN v1");
+	        assertIndexExists(conn, dataTableName, false);
 	
-	        query = "SELECT * FROM t";
+	        query = "SELECT * FROM " + dataTableName;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
 	        assertEquals("a",rs.getString(1));
@@ -869,13 +891,13 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        assertFalse(rs.next());
 	
 	        // load some data into the table
-	        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?)");
 	        stmt.setString(1, "a");
 	        stmt.setString(2, "2");
 	        stmt.execute();
 	        conn.commit();
 	
-	        query = "SELECT * FROM t";
+	        query = "SELECT * FROM " + dataTableName;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
 	        assertEquals("a",rs.getString(1));
@@ -887,8 +909,8 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
         }
     }
     
-    private static void assertIndexExists(Connection conn, boolean exists) throws SQLException {
-        ResultSet rs = conn.getMetaData().getIndexInfo(null, null, "T", false, false);
+    private static void assertIndexExists(Connection conn, String tableName, boolean exists) throws SQLException {
+        ResultSet rs = conn.getMetaData().getIndexInfo(null, null, tableName, false, false);
         assertEquals(exists, rs.next());
     }
     
@@ -915,6 +937,9 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     public void helpTestDropCoveredColumn(boolean mutable, boolean local) throws Exception {
         ResultSet rs;
         PreparedStatement stmt;
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -923,20 +948,19 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	
 	        // make sure that the tables are empty, but reachable
 	        conn.createStatement().execute(
-	          "CREATE TABLE t"
+	          "CREATE TABLE " + dataTableName
 	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR)");
-	        String dataTableQuery = "SELECT * FROM t";
+	        String dataTableQuery = "SELECT * FROM " + dataTableName;
 	        rs = conn.createStatement().executeQuery(dataTableQuery);
 	        assertFalse(rs.next());
 	
-	        String indexName = "it_" + (mutable ? "m" : "im") + "_" + (local ? "l" : "h");
-	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON t (k || '_' || v1) include (v2, v3)");
+	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON " + dataTableName + " (k || '_' || v1) include (v2, v3)");
 	        String indexTableQuery = "SELECT * FROM " + indexName;
 	        rs = conn.createStatement().executeQuery(indexTableQuery);
 	        assertFalse(rs.next());
 	
 	        // load some data into the table
-	        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?,?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?,?,?)");
 	        stmt.setString(1, "a");
 	        stmt.setString(2, "x");
 	        stmt.setString(3, "1");
@@ -944,9 +968,9 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        stmt.execute();
 	        conn.commit();
 	
-	        assertIndexExists(conn,true);
-	        conn.createStatement().execute("ALTER TABLE t DROP COLUMN v2");
-	        assertIndexExists(conn,true);
+	        assertIndexExists(conn, dataTableName, true);
+	        conn.createStatement().execute("ALTER TABLE " + dataTableName + " DROP COLUMN v2");
+	        assertIndexExists(conn, dataTableName, true);
 	
 	        // verify data table rows
 	        rs = conn.createStatement().executeQuery(dataTableQuery);
@@ -965,7 +989,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        assertFalse(rs.next());
 	
 	        // add another row
-	        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?,?)");
 	        stmt.setString(1, "b");
 	        stmt.setString(2, "y");
 	        stmt.setString(3, "k");
@@ -1025,6 +1049,10 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
         ResultSet rs;
         PreparedStatement stmt;
 
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName = generateRandomString();
+
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -1032,29 +1060,28 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	
 	        // make sure that the tables are empty, but reachable
 	        conn.createStatement().execute(
-	          "CREATE TABLE t"
+	          "CREATE TABLE "  + dataTableName
 	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-	        String dataTableQuery = "SELECT * FROM t";
+	        String dataTableQuery = "SELECT * FROM " + dataTableName;
 	        rs = conn.createStatement().executeQuery(dataTableQuery);
 	        assertFalse(rs.next());
 	
-	        String indexName = "IT_" + (mutable ? "M" : "IM") + "_" + (local ? "L" : "H");
-	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON t (v1 || '_' || v2)");
+	        conn.createStatement().execute("CREATE " + ( local ? "LOCAL" : "") + " INDEX " + indexName + " ON " + dataTableName + " (v1 || '_' || v2)");
 	        String indexTableQuery = "SELECT * FROM " + indexName;
 	        rs = conn.createStatement().executeQuery(indexTableQuery);
 	        assertFalse(rs.next());
 	
 	        // load some data into the table
-	        stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?,?)");
 	        stmt.setString(1, "a");
 	        stmt.setString(2, "x");
 	        stmt.setString(3, "1");
 	        stmt.execute();
 	        conn.commit();
 	
-	        assertIndexExists(conn,true);
-	        conn.createStatement().execute("ALTER TABLE t ADD v3 VARCHAR, k2 DECIMAL PRIMARY KEY");
-	        rs = conn.getMetaData().getPrimaryKeys("", "", "T");
+	        assertIndexExists(conn, dataTableName, true);
+	        conn.createStatement().execute("ALTER TABLE " + dataTableName + " ADD v3 VARCHAR, k2 DECIMAL PRIMARY KEY");
+	        rs = conn.getMetaData().getPrimaryKeys("", "", dataTableName);
 	        assertTrue(rs.next());
 	        assertEquals("K",rs.getString("COLUMN_NAME"));
 	        assertEquals(1, rs.getShort("KEY_SEQ"));
@@ -1092,7 +1119,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        assertFalse(rs.next());
 	
 	        // load some data into the table
-	        stmt = conn.prepareStatement("UPSERT INTO t(K,K2,V1,V2) VALUES(?,?,?,?)");
+	        stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + "(K,K2,V1,V2) VALUES(?,?,?,?)");
 	        stmt.setString(1, "b");
 	        stmt.setBigDecimal(2, BigDecimal.valueOf(2));
 	        stmt.setString(3, "y");
@@ -1145,28 +1172,33 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
        
     private void helpTestUpdatableViewIndex(boolean local) throws Exception {
     	Connection conn = DriverManager.getConnection(getUrl());
+        String dataTableName = generateRandomString();
+        String fullDataTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + dataTableName;
+        String indexName1 = generateRandomString();
+        String viewName = generateRandomString();
+        String indexName2 = generateRandomString();
     	try {
-	        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))";
+	        String ddl = "CREATE TABLE " + dataTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2, k3))";
 	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE k1 = 1";
+	        ddl = "CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTableName + " WHERE k1 = 1";
 	        conn.createStatement().execute(ddl);
-	        conn.createStatement().execute("UPSERT INTO v(k2,s1,s2,k3) VALUES(120,'foo0','bar0',50.0)");
-	        conn.createStatement().execute("UPSERT INTO v(k2,s1,s2,k3) VALUES(121,'foo1','bar1',51.0)");
+	        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,s1,s2,k3) VALUES(120,'foo0','bar0',50.0)");
+	        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,s1,s2,k3) VALUES(121,'foo1','bar1',51.0)");
 	        conn.commit();
 	        
 	        ResultSet rs;
-	        conn.createStatement().execute("CREATE " + (local ? "LOCAL" : "") + " INDEX i1 on v(k1+k2+k3) include (s1, s2)");
-	        conn.createStatement().execute("UPSERT INTO v(k2,s1,s2,k3) VALUES(120,'foo2','bar2',50.0)");
+	        conn.createStatement().execute("CREATE " + (local ? "LOCAL" : "") + " INDEX " + indexName1 + " on " + viewName + "(k1+k2+k3) include (s1, s2)");
+	        conn.createStatement().execute("UPSERT INTO " + viewName + "(k2,s1,s2,k3) VALUES(120,'foo2','bar2',50.0)");
 	        conn.commit();
 	
-	        String query = "SELECT k1, k2, k3, s1, s2 FROM v WHERE 	k1+k2+k3 = 173.0";
+	        String query = "SELECT k1, k2, k3, s1, s2 FROM " + viewName + " WHERE 	k1+k2+k3 = 173.0";
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        String queryPlan = QueryUtil.getExplainPlan(rs);
 	        if (local) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,173]\n" + "CLIENT MERGE SORT",
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1,173]\n" + "CLIENT MERGE SORT",
 	                    queryPlan);
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [" + Short.MIN_VALUE + ",173]", queryPlan);
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + Short.MIN_VALUE + ",173]", queryPlan);
 	        }
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
@@ -1177,16 +1209,16 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 	        assertEquals("bar1", rs.getString(5));
 	        assertFalse(rs.next());
 	
-	        conn.createStatement().execute("CREATE " + (local ? "LOCAL" : "") + " INDEX i2 on v(s1||'_'||s2)");
+	        conn.createStatement().execute("CREATE " + (local ? "LOCAL" : "") + " INDEX " + indexName2 + " on " + viewName + "(s1||'_'||s2)");
 	        
-	        query = "SELECT k1, k2, s1||'_'||s2 FROM v WHERE (s1||'_'||s2)='foo2_bar2'";
+	        query = "SELECT k1, k2, s1||'_'||s2 FROM " + viewName + " WHERE (s1||'_'||s2)='foo2_bar2'";
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        if (local) {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [" + (2)
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [" + (2)
 	                    + ",'foo2_bar2']\n" + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
 	                    QueryUtil.getExplainPlan(rs));
 	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_T [" + (Short.MIN_VALUE + 1) + ",'foo2_bar2']\n"
+	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_" + dataTableName + " [" + (Short.MIN_VALUE + 1) + ",'foo2_bar2']\n"
 	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
 	        }
 	        rs = conn.createStatement().executeQuery(query);
@@ -1206,41 +1238,45 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl());
         try 
         {
+            String dataTableName = generateRandomString();
+            String indexName1 = generateRandomString();
+            String viewName = generateRandomString();
+            String indexName2 = generateRandomString();
         	ResultSet rs;
-	        String ddl = "CREATE TABLE t (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2))";
+	        String ddl = "CREATE TABLE " + dataTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2))";
 	        conn.createStatement().execute(ddl);
-	        conn.createStatement().execute("CREATE INDEX i1 ON t(k2, s2, s3, s1)");
-	        conn.createStatement().execute("CREATE INDEX i2 ON t(k2, s2||'_'||s3, s1, s4)");
+	        conn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + dataTableName + "(k2, s2, s3, s1)");
+	        conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON " + dataTableName + "(k2, s2||'_'||s3, s1, s4)");
 	        
-	        ddl = "CREATE VIEW v AS SELECT * FROM t WHERE s1 = 'foo'";
+	        ddl = "CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTableName + " WHERE s1 = 'foo'";
 	        conn.createStatement().execute(ddl);
-	        conn.createStatement().execute("UPSERT INTO t VALUES(1,1,'foo','abc','cab')");
-	        conn.createStatement().execute("UPSERT INTO t VALUES(2,2,'bar','xyz','zyx')");
+	        conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES(1,1,'foo','abc','cab')");
+	        conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES(2,2,'bar','xyz','zyx')");
 	        conn.commit();
 	        
-	        rs = conn.createStatement().executeQuery("SELECT count(*) FROM v");
+	        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + viewName);
 	        assertTrue(rs.next());
 	        assertEquals(1, rs.getLong(1));
 	        assertFalse(rs.next());
 	        
 	        //i2 should be used since it contains s3||'_'||s4 i
-	        String query = "SELECT s2||'_'||s3 FROM v WHERE k2=1 AND (s2||'_'||s3)='abc_cab'";
+	        String query = "SELECT s2||'_'||s3 FROM " + viewName + " WHERE k2=1 AND (s2||'_'||s3)='abc_cab'";
 	        rs = conn.createStatement(  ).executeQuery("EXPLAIN " + query);
 	        String queryPlan = QueryUtil.getExplainPlan(rs);
 	        assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I2 [1,'abc_cab','foo']\n" + 
+	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexName2 + " [1,'abc_cab','foo']\n" +
 	                "    SERVER FILTER BY FIRST KEY ONLY", queryPlan);
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
 	        assertEquals("abc_cab", rs.getString(1));
 	        assertFalse(rs.next());
 	        
-	        conn.createStatement().execute("ALTER VIEW v DROP COLUMN s4");
+	        conn.createStatement().execute("ALTER VIEW " + viewName + " DROP COLUMN s4");
 	        //i2 cannot be used since s4 has been dropped from the view, so i1 will be used 
 	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 	        queryPlan = QueryUtil.getExplainPlan(rs);
 	        assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER I1 [1]\n" + 
+	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexName1 + " [1]\n" +
 	                "    SERVER FILTER BY FIRST KEY ONLY AND ((\"S2\" || '_' || \"S3\") = 'abc_cab' AND \"S1\" = 'foo')", queryPlan);
 	        rs = conn.createStatement().executeQuery(query);
 	        assertTrue(rs.next());
@@ -1254,17 +1290,19 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
     
 	@Test
 	public void testExpressionThrowsException() throws Exception {
-		Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = DriverManager.getConnection(getUrl());
+        String dataTableName = generateRandomString();
+        String indexName = generateRandomString();
 		try {
-			String ddl = "CREATE TABLE t (k1 INTEGER PRIMARY KEY, k2 INTEGER)";
+			String ddl = "CREATE TABLE " + dataTableName + " (k1 INTEGER PRIMARY KEY, k2 INTEGER)";
 			conn.createStatement().execute(ddl);
-			ddl = "CREATE INDEX i on t(k1/k2)";
+			ddl = "CREATE INDEX " + indexName + " on " + dataTableName + "(k1/k2)";
 			conn.createStatement().execute(ddl);
 			// upsert should succeed
-			conn.createStatement().execute("UPSERT INTO T VALUES(1,1)");
+			conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES(1,1)");
 			conn.commit();
 			// divide by zero should fail
-			conn.createStatement().execute("UPSERT INTO T VALUES(1,0)");
+			conn.createStatement().execute("UPSERT INTO " + dataTableName + " VALUES(1,0)");
 			conn.commit();
 			fail();
 		} catch (CommitException e) {
@@ -1297,22 +1335,24 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 			boolean localIndex) throws Exception {
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		Connection conn = DriverManager.getConnection(getUrl(), props);
+        String dataTableName = generateRandomString();
+        String indexName = generateRandomString();
 		try {
 			conn.createStatement().execute(
-					"CREATE TABLE t (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) "
+					"CREATE TABLE " + dataTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) "
 							+ (mutable ? "IMMUTABLE_ROWS=true" : ""));
-			String query = "SELECT * FROM t";
+			String query = "SELECT * FROM  " + dataTableName;
 			ResultSet rs = conn.createStatement().executeQuery(query);
 			assertFalse(rs.next());
 			String ddl = "CREATE " + (localIndex ? "LOCAL" : "")
-					+ " INDEX idx ON t (REGEXP_SUBSTR(v,'id:\\\\w+'))";
+					+ " INDEX " + indexName + " ON " + dataTableName + " (REGEXP_SUBSTR(v,'id:\\\\w+'))";
 			PreparedStatement stmt = conn.prepareStatement(ddl);
 			stmt.execute();
-			query = "SELECT * FROM idx";
+			query = "SELECT * FROM " + indexName;
 			rs = conn.createStatement().executeQuery(query);
 			assertFalse(rs.next());
 
-			stmt = conn.prepareStatement("UPSERT INTO t VALUES(?,?)");
+			stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?)");
 			stmt.setString(1, "k1");
 			stmt.setString(2, "{id:id1}");
 			stmt.execute();
@@ -1321,16 +1361,16 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 			stmt.execute();
 			conn.commit();
 			
-			query = "SELECT k FROM t WHERE REGEXP_SUBSTR(v,'id:\\\\w+') = 'id:id1'";
+			query = "SELECT k FROM " + dataTableName + " WHERE REGEXP_SUBSTR(v,'id:\\\\w+') = 'id:id1'";
 			rs = conn.createStatement().executeQuery("EXPLAIN " + query);
 			if (localIndex) {
 				assertEquals(
-						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER T [1,'id:id1']\n"
+						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + dataTableName + " [1,'id:id1']\n"
 								+ "    SERVER FILTER BY FIRST KEY ONLY\nCLIENT MERGE SORT",
 						QueryUtil.getExplainPlan(rs));
 			} else {
 				assertEquals(
-						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER IDX ['id:id1']\n"
+						"CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + indexName + " ['id:id1']\n"
 								+ "    SERVER FILTER BY FIRST KEY ONLY",
 						QueryUtil.getExplainPlan(rs));
 			}
@@ -1368,27 +1408,26 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 			boolean localIndex) throws Exception {
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		Connection conn = DriverManager.getConnection(getUrl(), props);
-		String nameSuffix = "t" + (mutable ? "_mutable" : "_immutable") + (localIndex ? "_local" : "_global");
-		String tableName = "t" + nameSuffix;
-		String indexName = "idx" + nameSuffix;
-		try {
+        String dataTableName = generateRandomString();
+        String indexName = generateRandomString();
+        try {
 			conn.createStatement().execute(
-				"CREATE TABLE " + tableName + " ("
+				"CREATE TABLE " + dataTableName + " ("
 							+ "pk1 VARCHAR not null, "
 							+ "pk2 VARCHAR not null, "
 							+ "CONSTRAINT PK PRIMARY KEY (pk1, pk2))"
 							+ (!mutable ? "IMMUTABLE_ROWS=true" : ""));
-			String query = "SELECT * FROM " + tableName;
+			String query = "SELECT * FROM " + dataTableName;
 			ResultSet rs = conn.createStatement().executeQuery(query);
 			assertFalse(rs.next());
 			conn.createStatement().execute(
 				"CREATE " + (localIndex ? "LOCAL" : "")
-					+ " INDEX " + indexName + " ON " + tableName + " (pk2, pk1)");
+					+ " INDEX " + indexName + " ON " + dataTableName + " (pk2, pk1)");
 			query = "SELECT * FROM " + indexName;
 			rs = conn.createStatement().executeQuery(query);
 			assertFalse(rs.next());
 
-			PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?)");
+			PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + dataTableName + " VALUES(?,?)");
 			stmt.setString(1, "k11");
 			stmt.setString(2, "k21");
 			stmt.execute();
@@ -1401,7 +1440,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals("k11", rs.getString(2));
             assertFalse(rs.next());
             
-			query = "SELECT * FROM " + tableName + " WHERE pk2='k21'";
+			query = "SELECT * FROM " + dataTableName + " WHERE pk2='k21'";
 			rs = conn.createStatement().executeQuery(query);
 			assertTrue(rs.next());
 			assertEquals("k11", rs.getString(1));


[22/50] [abbrv] phoenix git commit: PHOENIX-3195 Slight safety improvement for using DistinctPrefixFilter.

Posted by ma...@apache.org.
PHOENIX-3195 Slight safety improvement for using DistinctPrefixFilter.


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

Branch: refs/heads/calcite
Commit: 1f7a05b63ee86d79159ecb5fdeb01338207567af
Parents: bea9e03
Author: Lars Hofhansl <la...@apache.org>
Authored: Mon Aug 22 13:37:02 2016 -0700
Committer: Lars Hofhansl <la...@apache.org>
Committed: Mon Aug 22 13:40:08 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/iterate/BaseResultIterators.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1f7a05b6/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index ceba000..8b9adfd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -227,7 +227,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             }
 
             int cols = plan.getGroupBy().getOrderPreservingColumnCount();
-            if (cols > 0 && context.getWhereConditionColumns().size() == 0 &&
+            if (cols > 0 && keyOnlyFilter &&
                 !plan.getStatement().getHint().hasHint(HintNode.Hint.RANGE_SCAN) &&
                 cols < plan.getTableRef().getTable().getRowKeySchema().getFieldCount() &&
                 plan.getGroupBy().isOrderPreserving() &&


[25/50] [abbrv] phoenix git commit: PHOENIX-3148 Reduce size of PTable so that more tables can be cached in the metada cache

Posted by ma...@apache.org.
PHOENIX-3148 Reduce size of PTable so that more tables can be cached in the metada cache


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

Branch: refs/heads/calcite
Commit: a0ae8025eccbd76b4277f74e57081ddbb5a7babe
Parents: b36bb31
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Fri Aug 19 16:39:28 2016 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Aug 23 23:00:45 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/IndexToolIT.java |   1 -
 .../apache/phoenix/execute/MutationState.java   |   6 +-
 .../query/ConnectionQueryServicesImpl.java      |   5 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   4 +-
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java     |   3 +
 .../apache/phoenix/schema/MetaDataClient.java   |  25 +-
 .../apache/phoenix/schema/PMetaDataCache.java   | 221 +++++++++++++++
 .../apache/phoenix/schema/PMetaDataImpl.java    | 268 +++----------------
 .../org/apache/phoenix/schema/PTableImpl.java   |  29 +-
 .../org/apache/phoenix/schema/PTableRef.java    |  56 ++--
 .../apache/phoenix/schema/PTableRefFactory.java |  52 ++++
 .../apache/phoenix/schema/PTableRefImpl.java    |  39 +++
 .../phoenix/schema/SerializedPTableRef.java     |  47 ++++
 .../schema/SerializedPTableRefFactory.java      |  37 +++
 .../phoenix/schema/PMetaDataImplTest.java       |  34 ++-
 16 files changed, 532 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index c66fea3..16db876 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -37,7 +37,6 @@ import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.mapreduce.index.IndexTool;
-import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 7a9282c..e7e6aa7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -680,7 +680,7 @@ public class MutationState implements SQLCloseable {
             return Iterators.emptyIterator();
         }
         Long scn = connection.getSCN();
-        final long timestamp = (tableTimestamp!=null && tableTimestamp!=QueryConstants.UNSET_TIMESTAMP) ? tableTimestamp : (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
+        final long timestamp = getMutationTimestamp(tableTimestamp, scn);
         return new Iterator<Pair<byte[],List<Mutation>>>() {
             private Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> current = iterator.next();
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
@@ -726,6 +726,10 @@ public class MutationState implements SQLCloseable {
             
         };
     }
+
+    public static long getMutationTimestamp(final Long tableTimestamp, Long scn) {
+        return (tableTimestamp!=null && tableTimestamp!=QueryConstants.UNSET_TIMESTAMP) ? tableTimestamp : (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
+    }
         
     /**
      * Validates that the meta data is valid against the server meta data if we haven't yet done so.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 7a57103..524067d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -32,6 +32,7 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THRESHOLD_MILLISECONDS;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS;
+import static org.apache.phoenix.util.UpgradeUtil.getUpgradeSnapshotName;
 import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
@@ -288,9 +289,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             });
     
     private PMetaData newEmptyMetaData() {
-        long maxSizeBytes = props.getLong(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE);
-        return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, maxSizeBytes));
+        return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
     
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 25aca74..560b5d9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -144,9 +144,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     private PMetaData newEmptyMetaData() {
-        long maxSizeBytes = getProps().getLong(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE);
-        return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, maxSizeBytes);
+        return new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 22fa45a..d7c7c62 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -223,6 +223,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String LIMITED_QUERY_SERIAL_THRESHOLD = "phoenix.limited.query.serial.threshold";
 
     public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
+    
+    public static final String CLIENT_CACHE_ENCODING = "phoenix.table.client.cache.encoding";
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 83347c8..d874860 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ClientRpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
+import org.apache.phoenix.schema.PTableRefFactory;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -255,6 +256,8 @@ public class QueryServicesOptions {
     public static final float DEFAULT_LIMITED_QUERY_SERIAL_THRESHOLD = 0.2f;
     
     public static final boolean DEFAULT_INDEX_ASYNC_BUILD_ENABLED = true;
+    
+    public static final String DEFAULT_CLIENT_CACHE_ENCODING = PTableRefFactory.Encoding.OBJECT.toString();
 
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 7f97f4a..efe60ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -3147,7 +3147,7 @@ public class MetaDataClient {
         }
     }
 
-    private String dropColumnMutations(PTable table, List<PColumn> columnsToDrop, List<Mutation> tableMetaData) throws SQLException {
+    private String dropColumnMutations(PTable table, List<PColumn> columnsToDrop) throws SQLException {
         String tenantId = connection.getTenantId() == null ? "" : connection.getTenantId().getString();
         String schemaName = table.getSchemaName().getString();
         String tableName = table.getTableName().getString();
@@ -3263,7 +3263,9 @@ public class MetaDataClient {
                     columnsToDrop.add(new ColumnRef(columnRef.getTableRef(), columnToDrop.getPosition()));
                 }
 
-                dropColumnMutations(table, tableColumnsToDrop, tableMetaData);
+                dropColumnMutations(table, tableColumnsToDrop);
+                boolean removedIndexTableOrColumn=false;
+                Long timeStamp = table.isTransactional() ? tableRef.getTimeStamp() : null;
                 for (PTable index : table.getIndexes()) {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
@@ -3278,6 +3280,7 @@ public class MetaDataClient {
                             if (index.getViewIndexId()==null) 
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
+                            removedIndexTableOrColumn = true;
                         } 
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
@@ -3285,15 +3288,18 @@ public class MetaDataClient {
                             indexColumnsToDrop.add(indexColumn);
                             // add the index column to be dropped so that we actually delete the column values
                             columnsToDrop.add(new ColumnRef(new TableRef(index), indexColumn.getPosition()));
+                            removedIndexTableOrColumn = true;
                         }
                     }
                     if(!indexColumnsToDrop.isEmpty()) {
-                        incrementTableSeqNum(index, index.getType(), -indexColumnsToDrop.size(), null, null);
-                        dropColumnMutations(index, indexColumnsToDrop, tableMetaData);
+                        long indexTableSeqNum = incrementTableSeqNum(index, index.getType(), -indexColumnsToDrop.size(), null, null);
+                        dropColumnMutations(index, indexColumnsToDrop);
+                        long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
+                        connection.removeColumn(tenantId, index.getName().getString(),
+                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
-
                 }
-                Long timeStamp = table.isTransactional() ? tableRef.getTimeStamp() : null;
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                 connection.rollback();
 
@@ -3348,8 +3354,11 @@ public class MetaDataClient {
                     // If we've done any index metadata updates, don't bother trying to update
                     // client-side cache as it would be too painful. Just let it pull it over from
                     // the server when needed.
-                    if (tableColumnsToDrop.size() > 0 && indexesToDrop.isEmpty()) {
-                        connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
+                    if (tableColumnsToDrop.size() > 0) {
+                        if (removedIndexTableOrColumn)
+                            connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
+                        else  
+                            connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
                     if (table.getType() != PTableType.VIEW) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataCache.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataCache.java
new file mode 100644
index 0000000..9992adb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataCache.java
@@ -0,0 +1,221 @@
+/*
+ * 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.schema;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.phoenix.parse.PFunction;
+import org.apache.phoenix.parse.PSchema;
+import org.apache.phoenix.util.TimeKeeper;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.MinMaxPriorityQueue;
+import com.google.common.primitives.Longs;
+
+class PMetaDataCache implements Cloneable {
+    private static final int MIN_REMOVAL_SIZE = 3;
+    private static final Comparator<PTableRef> COMPARATOR = new Comparator<PTableRef>() {
+        @Override
+        public int compare(PTableRef tableRef1, PTableRef tableRef2) {
+            return Longs.compare(tableRef1.getLastAccessTime(), tableRef2.getLastAccessTime());
+        }
+    };
+    private static final MinMaxPriorityQueue.Builder<PTableRef> BUILDER = MinMaxPriorityQueue.orderedBy(COMPARATOR);
+    
+    private long currentByteSize;
+    private final long maxByteSize;
+    private final int expectedCapacity;
+    private final TimeKeeper timeKeeper;
+    private final PTableRefFactory tableRefFactory;
+
+    private final Map<PTableKey,PTableRef> tables;
+    final Map<PTableKey,PFunction> functions;
+    final Map<PTableKey,PSchema> schemas;
+    
+    private static Map<PTableKey,PTableRef> newMap(int expectedCapacity) {
+        // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
+        // safely across multiple threads (as the underlying collection is not thread safe).
+        // Instead, we track access time and prune it based on the copy we've made.
+        return Maps.newHashMapWithExpectedSize(expectedCapacity);
+    }
+
+    private static Map<PTableKey,PFunction> newFunctionMap(int expectedCapacity) {
+        // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
+        // safely across multiple threads (as the underlying collection is not thread safe).
+        // Instead, we track access time and prune it based on the copy we've made.
+        return Maps.newHashMapWithExpectedSize(expectedCapacity);
+    }
+
+    private static Map<PTableKey,PSchema> newSchemaMap(int expectedCapacity) {
+        // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
+        // safely across multiple threads (as the underlying collection is not thread safe).
+        // Instead, we track access time and prune it based on the copy we've made.
+        return Maps.newHashMapWithExpectedSize(expectedCapacity);
+    }
+
+    private Map<PTableKey,PTableRef> cloneMap(Map<PTableKey,PTableRef> tables, int expectedCapacity) {
+        Map<PTableKey,PTableRef> newTables = newMap(Math.max(tables.size(),expectedCapacity));
+        // Copy value so that access time isn't changing anymore
+        for (PTableRef tableAccess : tables.values()) {
+            newTables.put(tableAccess.getTable().getKey(), tableRefFactory.makePTableRef(tableAccess));
+        }
+        return newTables;
+    }
+
+    private static Map<PTableKey, PSchema> cloneSchemaMap(Map<PTableKey, PSchema> schemas, int expectedCapacity) {
+        Map<PTableKey, PSchema> newSchemas = newSchemaMap(Math.max(schemas.size(), expectedCapacity));
+        // Copy value so that access time isn't changing anymore
+        for (PSchema schema : schemas.values()) {
+            newSchemas.put(schema.getSchemaKey(), new PSchema(schema));
+        }
+        return newSchemas;
+    }
+
+    private static Map<PTableKey,PFunction> cloneFunctionsMap(Map<PTableKey,PFunction> functions, int expectedCapacity) {
+        Map<PTableKey,PFunction> newFunctions = newFunctionMap(Math.max(functions.size(),expectedCapacity));
+        for (PFunction functionAccess : functions.values()) {
+            newFunctions.put(functionAccess.getKey(), new PFunction(functionAccess));
+        }
+        return newFunctions;
+    }
+
+    PMetaDataCache(PMetaDataCache toClone) {
+        this.tableRefFactory = toClone.tableRefFactory;
+        this.timeKeeper = toClone.timeKeeper;
+        this.maxByteSize = toClone.maxByteSize;
+        this.currentByteSize = toClone.currentByteSize;
+        this.expectedCapacity = toClone.expectedCapacity;
+        this.tables = cloneMap(toClone.tables, expectedCapacity);
+        this.functions = cloneFunctionsMap(toClone.functions, expectedCapacity);
+        this.schemas = cloneSchemaMap(toClone.schemas, expectedCapacity);
+    }
+    
+    public PMetaDataCache(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper, PTableRefFactory tableRefFactory) {
+        this.currentByteSize = 0;
+        this.maxByteSize = maxByteSize;
+        this.expectedCapacity = initialCapacity;
+        this.tables = newMap(this.expectedCapacity);
+        this.functions = newFunctionMap(this.expectedCapacity);
+        this.timeKeeper = timeKeeper;
+        this.schemas = newSchemaMap(this.expectedCapacity);
+        this.tableRefFactory = tableRefFactory;
+    }
+    
+    public PTableRef get(PTableKey key) {
+        PTableRef tableAccess = this.tables.get(key);
+        if (tableAccess == null) {
+            return null;
+        }
+        tableAccess.setLastAccessTime(timeKeeper.getCurrentTime());
+        return tableAccess;
+    }
+    
+    @Override
+    public PMetaDataCache clone() {
+        return new PMetaDataCache(this);
+    }
+    
+    /**
+     * Used when the cache is growing past its max size to clone in a single pass.
+     * Removes least recently used tables to get size of cache below its max size by
+     * the overage amount.
+     */
+    public PMetaDataCache cloneMinusOverage(long overage) {
+        assert(overage > 0);
+        int nToRemove = Math.max(MIN_REMOVAL_SIZE, (int)Math.ceil((currentByteSize-maxByteSize) / ((double)currentByteSize / size())) + 1);
+        MinMaxPriorityQueue<PTableRef> toRemove = BUILDER.expectedSize(nToRemove).create();
+        PMetaDataCache newCache = new PMetaDataCache(this.size(), this.maxByteSize, this.timeKeeper, this.tableRefFactory);
+        
+        long toRemoveBytes = 0;
+        // Add to new cache, but track references to remove when done
+        // to bring cache at least overage amount below it's max size.
+        for (PTableRef tableRef : this.tables.values()) {
+            newCache.put(tableRef.getTable().getKey(), tableRefFactory.makePTableRef(tableRef));
+            toRemove.add(tableRef);
+            toRemoveBytes += tableRef.getEstimatedSize();
+            while (toRemoveBytes - toRemove.peekLast().getEstimatedSize() >= overage) {
+                PTableRef removedRef = toRemove.removeLast();
+                toRemoveBytes -= removedRef.getEstimatedSize();
+            }
+        }
+        for (PTableRef toRemoveRef : toRemove) {
+            newCache.remove(toRemoveRef.getTable().getKey());
+        }
+        return newCache;
+    }
+
+    PTable put(PTableKey key, PTableRef ref) {
+        currentByteSize += ref.getEstimatedSize();
+        PTableRef oldTableAccess = this.tables.put(key, ref);
+        PTable oldTable = null;
+        if (oldTableAccess != null) {
+            currentByteSize -= oldTableAccess.getEstimatedSize();
+            oldTable = oldTableAccess.getTable();
+        }
+        return oldTable;
+    }
+
+    public long getAge(PTableRef ref) {
+        return timeKeeper.getCurrentTime() - ref.getCreateTime();
+    }
+    
+    public PTable remove(PTableKey key) {
+        PTableRef value = this.tables.remove(key);
+        if (value == null) {
+            return null;
+        }
+        currentByteSize -= value.getEstimatedSize();
+        return value.getTable();
+    }
+    
+    public Iterator<PTable> iterator() {
+        final Iterator<PTableRef> iterator = this.tables.values().iterator();
+        return new Iterator<PTable>() {
+
+            @Override
+            public boolean hasNext() {
+                return iterator.hasNext();
+            }
+
+            @Override
+            public PTable next() {
+                return iterator.next().getTable();
+            }
+
+            @Override
+            public void remove() {
+                throw new UnsupportedOperationException();
+            }
+            
+        };
+    }
+
+    public int size() {
+        return this.tables.size();
+    }
+
+    public long getCurrentSize() {
+        return this.currentByteSize;
+    }
+
+    public long getMaxSize() {
+        return this.maxByteSize;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 5ffacca..7a78006 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -18,253 +18,50 @@
 package org.apache.phoenix.schema;
 
 import java.sql.SQLException;
-import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TimeKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.MinMaxPriorityQueue;
-import com.google.common.primitives.Longs;
 
 /**
- * 
- * Client-side cache of MetaData. Not thread safe, but meant to be used
- * in a copy-on-write fashion. Internally uses a LinkedHashMap that evicts
- * the oldest entries when size grows beyond the maxSize specified at
- * create time.
- *
+ * Client-side cache of MetaData, not thread safe. Internally uses a LinkedHashMap that evicts the
+ * oldest entries when size grows beyond the maxSize specified at create time.
  */
 public class PMetaDataImpl implements PMetaData {
-    private static final Logger logger = LoggerFactory.getLogger(PMetaDataImpl.class);
-        static class PMetaDataCache implements Cloneable {
-            private static final int MIN_REMOVAL_SIZE = 3;
-            private static final Comparator<PTableRef> COMPARATOR = new Comparator<PTableRef>() {
-                @Override
-                public int compare(PTableRef tableRef1, PTableRef tableRef2) {
-                    return Longs.compare(tableRef1.getLastAccessTime(), tableRef2.getLastAccessTime());
-                }
-            };
-            private static final MinMaxPriorityQueue.Builder<PTableRef> BUILDER = MinMaxPriorityQueue.orderedBy(COMPARATOR);
-            
-            private long currentByteSize;
-            private final long maxByteSize;
-            private final int expectedCapacity;
-            private final TimeKeeper timeKeeper;
-
-            private final Map<PTableKey,PTableRef> tables;
-            private final Map<PTableKey,PFunction> functions;
-            private final Map<PTableKey,PSchema> schemas;
-            
-            private static Map<PTableKey,PTableRef> newMap(int expectedCapacity) {
-                // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
-                // safely across multiple threads (as the underlying collection is not thread safe).
-                // Instead, we track access time and prune it based on the copy we've made.
-                return Maps.newHashMapWithExpectedSize(expectedCapacity);
-            }
-
-            private static Map<PTableKey,PFunction> newFunctionMap(int expectedCapacity) {
-                // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
-                // safely across multiple threads (as the underlying collection is not thread safe).
-                // Instead, we track access time and prune it based on the copy we've made.
-                return Maps.newHashMapWithExpectedSize(expectedCapacity);
-            }
-
-            private static Map<PTableKey,PSchema> newSchemaMap(int expectedCapacity) {
-                // Use regular HashMap, as we cannot use a LinkedHashMap that orders by access time
-                // safely across multiple threads (as the underlying collection is not thread safe).
-                // Instead, we track access time and prune it based on the copy we've made.
-                return Maps.newHashMapWithExpectedSize(expectedCapacity);
-            }
-
-            private static Map<PTableKey,PTableRef> cloneMap(Map<PTableKey,PTableRef> tables, int expectedCapacity) {
-                Map<PTableKey,PTableRef> newTables = newMap(Math.max(tables.size(),expectedCapacity));
-                // Copy value so that access time isn't changing anymore
-                for (PTableRef tableAccess : tables.values()) {
-                    newTables.put(tableAccess.getTable().getKey(), new PTableRef(tableAccess));
-                }
-                return newTables;
-            }
-
-            private static Map<PTableKey, PSchema> cloneSchemaMap(Map<PTableKey, PSchema> schemas, int expectedCapacity) {
-                Map<PTableKey, PSchema> newSchemas = newSchemaMap(Math.max(schemas.size(), expectedCapacity));
-                // Copy value so that access time isn't changing anymore
-                for (PSchema schema : schemas.values()) {
-                    newSchemas.put(schema.getSchemaKey(), new PSchema(schema));
-                }
-                return newSchemas;
-            }
-
-            private static Map<PTableKey,PFunction> cloneFunctionsMap(Map<PTableKey,PFunction> functions, int expectedCapacity) {
-                Map<PTableKey,PFunction> newFunctions = newFunctionMap(Math.max(functions.size(),expectedCapacity));
-                for (PFunction functionAccess : functions.values()) {
-                    newFunctions.put(functionAccess.getKey(), new PFunction(functionAccess));
-                }
-                return newFunctions;
-            }
-
-            private PMetaDataCache(PMetaDataCache toClone) {
-                this.timeKeeper = toClone.timeKeeper;
-                this.maxByteSize = toClone.maxByteSize;
-                this.currentByteSize = toClone.currentByteSize;
-                this.expectedCapacity = toClone.expectedCapacity;
-                this.tables = cloneMap(toClone.tables, expectedCapacity);
-                this.functions = cloneFunctionsMap(toClone.functions, expectedCapacity);
-                this.schemas = cloneSchemaMap(toClone.schemas, expectedCapacity);
-            }
-            
-            public PMetaDataCache(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper) {
-                this.currentByteSize = 0;
-                this.maxByteSize = maxByteSize;
-                this.expectedCapacity = initialCapacity;
-                this.tables = newMap(this.expectedCapacity);
-                this.functions = newFunctionMap(this.expectedCapacity);
-                this.timeKeeper = timeKeeper;
-                this.schemas = newSchemaMap(this.expectedCapacity);
-            }
-            
-            public PTableRef get(PTableKey key) {
-                PTableRef tableAccess = this.tables.get(key);
-                if (tableAccess == null) {
-                    return null;
-                }
-                tableAccess.setLastAccessTime(timeKeeper.getCurrentTime());
-                return tableAccess;
-            }
-            
-            @Override
-            public PMetaDataCache clone() {
-                return new PMetaDataCache(this);
-            }
-
-            /**
-             * Used when the cache is growing past its max size to clone in a single pass.
-             * Removes least recently used tables to get size of cache below its max size by
-             * the overage amount.
-             */
-            public PMetaDataCache cloneMinusOverage(long overage) {
-                assert(overage > 0);
-                int nToRemove = Math.max(MIN_REMOVAL_SIZE, (int)Math.ceil((currentByteSize-maxByteSize) / ((double)currentByteSize / size())) + 1);
-                MinMaxPriorityQueue<PTableRef> toRemove = BUILDER.expectedSize(nToRemove).create();
-                PMetaDataCache newCache = new PMetaDataCache(this.size(), this.maxByteSize, this.timeKeeper);
-                
-                long toRemoveBytes = 0;
-                // Add to new cache, but track references to remove when done
-                // to bring cache at least overage amount below it's max size.
-                for (PTableRef tableRef : this.tables.values()) {
-                    newCache.put(tableRef.getTable().getKey(), new PTableRef(tableRef));
-                    toRemove.add(tableRef);
-                    toRemoveBytes += tableRef.getEstSize();
-                    while (toRemoveBytes - toRemove.peekLast().getEstSize() >= overage) {
-                        PTableRef removedRef = toRemove.removeLast();
-                        toRemoveBytes -= removedRef.getEstSize();
-                    }
-                }
-                for (PTableRef toRemoveRef : toRemove) {
-                    newCache.remove(toRemoveRef.getTable().getKey());
-                }
-                return newCache;
-            }
-
-            private PTable put(PTableKey key, PTableRef ref) {
-                currentByteSize += ref.getEstSize();
-                PTableRef oldTableAccess = this.tables.put(key, ref);
-                PTable oldTable = null;
-                if (oldTableAccess != null) {
-                    currentByteSize -= oldTableAccess.getEstSize();
-                    oldTable = oldTableAccess.getTable();
-                }
-                return oldTable;
-            }
-
-            public PTable put(PTableKey key, PTable value, long resolvedTime) {
-                return put(key, new PTableRef(value, timeKeeper.getCurrentTime(), resolvedTime));
-            }
-            
-            public PTable putDuplicate(PTableKey key, PTable value, long resolvedTime) {
-                return put(key, new PTableRef(value, timeKeeper.getCurrentTime(), 0, resolvedTime));
-            }
-            
-            public long getAge(PTableRef ref) {
-                return timeKeeper.getCurrentTime() - ref.getCreateTime();
-            }
-            
-            public PTable remove(PTableKey key) {
-                PTableRef value = this.tables.remove(key);
-                if (value == null) {
-                    return null;
-                }
-                currentByteSize -= value.getEstSize();
-                return value.getTable();
-            }
-            
-            public Iterator<PTable> iterator() {
-                final Iterator<PTableRef> iterator = this.tables.values().iterator();
-                return new Iterator<PTable>() {
-
-                    @Override
-                    public boolean hasNext() {
-                        return iterator.hasNext();
-                    }
-
-                    @Override
-                    public PTable next() {
-                        return iterator.next().getTable();
-                    }
-
-                    @Override
-                    public void remove() {
-                        throw new UnsupportedOperationException();
-                    }
-                    
-                };
-            }
-
-            public int size() {
-                return this.tables.size();
-            }
-
-            public long getCurrentSize() {
-                return this.currentByteSize;
-            }
-
-            public long getMaxSize() {
-                return this.maxByteSize;
-            }
-        }
-            
-    private PMetaDataCache metaData;
     
-    @VisibleForTesting
-    public PMetaDataCache getMetaData() {
-        return metaData;
-    }
+    private PMetaDataCache metaData;
+    private final TimeKeeper timeKeeper;
+    private final PTableRefFactory tableRefFactory;
     
-    public PMetaDataImpl(int initialCapacity, long maxByteSize) {
-        this.metaData = new PMetaDataCache(initialCapacity, maxByteSize, TimeKeeper.SYSTEM);
+    public PMetaDataImpl(int initialCapacity, ReadOnlyProps props) {
+        this(initialCapacity, TimeKeeper.SYSTEM, props);
     }
 
-    public PMetaDataImpl(int initialCapacity, long maxByteSize, TimeKeeper timeKeeper) {
-        this.metaData = new PMetaDataCache(initialCapacity, maxByteSize, timeKeeper);
+    public PMetaDataImpl(int initialCapacity, TimeKeeper timeKeeper, ReadOnlyProps props) {
+        this(new PMetaDataCache(initialCapacity, props.getLong(
+            QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB,
+            QueryServicesOptions.DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE), timeKeeper,
+                PTableRefFactory.getFactory(props)), timeKeeper, PTableRefFactory.getFactory(props));
     }
 
-    private PMetaDataImpl(PMetaDataCache metaData) {
+    private PMetaDataImpl(PMetaDataCache metaData, TimeKeeper timeKeeper, PTableRefFactory tableRefFactory) {
+        this.timeKeeper = timeKeeper;
         this.metaData = metaData;
+        this.tableRefFactory = tableRefFactory;
     }
-    
+
     @Override
     public PMetaDataImpl clone() {
-        return new PMetaDataImpl(new PMetaDataCache(this.metaData));
+        return new PMetaDataImpl(new PMetaDataCache(this.metaData), this.timeKeeper, this.tableRefFactory);
     }
     
     @Override
@@ -292,18 +89,20 @@ public class PMetaDataImpl implements PMetaData {
 
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException {
-    	metaData.putDuplicate(table.getKey(), table, resolvedTimestamp);
+    	metaData.put(table.getKey(), tableRefFactory.makePTableRef(table, this.timeKeeper.getCurrentTime(), resolvedTimestamp));
     }
 
     @Override
     public void addTable(PTable table, long resolvedTime) throws SQLException {
+        PTableRef tableRef = tableRefFactory.makePTableRef(table, this.timeKeeper.getCurrentTime(), resolvedTime);
         int netGain = 0;
         PTableKey key = table.getKey();
         PTableRef oldTableRef = metaData.get(key);
         if (oldTableRef != null) {
-            netGain -= oldTableRef.getEstSize();
+            netGain -= oldTableRef.getEstimatedSize();
         }
         PTable newParentTable = null;
+        PTableRef newParentTableRef = null;
         long parentResolvedTimestamp = resolvedTime;
         if (table.getParentName() != null) { // Upsert new index table into parent data table list
             String parentName = table.getParentName().getString();
@@ -321,25 +120,26 @@ public class PMetaDataImpl implements PMetaData {
                     }
                 }
                 newIndexes.add(table);
-                netGain -= oldParentRef.getEstSize();
+                netGain -= oldParentRef.getEstimatedSize();
                 newParentTable = PTableImpl.makePTable(oldParentRef.getTable(), table.getTimeStamp(), newIndexes);
-                netGain += newParentTable.getEstimatedSize();
+                newParentTableRef = tableRefFactory.makePTableRef(newParentTable, this.timeKeeper.getCurrentTime(), parentResolvedTimestamp);
+                netGain += newParentTableRef.getEstimatedSize();
             }
         }
         if (newParentTable == null) { // Don't count in gain if we found a parent table, as its accounted for in newParentTable
-            netGain += table.getEstimatedSize();
+            netGain += tableRef.getEstimatedSize();
         }
         long overage = metaData.getCurrentSize() + netGain - metaData.getMaxSize();
         metaData = overage <= 0 ? metaData : metaData.cloneMinusOverage(overage);
         
         if (newParentTable != null) { // Upsert new index table into parent data table list
-            metaData.put(newParentTable.getKey(), newParentTable, parentResolvedTimestamp);
-            metaData.putDuplicate(table.getKey(), table, resolvedTime);
+            metaData.put(newParentTable.getKey(), newParentTableRef);
+            metaData.put(table.getKey(), tableRef);
         } else {
-            metaData.put(table.getKey(), table, resolvedTime);
+            metaData.put(table.getKey(), tableRef);
         }
         for (PTable index : table.getIndexes()) {
-            metaData.putDuplicate(index.getKey(), index, resolvedTime);
+            metaData.put(index.getKey(), tableRefFactory.makePTableRef(index, this.timeKeeper.getCurrentTime(), resolvedTime));
         }
     }
 
@@ -401,7 +201,7 @@ public class PMetaDataImpl implements PMetaData {
                                 parentTableRef.getTable(),
                                 tableTimeStamp == HConstants.LATEST_TIMESTAMP ? parentTableRef.getTable().getTimeStamp() : tableTimeStamp,
                                 newIndexes);
-                        metaData.put(parentTable.getKey(), parentTable, parentTableRef.getResolvedTimeStamp());
+                        metaData.put(parentTable.getKey(), tableRefFactory.makePTableRef(parentTable, this.timeKeeper.getCurrentTime(), parentTableRef.getResolvedTimeStamp()));
                         break;
                     }
                 }
@@ -444,7 +244,7 @@ public class PMetaDataImpl implements PMetaData {
             
             table = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, columns);
         }
-        tables.put(table.getKey(), table, resolvedTime);
+        tables.put(table.getKey(), tableRefFactory.makePTableRef(table, this.timeKeeper.getCurrentTime(), resolvedTime));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 92c49f9..c485a30 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -77,7 +77,6 @@ import com.google.common.collect.Maps;
  * storing data in a single column (ColumnLayout.SINGLE) or in
  * multiple columns (ColumnLayout.MULTI).
  *
- * TODO add hashCode and equal methods to check equality of two PTableImpl objects.
  * @since 0.1
  */
 public class PTableImpl implements PTable {
@@ -1073,9 +1072,9 @@ public class PTableImpl implements PTable {
       List<PName> physicalNames = Collections.emptyList();
       if (tableType == PTableType.VIEW) {
         viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
-        if(table.hasViewStatement()){
-          viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
-        }
+      }
+      if(table.hasViewStatement()){
+        viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
       }
       if (tableType == PTableType.VIEW || viewIndexId != null) {
         physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
@@ -1181,6 +1180,8 @@ public class PTableImpl implements PTable {
       builder.setTransactional(table.isTransactional());
       if(table.getType() == PTableType.VIEW){
         builder.setViewType(ByteStringer.wrap(new byte[]{table.getViewType().getSerializedValue()}));
+      }
+      if(table.getViewStatement()!=null){
         builder.setViewStatement(ByteStringer.wrap(PVarchar.INSTANCE.toBytes(table.getViewStatement())));
       }
       if(table.getType() == PTableType.VIEW || table.getViewIndexId() != null){
@@ -1244,4 +1245,24 @@ public class PTableImpl implements PTable {
     public boolean isAppendOnlySchema() {
         return isAppendOnlySchema;
     }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((key == null) ? 0 : key.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) return true;
+        if (obj == null) return false;
+        if (getClass() != obj.getClass()) return false;
+        PTableImpl other = (PTableImpl) obj;
+        if (key == null) {
+            if (other.key != null) return false;
+        } else if (!key.equals(other.key)) return false;
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
index c4bc510..0a601b0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
@@ -17,28 +17,19 @@
  */
 package org.apache.phoenix.schema;
 
-public class PTableRef {
-    private final PTable table;
-    private final int estSize;
-    private final long createTime;
-    private final long resolvedTimeStamp;
-	private volatile long lastAccessTime;
+public abstract class PTableRef {
     
-    public PTableRef(PTable table, long lastAccessTime, int estSize, long resolvedTime) {
-        this.table = table;
+    protected final int estSize;
+    protected final long createTime;
+    protected final long resolvedTimeStamp;
+    protected volatile long lastAccessTime;
+    
+    public PTableRef(long lastAccessTime, long resolvedTime, int estimatedSize) {
         this.lastAccessTime = lastAccessTime;
-        this.estSize = estSize;
+        this.estSize = estimatedSize;
         this.resolvedTimeStamp = resolvedTime;
         this.createTime = lastAccessTime;
     }
-
-    public PTableRef(PTable table, long lastAccessTime, long resolvedTime) {
-        this (table, lastAccessTime, table.getEstimatedSize(), resolvedTime);
-    }
-
-    public PTableRef(PTableRef tableRef) {
-        this (tableRef.table, tableRef.lastAccessTime, tableRef.estSize, tableRef.resolvedTimeStamp);
-    }
     
     /**
      * Tracks how long this entry has been in the cache
@@ -48,23 +39,22 @@ public class PTableRef {
         return createTime;
     }
     
-    public PTable getTable() {
-		return table;
-	}
+    public abstract PTable getTable();
 
-	public long getResolvedTimeStamp() {
-		return resolvedTimeStamp;
-	}
-	
-    public int getEstSize() {
-		return estSize;
-	}
+    public long getResolvedTimeStamp() {
+        return resolvedTimeStamp;
+    }
+    
+    public int getEstimatedSize() {
+        return estSize;
+    }
 
-	public long getLastAccessTime() {
-		return lastAccessTime;
-	}
+    public long getLastAccessTime() {
+        return lastAccessTime;
+    }
 
-	public void setLastAccessTime(long lastAccessTime) {
-		this.lastAccessTime = lastAccessTime;
-	}
+    public void setLastAccessTime(long lastAccessTime) {
+        this.lastAccessTime = lastAccessTime;
+    }
+	
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefFactory.java
new file mode 100644
index 0000000..14eb235
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefFactory.java
@@ -0,0 +1,52 @@
+/*
+ * 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.schema;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+public class PTableRefFactory {
+    public PTableRef makePTableRef(PTable table, long lastAccessTime, long resolvedTime) {
+        return new PTableRefImpl(table, lastAccessTime, resolvedTime, table.getEstimatedSize());
+    }
+
+    public PTableRef makePTableRef(PTableRef tableRef) {
+        return new PTableRefImpl(tableRef);
+    }
+
+    private static final PTableRefFactory INSTANCE = new PTableRefFactory();
+
+    public static enum Encoding {
+        OBJECT, PROTOBUF
+    };
+
+    public static PTableRefFactory getFactory(ReadOnlyProps props) {
+        String encodingEnumString =
+                props.get(QueryServices.CLIENT_CACHE_ENCODING,
+                    QueryServicesOptions.DEFAULT_CLIENT_CACHE_ENCODING);
+        Encoding encoding = Encoding.valueOf(encodingEnumString.toUpperCase());
+        switch (encoding) {
+        case PROTOBUF:
+            return SerializedPTableRefFactory.getFactory();
+        case OBJECT:
+        default:
+            return INSTANCE;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefImpl.java
new file mode 100644
index 0000000..ffc5c2b
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRefImpl.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema;
+
+
+public class PTableRefImpl extends PTableRef {
+    
+    private final PTable table;
+    
+    public PTableRefImpl(PTable table, long lastAccessTime, long resolvedTime, int estimatedSize) {
+        super(lastAccessTime, resolvedTime, estimatedSize);
+        this.table = table;
+    }
+
+    public PTableRefImpl(PTableRef tableRef) {
+        super(tableRef.getLastAccessTime(), tableRef.getResolvedTimeStamp(), tableRef.getEstimatedSize());
+        this.table = tableRef.getTable();
+    }
+
+    @Override
+    public PTable getTable() {
+        return table;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRef.java
new file mode 100644
index 0000000..a57fc72
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRef.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema;
+
+import java.io.IOException;
+
+import org.apache.phoenix.coprocessor.generated.PTableProtos;
+
+public class SerializedPTableRef extends PTableRef {
+
+    private final byte[] tableBytes;
+
+    public SerializedPTableRef(byte[] tableBytes, long lastAccessTime, long resolvedTime, int estimatedSize) {
+        super(lastAccessTime, resolvedTime, tableBytes.length);
+        this.tableBytes = tableBytes;
+    }
+
+    public SerializedPTableRef(PTableRef tableRef) {
+        super(tableRef.getLastAccessTime(), tableRef.getResolvedTimeStamp(), tableRef.getEstimatedSize());
+        this.tableBytes = ((SerializedPTableRef)tableRef).tableBytes;
+    }
+
+    @Override
+    public PTable getTable() {
+        try {
+            return PTableImpl.createFromProto(PTableProtos.PTable.parseFrom(tableBytes));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRefFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRefFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRefFactory.java
new file mode 100644
index 0000000..5da1fd6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SerializedPTableRefFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.schema;
+
+class SerializedPTableRefFactory extends PTableRefFactory {
+    @Override
+    public PTableRef makePTableRef(PTable table, long lastAccessTime, long resolvedTime) {
+        byte[] serializedBytes = PTableImpl.toProto(table).toByteArray();
+        return new SerializedPTableRef(serializedBytes, lastAccessTime, resolvedTime, table.getEstimatedSize());
+    }
+    
+    @Override
+    public PTableRef makePTableRef(PTableRef tableRef) {
+        return new SerializedPTableRef(tableRef);
+    }
+    
+    private static final SerializedPTableRefFactory INSTANCE = new SerializedPTableRefFactory();
+    
+    public static PTableRefFactory getFactory() {
+        return INSTANCE;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0ae8025/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
index ef88c8c..a5660db 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
@@ -21,12 +21,16 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
 import java.sql.SQLException;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TimeKeeper;
 import org.junit.Test;
 
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
 public class PMetaDataImplTest {
@@ -72,9 +76,11 @@ public class PMetaDataImplTest {
     
     @Test
     public void testEviction() throws Exception {
-        long maxSize = 10;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB, "10");
+        props.put(QueryServices.CLIENT_CACHE_ENCODING, "object");
+        PMetaData metaData = new PMetaDataImpl(5, timeKeeper,  new ReadOnlyProps(props));
         addToTable(metaData, "a", 5, timeKeeper);
         assertEquals(1, metaData.size());
         addToTable(metaData, "b", 4, timeKeeper);
@@ -116,9 +122,11 @@ public class PMetaDataImplTest {
 
     @Test
     public void shouldNotEvictMoreEntriesThanNecessary() throws Exception {
-        long maxSize = 5;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB, "5");
+        props.put(QueryServices.CLIENT_CACHE_ENCODING, "object");
+        PMetaData metaData = new PMetaDataImpl(5, timeKeeper,  new ReadOnlyProps(props));
         addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
         addToTable(metaData, "b", 1, timeKeeper);
@@ -136,9 +144,11 @@ public class PMetaDataImplTest {
 
     @Test
     public void shouldAlwaysKeepAtLeastOneEntryEvenIfTooLarge() throws Exception {
-        long maxSize = 5;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB, "5");
+        props.put(QueryServices.CLIENT_CACHE_ENCODING, "object");
+        PMetaData metaData = new PMetaDataImpl(5, timeKeeper,  new ReadOnlyProps(props));
         addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
         addToTable(metaData, "b", 1, timeKeeper);
@@ -157,9 +167,11 @@ public class PMetaDataImplTest {
 
     @Test
     public void shouldAlwaysKeepOneEntryIfMaxSizeIsZero() throws Exception {
-        long maxSize = 0;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
-        PMetaData metaData = new PMetaDataImpl(0, maxSize, timeKeeper);
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB, "0");
+        props.put(QueryServices.CLIENT_CACHE_ENCODING, "object");
+        PMetaData metaData = new PMetaDataImpl(5, timeKeeper,  new ReadOnlyProps(props));
         addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
         addToTable(metaData, "b", 1, timeKeeper);
@@ -178,9 +190,11 @@ public class PMetaDataImplTest {
 
     @Test
     public void testAge() throws Exception {
-        long maxSize = 10;
         TestTimeKeeper timeKeeper = new TestTimeKeeper();
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
+        props.put(QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB, "10");
+        props.put(QueryServices.CLIENT_CACHE_ENCODING, "object");
+        PMetaData metaData = new PMetaDataImpl(5, timeKeeper,  new ReadOnlyProps(props));
         String tableName = "a";
         addToTable(metaData, tableName, 1, timeKeeper);
         PTableRef aTableRef = metaData.getTableRef(new PTableKey(null,tableName));


[47/50] [abbrv] phoenix git commit: PHOENIX-3189 Ensure that there is no invalid cached state in KerberosName

Posted by ma...@apache.org.
PHOENIX-3189 Ensure that there is no invalid cached state in KerberosName


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

Branch: refs/heads/calcite
Commit: 1ce9845b8f6c2b456fbab913cd1bd647c6c4d281
Parents: 2dc6be4
Author: Josh Elser <el...@apache.org>
Authored: Wed Aug 31 22:28:45 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Aug 31 22:29:40 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/jdbc/SecureUserConnectionsTest.java | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1ce9845b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
index 6a33142..a14e47b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/SecureUserConnectionsTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -35,6 +36,8 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.query.ConfigurationFactory;
 import org.apache.phoenix.util.InstanceResolver;
@@ -94,6 +97,16 @@ public class SecureUserConnectionsTest {
                 return copy;
             }
         });
+        updateDefaultRealm();
+    }
+
+    private static void updateDefaultRealm() throws Exception {
+        // (at least) one other phoenix test triggers the caching of this field before the KDC is up
+        // which causes principal parsing to fail.
+        Field f = KerberosName.class.getDeclaredField("defaultRealm");
+        f.setAccessible(true);
+        // Default realm for MiniKDC
+        f.set(null, "EXAMPLE.COM");
     }
 
     @AfterClass


[10/50] [abbrv] phoenix git commit: Set version to 4.9.0-HBase-1.2-SNAPSHOT after release

Posted by ma...@apache.org.
Set version to 4.9.0-HBase-1.2-SNAPSHOT after release


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

Branch: refs/heads/calcite
Commit: 137eb98d4c78f7edaa8fa5e7388476b4a96afcde
Parents: 4702080
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Aug 4 16:54:29 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Aug 4 16:54:29 2016 +0530

----------------------------------------------------------------------
 phoenix-assembly/pom.xml           | 2 +-
 phoenix-client/pom.xml             | 2 +-
 phoenix-core/pom.xml               | 2 +-
 phoenix-flume/pom.xml              | 2 +-
 phoenix-hive/pom.xml               | 2 +-
 phoenix-pherf/pom.xml              | 2 +-
 phoenix-pig/pom.xml                | 2 +-
 phoenix-queryserver-client/pom.xml | 2 +-
 phoenix-queryserver/pom.xml        | 2 +-
 phoenix-server/pom.xml             | 2 +-
 phoenix-spark/pom.xml              | 2 +-
 phoenix-tracing-webapp/pom.xml     | 2 +-
 pom.xml                            | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 05ec173..4826e1d 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index cda26c7..2edf9d5 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 17a96fd..ab2f80e 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index b6e5ff9..82be63e 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-hive/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index 21ccab7..6203754 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index d013003..86880a3 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.8.0-HBase-1.2</version>
+		<version>4.9.0-HBase-1.2-SNAPSHOT</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 709f2f8..860f80f 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index a310961..a4e9b82e 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index 25bf6bf..10f918d 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index af756a4..8b641f0 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 3177b09..5f20e3a 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.8.0-HBase-1.2</version>
+    <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 9747c3e..3533682 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.8.0-HBase-1.2</version>
+      <version>4.9.0-HBase-1.2-SNAPSHOT</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/137eb98d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 79cb870..6c96293 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.8.0-HBase-1.2</version>
+  <version>4.9.0-HBase-1.2-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>


[14/50] [abbrv] phoenix git commit: PHOENIX-3149 Local index got corrupted if intermediate compaction happen during the split.(Sergey Soldatov)

Posted by ma...@apache.org.
PHOENIX-3149 Local index got corrupted if intermediate compaction happen during the split.(Sergey Soldatov)


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

Branch: refs/heads/calcite
Commit: ba82b1cb5a14c2cf109deb8a862389142d92f541
Parents: bb0d89b
Author: Ankit Singhal <an...@gmail.com>
Authored: Sun Aug 7 13:50:06 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Sun Aug 7 13:51:23 2016 +0530

----------------------------------------------------------------------
 .../hbase/regionserver/IndexHalfStoreFileReaderGenerator.java       | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ba82b1cb/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index 4a42984..7cd6405 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -186,7 +186,6 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
             long earliestPutTs, InternalScanner s, CompactionRequest request) throws IOException {
         if (!store.getFamily().getNameAsString()
                 .startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)
-                || !scanType.equals(ScanType.COMPACT_DROP_DELETES)
                 || s != null
                 || !store.hasReferences()) {
             return s;


[03/50] [abbrv] phoenix git commit: PHOENIX-3138 Integration tests are timing out

Posted by ma...@apache.org.
PHOENIX-3138 Integration tests are timing out


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

Branch: refs/heads/calcite
Commit: 92c7faf766a4e96a9d99fe025b0bb0f56b66ab32
Parents: ca7c36d
Author: James Taylor <ja...@apache.org>
Authored: Tue Aug 2 14:56:24 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Tue Aug 2 16:26:40 2016 -0700

----------------------------------------------------------------------
 .../src/it/java/org/apache/phoenix/end2end/index/IndexIT.java     | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/92c7faf7/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index df45ecb..aba7dac 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -53,7 +53,6 @@ import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.coprocessor.generated.PTableProtos.PTableType;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
 import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -83,7 +82,7 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class IndexIT extends BaseHBaseManagedTimeTableReuseIT {
+public class IndexIT extends BaseHBaseManagedTimeIT {
     private static final long MAX_WAIT_FOR_ASYNC_INDEX_BUILD = 30000;
     	
 	private final boolean localIndex;


[38/50] [abbrv] phoenix git commit: Modify phoenix IT tests to extend BaseHBaseManagedTimeTableReuseIT (Prakul Agarwal, Samarth Jain)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
index b78bb63..7466cd3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
@@ -45,15 +45,13 @@ import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
 
-public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
+public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeTableReuseIT {
 
     private static final String DATATYPE_TABLE = "DATATYPE";
     private static final String DATATYPES_CSV_VALUES = "CKEY, CVARCHAR, CCHAR, CINTEGER, CDECIMAL, CUNSIGNED_INT, CBOOLEAN, CBIGINT, CUNSIGNED_LONG, CTIME, CDATE\n"
             + "KEY1,A,A,2147483647,1.1,0,TRUE,9223372036854775807,0,1990-12-31 10:59:59,1999-12-31 23:59:59\n"
             + "KEY2,B,B,-2147483648,-1.1,2147483647,FALSE,-9223372036854775808,9223372036854775807,2000-01-01 00:00:01,2012-02-29 23:59:59\n"
             + "KEY3,,,,,,,,,,\n";
-    private static final String STOCK_TABLE = "STOCK_SYMBOL";
-    private static final String STOCK_TABLE_MULTI = "STOCK_SYMBOL_MULTI";
     private static final String STOCK_CSV_VALUES = "AAPL,APPLE Inc.\n"
             + "CRM,SALESFORCE\n" + "GOOG,Google\n"
             + "HOG,Harlet-Davidson Inc.\n" + "HPQ,Hewlett Packard\n"
@@ -105,9 +103,9 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
-
+            String stockTableName = generateRandomString();
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl()).unwrap(
                     PhoenixConnection.class);
@@ -115,14 +113,14 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Collections.<String> emptyList(), true);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES_WITH_HEADER));
 
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_CSV_VALUES_WITH_HEADER), csvUtil.getFormat());
@@ -149,9 +147,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         PhoenixConnection globalConn = null;
         PhoenixConnection tenantConn = null;
         try {
+            String stockTableMultiName = generateRandomString();
 
             // Create table using the global connection
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE_MULTI
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableMultiName
                     + "(TENANT_ID VARCHAR NOT NULL, SYMBOL VARCHAR NOT NULL, COMPANY VARCHAR," +
                     " CONSTRAINT PK PRIMARY KEY(TENANT_ID,SYMBOL)) MULTI_TENANT = true;";
             globalConn = DriverManager.getConnection(getUrl()).unwrap(
@@ -164,14 +163,14 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     PhoenixConnection.class);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(tenantConn, STOCK_TABLE_MULTI,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(tenantConn, stockTableMultiName,
                     Collections.<String> emptyList(), true);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES_WITH_HEADER));
 
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = tenantConn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE_MULTI);
+                            + stockTableMultiName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_CSV_VALUES_WITH_HEADER), csvUtil.getFormat());
@@ -197,9 +196,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
 
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl()).unwrap(
                     PhoenixConnection.class);
@@ -207,14 +207,14 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert TDV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,Collections.<String> emptyList()
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,Collections.<String> emptyList()
                     , true, '\t', '"', null, CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
             csvUtil.upsert(new StringReader(STOCK_TDV_VALUES_WITH_HEADER));
 
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_TDV_VALUES_WITH_HEADER), csvUtil.getFormat());
@@ -240,8 +240,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl()).unwrap(
                     PhoenixConnection.class);
@@ -249,7 +251,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.<String> asList(STOCK_COLUMNS), true,
                     '1', '2', '3', CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES_WITH_DELIMITER));
@@ -257,7 +259,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_CSV_VALUES_WITH_DELIMITER), csvUtil.getFormat());
@@ -283,8 +285,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -292,7 +296,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.<String> asList(STOCK_COLUMNS), true);
             // no header
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES));
@@ -300,7 +304,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_CSV_VALUES), csvUtil.getFormat());
@@ -327,8 +331,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -336,14 +342,14 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     null, true);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES));
 
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(
                     STOCK_CSV_VALUES), csvUtil.getFormat());
@@ -370,8 +376,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -379,14 +387,14 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file, not strict
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.asList(STOCK_COLUMNS_WITH_BOGUS), false);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES));
 
             // Compare Phoenix ResultSet with CSV file content
             PreparedStatement statement = conn
                     .prepareStatement("SELECT SYMBOL, COMPANY FROM "
-                            + STOCK_TABLE);
+                            + stockTableName);
             ResultSet phoenixResultSet = statement.executeQuery();
             parser = new CSVParser(new StringReader(STOCK_CSV_VALUES),
                     csvUtil.getFormat());
@@ -412,8 +420,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY_ID BIGINT);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -421,7 +431,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file in strict mode
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.asList("SYMBOL", "COMPANY_ID"), true);
             try {
                 csvUtil.upsert(new StringReader(STOCK_CSV_VALUES));
@@ -444,8 +454,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -453,7 +465,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.asList("FOO", "BAR"), false);
 
             try {
@@ -464,7 +476,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                         e.getMessage(),
                         e.getMessage()
                                 .contains(
-                                        "ERROR 504 (42703): Undefined column. columnName=STOCK_SYMBOL.[FOO, BAR]"));
+                                        "ERROR 504 (42703): Undefined column. columnName=" + stockTableName + ".[FOO, BAR]"));
             }
         } finally {
             if (parser != null)
@@ -479,8 +491,10 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
         CSVParser parser = null;
         PhoenixConnection conn = null;
         try {
+            String stockTableName = generateRandomString();
+
             // Create table
-            String statements = "CREATE TABLE IF NOT EXISTS " + STOCK_TABLE
+            String statements = "CREATE TABLE IF NOT EXISTS " + stockTableName
                     + "(SYMBOL VARCHAR NOT NULL PRIMARY KEY, COMPANY VARCHAR);";
             conn = DriverManager.getConnection(getUrl())
                     .unwrap(PhoenixConnection.class);
@@ -488,7 +502,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                     new StringReader(statements), null);
 
             // Upsert CSV file
-            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
+            CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, stockTableName,
                     Arrays.asList(STOCK_COLUMNS_WITH_BOGUS), true);
             try {
                 csvUtil.upsert(new StringReader(STOCK_CSV_VALUES));
@@ -498,7 +512,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
                         e.getMessage(),
                         e.getMessage()
                                 .contains(
-                                        "ERROR 504 (42703): Undefined column. columnName=STOCK_SYMBOL.BOGUS"));
+                                        "ERROR 504 (42703): Undefined column. columnName=" + stockTableName + ".BOGUS"));
             }
         } finally {
             if (parser != null)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index 68165e2..0c270f2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -38,15 +38,15 @@ import org.junit.Assert;
 import org.junit.Test;
 
 
-public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
+public class CoalesceFunctionIT extends BaseHBaseManagedTimeTableReuseIT {
 
     @Test
     public void testCoalesce() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), getUrl());
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String query = "SELECT entity_id, a_integer + COALESCE(x_integer,1) FROM ATABLE WHERE organization_id = ? AND a_integer >= 6 AND a_integer <= 7";
+        String query = "SELECT entity_id, a_integer + COALESCE(x_integer,1) FROM " + tableName + " WHERE organization_id = ? AND a_integer >= 6 AND a_integer <= 7";
         PreparedStatement statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         ResultSet rs = statement.executeQuery();
@@ -66,21 +66,21 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void coalesceWithSumExplicitLong() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    COUNT BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + "COALESCE(SUM(COUNT), CAST(0 AS BIGINT)) " //explicitly def long
-                + "FROM TEST_COALESCE "
-                + "GROUP BY ID");
+                + "FROM  " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(0, rs.getLong(1));
@@ -90,21 +90,21 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void coalesceWithSumImplicitLong() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    COUNT BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + "COALESCE(SUM(COUNT), 0) " // no long def
-                + "FROM TEST_COALESCE "
-                + "GROUP BY ID");
+                + "FROM " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(0, rs.getLong(1));
@@ -114,21 +114,21 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void coalesceWithSecondParamAsExpression() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    COUNT BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + "COALESCE(SUM(COUNT), SUM(ID)) " // second param as expression
-                + "FROM TEST_COALESCE "
-                + "GROUP BY ID");
+                + "FROM " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(2, rs.getLong(1));
@@ -138,21 +138,22 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void nonTypedSecondParameterLong() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    COUNT BIGINT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO "  + tableName + "(ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + "COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 0) " //second param is int
-                + "FROM TEST_COALESCE "
-                + "GROUP BY ID");
+                + "FROM " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(0, rs.getLong(1));
@@ -162,21 +163,21 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void nonTypedSecondParameterUnsignedDataTypes() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-
-        String ddl = "CREATE TABLE TEST_COALESCE ("
+        String tableName = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    COUNT UNSIGNED_INT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE (ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         //second param to coalesce is signed int
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + " COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 1) "
-                + " FROM TEST_COALESCE" 
+                + " FROM " + tableName
                 + " GROUP BY ID");
 
         assertTrue(rs.next());
@@ -187,17 +188,18 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testWithNthValueAggregationFunction() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE TEST_NTH("
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    DATE TIMESTAMP NOT NULL, "
                 + "    COUNT BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID, DATE))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_NTH(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 1)");
-        conn.createStatement().execute("UPSERT INTO TEST_NTH(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 2)");
-        conn.createStatement().execute("UPSERT INTO TEST_NTH(ID, DATE, COUNT) VALUES(2, CURRENT_TIME(), 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(2, CURRENT_TIME(), 1)");
         conn.commit();
 
         //second param to coalesce is signed int
@@ -207,8 +209,8 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
                 + "            NTH_VALUE(COUNT, 2000)" // should evaluate null
                 + "            WITHIN GROUP (ORDER BY COUNT DESC),"
                 + "       0)"
-                + "FROM TEST_NTH "
-                + "GROUP BY ID");
+                + "FROM  " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(0, rs.getLong(1));
@@ -218,22 +220,23 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void wrongDataTypeOfSecondParameter() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID UNSIGNED_INT NOT NULL, "
                 + "    COUNT UNSIGNED_INT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
         conn.commit();
 
         try {
             conn.createStatement().executeQuery(
                     "SELECT "
                     + "COALESCE(MIN(COUNT), -1) " // invalid value for UNSIGNED_INT
-                    + "FROM TEST_COALESCE "
-                    + "GROUP BY ID");
+                    + "FROM " + tableName
+                    + " GROUP BY ID");
 
             Assert.fail("CANNOT CONVERT TYPE exception expected");
         } catch (SQLException e) {
@@ -244,22 +247,23 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testImplicitSecondArgCastingException() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID INTEGER NOT NULL, "
                 + "    COUNT UNSIGNED_INT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(-2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(-2, null)");
         conn.commit();
 
         try {
             ResultSet rs = conn.createStatement().executeQuery(
                     "SELECT "
                     + "COALESCE(MIN(COUNT), ID) "
-                    + "FROM TEST_COALESCE "
-                    + "GROUP BY ID");
+                    + "FROM " + tableName
+                    + " GROUP BY ID");
 
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
@@ -272,21 +276,22 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testImplicitSecondArgCasting() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = generateRandomString();
 
-        String ddl = "CREATE TABLE TEST_COALESCE("
+        String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID DOUBLE NOT NULL, "
                 + "    COUNT INTEGER " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO TEST_COALESCE(ID, COUNT) VALUES(2.0, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2.0, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + "COALESCE(MIN(COUNT), ID) "
-                + "FROM TEST_COALESCE "
-                + "GROUP BY ID");
+                + "FROM " + tableName
+                + " GROUP BY ID");
 
         assertTrue(rs.next());
         assertEquals(2, rs.getLong(1));
@@ -297,13 +302,14 @@ public class CoalesceFunctionIT extends BaseHBaseManagedTimeIT {
     public void testCoalesceInRowKeyColumn() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.createStatement().execute("CREATE TABLE coalesceTest(k1 decimal, k2 decimal, constraint pk primary key (k1,k2))");
-        conn.createStatement().execute("UPSERT INTO coalesceTest(k2) VALUES (1)");
-        conn.createStatement().execute("UPSERT INTO coalesceTest VALUES (2,2)");
-        conn.createStatement().execute("UPSERT INTO coalesceTest VALUES (3,3)");
+        String tableName = generateRandomString();
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(k1 decimal, k2 decimal, constraint pk primary key (k1,k2))");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(k2) VALUES (1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES (2,2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " VALUES (3,3)");
         conn.commit();
         
-        ResultSet rs = conn.createStatement().executeQuery("SELECT coalesce(k1, 1) ,k2 FROM coalesceTest");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT coalesce(k1, 1) ,k2 FROM " + tableName);
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(1, rs.getInt(2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
index e97c7e1..9a56136 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnProjectionOptimizationIT.java
@@ -63,18 +63,19 @@ import org.junit.Test;
 
 public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
 
+    private String tableName; 
     @Test
     public void testSelect() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
+        String tableName = initATableValues(tenantId, getDefaultSplits(tenantId), null, ts);
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
         Connection conn = DriverManager.getConnection(getUrl(), props);
 
         // Table wildcard query
-        String query = "SELECT * FROM aTable";
+        String query = "SELECT * FROM " + tableName ;
         try {
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
@@ -99,7 +100,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
             assertFalse(rs.next());
 
             // Select only specific columns
-            query = "SELECT A_STRING, A_INTEGER FROM aTable";
+            query = "SELECT A_STRING, A_INTEGER FROM " + tableName ;
             statement = conn.prepareStatement(query);
             rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -132,7 +133,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
             assertFalse(rs.next());
 
             // Select only specific columns with condition on another column (Not in select)
-            query = "SELECT B_STRING, A_SHORT FROM aTable WHERE X_INTEGER = ?";
+            query = "SELECT B_STRING, A_SHORT FROM " + tableName + " WHERE X_INTEGER = ?";
             statement = conn.prepareStatement(query);
             statement.setInt(1, 4);
             rs = statement.executeQuery();
@@ -143,7 +144,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
 
             // Select only specific columns with condition on another column (Not in select) and one row elements are
             // nulls
-            query = "SELECT X_LONG, X_INTEGER, Y_INTEGER FROM aTable WHERE B_STRING = ?";
+            query = "SELECT X_LONG, X_INTEGER, Y_INTEGER FROM " + tableName + " WHERE B_STRING = ?";
             statement = conn.prepareStatement(query);
             statement.setString(1, E_VALUE);
             rs = statement.executeQuery();
@@ -168,7 +169,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
             assertFalse(rs.next());
 
             // Select only specific columns with condition on one of the selected column
-            query = "SELECT A_STRING, A_INTEGER FROM aTable WHERE A_INTEGER = ?";
+            query = "SELECT A_STRING, A_INTEGER FROM " + tableName + " WHERE A_INTEGER = ?";
             statement = conn.prepareStatement(query);
             statement.setInt(1, 9);
             rs = statement.executeQuery();
@@ -177,7 +178,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
             assertEquals(9, rs.getInt(2));
 
             // Select all columns with order by on non PK column
-            query = "SELECT * FROM aTable ORDER BY A_INTEGER";
+            query = "SELECT * FROM " + tableName + " ORDER BY A_INTEGER";
             statement = conn.prepareStatement(query);
             rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -316,7 +317,7 @@ public class ColumnProjectionOptimizationIT extends BaseClientManagedTimeIT {
     
     private static void initMultiCFTable(long ts) throws Exception {
         String url = getUrl();
-        ensureTableCreated(url, MULTI_CF_NAME, ts);
+        ensureTableCreated(url, MULTI_CF_NAME, MULTI_CF_NAME, ts);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/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 b4c84b0..677d76f 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,6 +18,7 @@
 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;
@@ -51,7 +52,7 @@ public class CountDistinctCompressionIT extends BaseOwnClusterHBaseManagedTimeIT
     @Test
     public void testDistinctCountOnColumn() throws Exception {
         String tenantId = getOrganizationId();
-        initATableValues(tenantId, getDefaultSplits(tenantId), (Date)null, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), (Date)null, null, getUrl());
 
         String query = "SELECT count(DISTINCT A_STRING) FROM aTable";
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
index 3717396..ad0f308 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CustomEntityDataIT.java
@@ -43,7 +43,7 @@ import org.junit.Test;
 public class CustomEntityDataIT extends BaseClientManagedTimeIT {
     
     protected static void initTableValues(String tenantId, byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(),CUSTOM_ENTITY_DATA_FULL_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),CUSTOM_ENTITY_DATA_FULL_NAME,CUSTOM_ENTITY_DATA_FULL_NAME, ts-2);
             
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index 7ffc54f..abdcf3b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -57,12 +57,13 @@ import org.junit.Before;
 import org.junit.Test;
 
 
-public class DateTimeIT extends BaseHBaseManagedTimeIT {
+public class DateTimeIT extends BaseHBaseManagedTimeTableReuseIT {
 
     protected Connection conn;
     protected Date date;
     protected static final String tenantId = getOrganizationId();
     protected final static String ROW10 = "00D123122312312";
+    protected  String tableName;
 
     public DateTimeIT() throws Exception {
         super();
@@ -72,7 +73,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
     @Before
     public void setUp() throws SQLException {
         conn = DriverManager.getConnection(getUrl());
-        initAtable();
+        this.tableName = initAtable();
     }
 
     @After
@@ -80,10 +81,11 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         conn.close();
     }
     
-    private void initAtable() throws SQLException { 
-        ensureTableCreated(getUrl(), ATABLE_NAME, (byte[][])null);
+    private String initAtable() throws SQLException {
+        String tableName = generateRandomString();
+        ensureTableCreated(getUrl(), tableName, ATABLE_NAME, (byte[][])null);
         PreparedStatement stmt = conn.prepareStatement(
-            "upsert into " + ATABLE_NAME +
+            "upsert into " + tableName +
             "(" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
@@ -288,6 +290,8 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         stmt.execute();
 
         conn.commit();
+        return  tableName;
+
     }
 
     @Test
@@ -298,9 +302,10 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         java.util.Date dateToday = cal.getTime();
         cal.add(Calendar.DAY_OF_YEAR, 1);
         java.util.Date dateTomorrow = cal.getTime();
+        String tableName = generateRandomString();
         String today = formatter.format(dateToday);
         String tomorrow = formatter.format(dateTomorrow);
-        String query = "SELECT entity_id FROM ATABLE WHERE a_integer < 4 AND a_date BETWEEN date '" + today + "' AND date '" + tomorrow + "' ";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE a_integer < 4 AND a_date BETWEEN date '" + today + "' AND date '" + tomorrow + "' ";
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertTrue(rs.next());
@@ -311,7 +316,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectLiteralDate() throws Exception {
         String s = DateUtil.DEFAULT_DATE_FORMATTER.format(date);
-        String query = "SELECT DATE '" + s + "' FROM ATABLE";
+        String query = "SELECT DATE '" + s + "' FROM " + this.tableName;
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertTrue(rs.next());
@@ -320,7 +325,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectLiteralDateCompare() throws Exception {
-        String query = "SELECT (DATE '" + date + "' = DATE '" + date + "') FROM ATABLE";
+        String query = "SELECT (DATE '" + date + "' = DATE '" + date + "') FROM " + this.tableName;
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertTrue(rs.next());
@@ -329,7 +334,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectWhereDatesEqual() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE  a_integer < 4 AND DATE '" + date + "' = DATE '" + date + "'";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE  a_integer < 4 AND DATE '" + date + "' = DATE '" + date + "'";
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertTrue(rs.next());
@@ -338,7 +343,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectWhereDateAndToDateEqual() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE  a_integer < 4 AND DATE '" + date + "' = TO_DATE ('" + date + "')";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE  a_integer < 4 AND DATE '" + date + "' = TO_DATE ('" + date + "')";
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertTrue(rs.next());
@@ -348,7 +353,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
     @Test
     public void testSelectWhereDateAndTimestampEqual() throws Exception {
         final String timestamp = "2012-09-08 07:08:23";
-        String query = "SELECT entity_id FROM ATABLE WHERE  a_integer < 4 AND DATE '" + timestamp + "' = TIMESTAMP '" + timestamp + "'";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE  a_integer < 4 AND DATE '" + timestamp + "' = TIMESTAMP '" + timestamp + "'";
 
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
@@ -357,7 +362,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSelectWhereSameDatesUnequal() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE  a_integer < 4 AND DATE '" + date + "' > DATE '" + date + "'";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE  a_integer < 4 AND DATE '" + date + "' > DATE '" + date + "'";
         Statement statement = conn.createStatement();
         ResultSet rs = statement.executeQuery(query);
         assertFalse(rs.next());
@@ -365,7 +370,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testDateInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE a_date IN (?,?) AND a_integer < 4";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE a_date IN (?,?) AND a_integer < 4";
             PreparedStatement statement = conn.prepareStatement(query);
             statement.setDate(1, new Date(0));
             statement.setDate(2, date);
@@ -385,7 +390,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         java.util.Date dateTomorrow = cal.getTime();
         String today = formatter.format(dateToday);
         String tomorrow = formatter.format(dateTomorrow);
-        String query = "SELECT entity_id FROM ATABLE WHERE a_integer < 4 AND a_date BETWEEN date '" + today + "' AND date '" + tomorrow + "' ";
+        String query = "SELECT entity_id FROM " + this.tableName + " WHERE a_integer < 4 AND a_date BETWEEN date '" + today + "' AND date '" + tomorrow + "' ";
             Statement statement = conn.createStatement();
             ResultSet rs = statement.executeQuery(query);
             assertTrue(rs.next());
@@ -456,23 +461,24 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testYearFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
                         "unsignedDates UNSIGNED_DATE, unsignedTimestamps UNSIGNED_TIMESTAMP, unsignedTimes UNSIGNED_TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
                 "TO_DATE('2010-03-01 00:00:00:896', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-02-01'), TO_TIME('2015-02-01 00:00:00'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2005-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('2005-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
                 "TO_DATE('2010-03-01 00:00:00:896', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-02-01'), TO_TIME('2015-02-01 00:00:00'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (3, TO_DATE('2006-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TO_DATE('2006-03-01 00:00:00'), TO_TIMESTAMP('2006-02-01 00:00:00'), TO_TIME('2008-02-01 00:00:00'), " +
                 "TO_DATE('2010-03-01 00:00:00:896', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-02-01'), TO_TIME('2015-02-01 00:00:00'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, YEAR(timestamps), YEAR(times), Year(unsignedDates), YEAR(unsignedTimestamps), " +
-                "YEAR(unsignedTimes) FROM T1 where YEAR(dates) = 2004");
+                "YEAR(unsignedTimes) FROM " + tableName + " where YEAR(dates) = 2004");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(2006, rs.getInt(2));
@@ -485,23 +491,24 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testMonthFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
                         "unsignedDates UNSIGNED_DATE, unsignedTimestamps UNSIGNED_TIMESTAMP, unsignedTimes UNSIGNED_TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-03-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-03-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
                 "TO_DATE('2010-06-20 00:00:00:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:00'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2004-04-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('2004-04-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
                 "TO_DATE('2010-06-20 00:00:00:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:00'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (3, TO_DATE('2004-05-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TO_DATE('2004-05-10 00:00:00'), TO_TIMESTAMP('2006-04-12 00:00:00'), TO_TIME('2008-05-16 00:00:00'), " +
                 "TO_DATE('2010-06-20 00:00:00:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:00'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, MONTH(timestamps), MONTH(times), MONTH(unsignedDates), MONTH(unsignedTimestamps), " +
-                "MONTH(unsignedTimes) FROM T1 where MONTH(dates) = 3");
+                "MONTH(unsignedTimes) FROM " + tableName + " where MONTH(dates) = 3");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(4, rs.getInt(2));
@@ -514,17 +521,18 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testUnsignedTimeDateWithLiteral() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS UT (k1 INTEGER NOT NULL," +
+                "CREATE TABLE IF NOT EXISTS " + tableName + "  (k1 INTEGER NOT NULL," +
                         "unsignedDates UNSIGNED_DATE, unsignedTimestamps UNSIGNED_TIMESTAMP, unsignedTimes UNSIGNED_TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO UT VALUES (1, " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, " +
                 "'2010-06-20 12:00:00', '2012-07-28 12:00:00', '2015-12-25 12:00:00')";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, unsignedDates, " +
-                "unsignedTimestamps, unsignedTimes FROM UT where k1 = 1");
+                "unsignedTimestamps, unsignedTimes FROM " + tableName + " where k1 = 1");
         assertTrue(rs.next());
         assertEquals(DateUtil.parseDate("2010-06-20 12:00:00"), rs.getDate(2));
         assertEquals(DateUtil.parseTimestamp("2012-07-28 12:00:00"), rs.getTimestamp(3));
@@ -534,23 +542,24 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testSecondFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
                         "unsignedDates UNSIGNED_DATE, unsignedTimestamps UNSIGNED_TIMESTAMP, unsignedTimes UNSIGNED_TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:00:20'), TO_TIME('2008-05-16 10:00:30'), " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:00:20'), TO_TIME('2008-05-16 10:00:30'), " +
                 "TO_DATE('2010-06-20 00:00:40:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:50'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:20:30'), TO_TIME('2008-05-16 10:00:30'), " +
+        dml = "UPSERT INTO " + tableName + "  VALUES (2, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:20:30'), TO_TIME('2008-05-16 10:00:30'), " +
                 "TO_DATE('2010-06-20 00:00:40:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:50'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (3, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:50:30'), TO_TIME('2008-05-16 10:00:30'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TO_DATE('2004-03-01 00:00:10'), TO_TIMESTAMP('2006-04-12 00:50:30'), TO_TIME('2008-05-16 10:00:30'), " +
                 "TO_DATE('2010-06-20 00:00:40:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:00:50'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, SECOND(dates), SECOND(times), SECOND(unsignedDates), SECOND(unsignedTimestamps), " +
-                "SECOND(unsignedTimes) FROM T1 where SECOND(timestamps)=20");
+                "SECOND(unsignedTimes) FROM " + tableName + " where SECOND(timestamps)=20");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(10, rs.getInt(2));
@@ -563,18 +572,19 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testWeekFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
+                "CREATE TABLE IF NOT EXISTS " + tableName + "  (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-04-12 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-04-12 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-05-18 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-05-18 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (3, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-05-18 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TO_DATE('2004-01-10 10:00:10'), TO_TIMESTAMP('2006-05-18 08:00:20'), TO_TIME('2008-05-16 10:00:30'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, WEEK(dates), WEEK(times) FROM T1 where WEEK(timestamps)=15");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, WEEK(dates), WEEK(times) FROM " + tableName + " where WEEK(timestamps)=15");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(2, rs.getInt(2));
@@ -584,21 +594,22 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testHourFuncAgainstColumns() throws Exception {
+        String tableName  = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 15:10:20'), " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 15:10:20'), " +
                 "TO_TIME('2008-05-16 20:40:30'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 10:10:20'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 10:10:20'), " +
                 "TO_TIME('2008-05-16 20:40:30'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (3, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 08:10:20'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TO_DATE('Sat, 3 Feb 2008 03:05:06 GMT', 'EEE, d MMM yyyy HH:mm:ss z', 'UTC'), TO_TIMESTAMP('2006-04-12 08:10:20'), " +
                 "TO_TIME('2008-05-16 20:40:30'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, HOUR(dates), HOUR(times) FROM T1 where HOUR(timestamps)=15");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, HOUR(dates), HOUR(times) FROM " + tableName + " where HOUR(timestamps)=15");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(3, rs.getInt(2));
@@ -608,11 +619,12 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testNowFunction() throws Exception {
+        String tableName = generateRandomString();
         Date date = new Date(System.currentTimeMillis());
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, timestamps TIMESTAMP CONSTRAINT pk PRIMARY KEY (k1))";
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, timestamps TIMESTAMP CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (?, ?)";
+        String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setInt(1, 1);
         stmt.setDate(2, new Date(date.getTime()-500));
@@ -622,7 +634,7 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         stmt.execute();
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT * from T1 where now() > timestamps");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * from " + tableName + "  where now() > timestamps");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(new Date(date.getTime()-500), rs.getDate(2));
@@ -631,20 +643,21 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testMinuteFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME, " +
                         "unsignedDates UNSIGNED_DATE, unsignedTimestamps UNSIGNED_TIMESTAMP, unsignedTimes UNSIGNED_TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-03-01 00:10:10'), TO_TIMESTAMP('2006-04-12 00:20:20'), TO_TIME('2008-05-16 10:30:30'), " +
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-03-01 00:10:10'), TO_TIMESTAMP('2006-04-12 00:20:20'), TO_TIME('2008-05-16 10:30:30'), " +
                 "TO_DATE('2010-06-20 00:40:40:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:50:50'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2004-03-01 00:10:10'), TO_TIMESTAMP('2006-04-12 00:50:20'), TO_TIME('2008-05-16 10:30:30'), " +
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('2004-03-01 00:10:10'), TO_TIMESTAMP('2006-04-12 00:50:20'), TO_TIME('2008-05-16 10:30:30'), " +
                 "TO_DATE('2010-06-20 00:40:40:789', 'yyyy-MM-dd HH:mm:ss:SSS'), TO_TIMESTAMP('2012-07-28'), TO_TIME('2015-12-25 00:50:50'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery("SELECT k1, MINUTE(dates), MINUTE(times), MINUTE(unsignedDates), MINUTE(unsignedTimestamps), " +
-                "MINUTE(unsignedTimes) FROM T1 where MINUTE(timestamps)=20");
+                "MINUTE(unsignedTimes) FROM " + tableName + " where MINUTE(timestamps)=20");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(10, rs.getInt(2));
@@ -657,16 +670,17 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
     
     @Test
     public void testDayOfMonthFuncAgainstColumns() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS T1 (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL, dates DATE, timestamps TIMESTAMP, times TIME CONSTRAINT pk PRIMARY KEY (k1))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO T1 VALUES (1, TO_DATE('2004-01-08 10:00:10'), TO_TIMESTAMP('2006-04-12 08:00:20'), TO_TIME('2008-05-26 11:00:30'))";
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TO_DATE('2004-01-08 10:00:10'), TO_TIMESTAMP('2006-04-12 08:00:20'), TO_TIME('2008-05-26 11:00:30'))";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO T1 VALUES (2, TO_DATE('2004-01-18 10:00:10'), TO_TIMESTAMP('2006-05-22 08:00:20'), TO_TIME('2008-12-30 11:00:30'))";
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TO_DATE('2004-01-18 10:00:10'), TO_TIMESTAMP('2006-05-22 08:00:20'), TO_TIME('2008-12-30 11:00:30'))";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, DAYOFMONTH(dates), DAYOFMONTH(times) FROM T1 where DAYOFMONTH(timestamps)=12");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1, DAYOFMONTH(dates), DAYOFMONTH(times) FROM " + tableName + " where DAYOFMONTH(timestamps)=12");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals(8, rs.getInt(2));
@@ -676,7 +690,8 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testNullDate() throws Exception {
-        ResultSet rs = conn.createStatement().executeQuery("SELECT a_date, entity_id from " + ATABLE_NAME + " WHERE entity_id = '" + ROW10 + "'");
+
+        ResultSet rs = conn.createStatement().executeQuery("SELECT a_date, entity_id from " + this.tableName + " WHERE entity_id = '" + ROW10 + "'");
         assertNotNull(rs);
         assertTrue(rs.next());
         assertEquals(ROW10, rs.getString(2));
@@ -695,19 +710,20 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
     }
     @Test
     public void testSelectBetweenNanos() throws Exception {
+        String tableName = generateRandomString();
         String ddl =
-                "CREATE TABLE IF NOT EXISTS N1 (k1 INTEGER NOT NULL PRIMARY KEY, ts " +
+                "CREATE TABLE IF NOT EXISTS " + tableName + " (k1 INTEGER NOT NULL PRIMARY KEY, ts " +
                         "TIMESTAMP(3))";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO N1 VALUES (1, TIMESTAMP'2015-01-01 00:00:00.111111111')";
+        String dml = "UPSERT INTO " + tableName + " VALUES (1, TIMESTAMP'2015-01-01 00:00:00.111111111')";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO N1 VALUES (2, TIMESTAMP'2015-01-01 00:00:00.111111115')";
+        dml = "UPSERT INTO " + tableName + " VALUES (2, TIMESTAMP'2015-01-01 00:00:00.111111115')";
         conn.createStatement().execute(dml);
-        dml = "UPSERT INTO N1 VALUES (3, TIMESTAMP'2015-01-01 00:00:00.111111113')";
+        dml = "UPSERT INTO " + tableName + " VALUES (3, TIMESTAMP'2015-01-01 00:00:00.111111113')";
         conn.createStatement().execute(dml);
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k1,ts from N1 where ts between" +
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k1,ts from " + tableName + " where ts between" +
                 " TIMESTAMP'2015-01-01 00:00:00.111111112' AND TIMESTAMP'2015-01-01 00:00:00" +
                 ".111111114'");
         assertTrue(rs.next());
@@ -718,17 +734,19 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testCurrentTimeWithProjectedTable () throws Exception {
-        String ddl = "CREATE TABLE T1 ( ID integer primary key)";
+        String tableName1 = generateRandomString();
+        String tableName2 = generateRandomString();
+        String ddl = "CREATE TABLE " + tableName1 + " ( ID integer primary key)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE TABLE T2 ( ID integer primary key)";
+        ddl = "CREATE TABLE " + tableName2 + " ( ID integer primary key)";
         conn.createStatement().execute(ddl);
-        String ups = "UPSERT INTO T1 VALUES (1)";
+        String ups = "UPSERT INTO " + tableName1 + " VALUES (1)";
         conn.createStatement().execute(ups);
-        ups = "UPSERT INTO T2 VALUES (1)";
+        ups = "UPSERT INTO " + tableName2 + " VALUES (1)";
         conn.createStatement().execute(ups);
         conn.commit();
         ResultSet rs = conn.createStatement().executeQuery("select /*+ USE_SORT_MERGE_JOIN */ op" +
-                ".id, current_time() from t1 op where op.id in (select id from t2)");
+                ".id, current_time() from " +tableName1 + " op where op.id in (select id from " + tableName2 + ")");
         assertTrue(rs.next());
         assertEquals(new java.util.Date().getYear(),rs.getTimestamp(2).getYear());
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/14dab2f4/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 558bc38..865b1fd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -38,13 +38,14 @@ import org.junit.Assert;
 import org.junit.Test;
 
 
-public class DeleteIT extends BaseHBaseManagedTimeIT {
+public class DeleteIT extends BaseHBaseManagedTimeTableReuseIT {
     private static final int NUMBER_OF_ROWS = 20;
     private static final int NTH_ROW_NULL = 5;
     
-    private static void initTableValues(Connection conn) throws SQLException {
-        ensureTableCreated(getUrl(),"IntIntKeyTest");
-        String upsertStmt = "UPSERT INTO IntIntKeyTest VALUES(?,?)";
+    private static String initTableValues(Connection conn) throws SQLException {
+        String tableName = generateRandomString();
+        ensureTableCreated(getUrl(), tableName, "IntIntKeyTest");
+        String upsertStmt = "UPSERT INTO " + tableName + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(upsertStmt);
         for (int i = 0; i < NUMBER_OF_ROWS; i++) {
             stmt.setInt(1, i);
@@ -56,6 +57,7 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             stmt.execute();
         }
         conn.commit();
+        return tableName;
     }
 
     @Test
@@ -70,18 +72,18 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
     
     private void testDeleteFilter(boolean autoCommit) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTableValues(conn);
+        String tableName = initTableValues(conn);
 
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS);
         
         conn.setAutoCommit(autoCommit);
-        String deleteStmt = "DELETE FROM IntIntKeyTest WHERE 20 = j";
+        String deleteStmt = "DELETE FROM " + tableName + " WHERE 20 = j";
         assertEquals(1,conn.createStatement().executeUpdate(deleteStmt));
         if (!autoCommit) {
             conn.commit();
         }
 
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS - 1);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS - 1);
     }
 
     @Test
@@ -97,34 +99,34 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
 
     private void testDeleteByFilterAndRow(boolean autoCommit) throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTableValues(conn);
+        String tableName = initTableValues(conn);
 
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS);
 
         conn.setAutoCommit(autoCommit);
 
         Statement stmt = conn.createStatement();
 
         // This shouldn't delete anything, because the key matches but the filter doesn't
-        assertEquals(0, stmt.executeUpdate("DELETE FROM IntIntKeyTest WHERE i = 1 AND j = 1"));
+        assertEquals(0, stmt.executeUpdate("DELETE FROM " + tableName + " WHERE i = 1 AND j = 1"));
         if (!autoCommit) {
             conn.commit();
         }
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS);
 
         // This shouldn't delete anything, because the filter matches but the key doesn't
-        assertEquals(0, stmt.executeUpdate("DELETE FROM IntIntKeyTest WHERE i = -1 AND j = 20"));
+        assertEquals(0, stmt.executeUpdate("DELETE FROM " + tableName + " WHERE i = -1 AND j = 20"));
         if (!autoCommit) {
             conn.commit();
         }
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS);
 
         // This should do a delete, because both the filter and key match
-        assertEquals(1, stmt.executeUpdate("DELETE FROM IntIntKeyTest WHERE i = 1 AND j = 10"));
+        assertEquals(1, stmt.executeUpdate("DELETE FROM " + tableName + " WHERE i = 1 AND j = 10"));
         if (!autoCommit) {
             conn.commit();
         }
-        assertTableCount(conn, "IntIntKeyTest", NUMBER_OF_ROWS - 1);
+        assertTableCount(conn, tableName, NUMBER_OF_ROWS - 1);
 
     }
 
@@ -155,40 +157,42 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
 
     private void testDeleteRange(boolean autoCommit, boolean createIndex, boolean local) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        initTableValues(conn);
-        
-        String indexName = "IDX";
+        String tableName = initTableValues(conn);
+        String indexName = generateRandomString();
+        String localIndexName = generateRandomString();
+
+        String indexInUse = indexName;
         if (createIndex) {
             if (local) {
-                conn.createStatement().execute("CREATE LOCAL INDEX IF NOT EXISTS local_idx ON IntIntKeyTest(j)");
-                indexName = "INTINTKEYTEST";
+                conn.createStatement().execute("CREATE LOCAL INDEX IF NOT EXISTS " + localIndexName + " ON " + tableName + "(j)");
+                indexInUse = tableName;
             } else {
-                conn.createStatement().execute("CREATE INDEX IF NOT EXISTS idx ON IntIntKeyTest(j)");
+                conn.createStatement().execute("CREATE INDEX IF NOT EXISTS " + indexName + " ON " + tableName + "(j)");
             }
         }
         
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM IntIntKeyTest");
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName);
         assertTrue(rs.next());
         assertEquals(NUMBER_OF_ROWS, rs.getInt(1));
 
-        rs = conn.createStatement().executeQuery("SELECT i FROM IntIntKeyTest WHERE j IS NULL");
+        rs = conn.createStatement().executeQuery("SELECT i FROM " + tableName + " WHERE j IS NULL");
         int i = 0, isNullCount = 0;
         while (rs.next()) {
             assertEquals(i,rs.getInt(1));
             i += NTH_ROW_NULL;
             isNullCount++;
         }
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM IntIntKeyTest WHERE j IS NOT NULL");
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName + " WHERE j IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(NUMBER_OF_ROWS-isNullCount, rs.getInt(1));
 
         String deleteStmt ;
         PreparedStatement stmt;
         conn.setAutoCommit(autoCommit);
-        deleteStmt = "DELETE FROM IntIntKeyTest WHERE i >= ? and i < ?";
+        deleteStmt = "DELETE FROM " + tableName + " WHERE i >= ? and i < ?";
         if(!local) {
-            assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexName, false);
+            assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexInUse, false);
         }
         stmt = conn.prepareStatement(deleteStmt);
         stmt.setInt(1, 5);
@@ -198,24 +202,24 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             conn.commit();
         }
         
-        String query = "SELECT count(*) FROM IntIntKeyTest";
-        assertIndexUsed(conn, query, indexName, createIndex);
-        query = "SELECT count(*) FROM IntIntKeyTest";
+        String query = "SELECT count(*) FROM " + tableName;
+        assertIndexUsed(conn, query, indexInUse, createIndex);
+        query = "SELECT count(*) FROM " + tableName;
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
         assertEquals(NUMBER_OF_ROWS - (10-5), rs.getInt(1));
         
-        deleteStmt = "DELETE FROM IntIntKeyTest WHERE j IS NULL";
+        deleteStmt = "DELETE FROM " + tableName + " WHERE j IS NULL";
         stmt = conn.prepareStatement(deleteStmt);
         if(!local) {
-            assertIndexUsed(conn, deleteStmt, indexName, createIndex);
+            assertIndexUsed(conn, deleteStmt, indexInUse, createIndex);
         }
         int deleteCount = stmt.executeUpdate();
         assertEquals(3, deleteCount);
         if (!autoCommit) {
             conn.commit();
         }
-        rs = conn.createStatement().executeQuery("SELECT count(*) FROM IntIntKeyTest");
+        rs = conn.createStatement().executeQuery("SELECT count(*) FROM " + tableName);
         assertTrue(rs.next());
         assertEquals(NUMBER_OF_ROWS - (10-5)-isNullCount+1, rs.getInt(1));
     }
@@ -291,7 +295,8 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             con.setAutoCommit(autoCommit);
 
             Statement stm = con.createStatement();
-            String s = "CREATE TABLE IF NOT EXISTS web_stats (" +
+            String tableName = generateRandomString();
+            String s = "CREATE TABLE IF NOT EXISTS " + tableName + "(" +
                     "HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
@@ -301,15 +306,17 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                     "STATS.ACTIVE_VISITOR INTEGER " +
                     "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE))" + (isSalted ? " SALT_BUCKETS=3" : "");
             stm.execute(s);
+            String localIndexName = generateRandomString();
+            String indexName = generateRandomString();
             if (localIndex) {
-                stm.execute("CREATE LOCAL INDEX local_web_stats_idx ON web_stats (CORE,DB,ACTIVE_VISITOR)");
+                stm.execute("CREATE LOCAL INDEX " + localIndexName + " ON " + tableName + " (CORE,DB,ACTIVE_VISITOR)");
             } else {
-                stm.execute("CREATE INDEX web_stats_idx ON web_stats (CORE,DB,ACTIVE_VISITOR)");
+                stm.execute("CREATE INDEX " + indexName + " ON " + tableName + " (CORE,DB,ACTIVE_VISITOR)");
             }
             stm.close();
 
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO web_stats(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -323,18 +330,18 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                 con.commit();
             }
             
-            con.createStatement().execute("DELETE FROM web_stats");
+            con.createStatement().execute("DELETE FROM " + tableName );
             if (!autoCommit) {
                 con.commit();
             }
             
-            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM web_stats");
+            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
             if(localIndex){
-                rs = con.createStatement().executeQuery("SELECT count(*) FROM local_web_stats_idx");
+                rs = con.createStatement().executeQuery("SELECT count(*) FROM " + localIndexName);
             } else {
-                rs = con.createStatement().executeQuery("SELECT count(*) FROM web_stats_idx");
+                rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName);
             }
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
@@ -365,7 +372,12 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             con.setAutoCommit(autoCommit);
 
             Statement stm = con.createStatement();
-            stm.execute("CREATE TABLE IF NOT EXISTS web_stats (" +
+
+            String tableName = generateRandomString();
+            String indexName1 = generateRandomString();
+            String indexName2 = generateRandomString();
+
+            stm.execute("CREATE TABLE IF NOT EXISTS " + tableName + " (" +
                     "HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
@@ -374,13 +386,13 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                     "USAGE.DB BIGINT," +
                     "STATS.ACTIVE_VISITOR INTEGER " +
                     "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE)) IMMUTABLE_ROWS=true");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX web_stats_idx ON web_stats (DATE, FEATURE)");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX web_stats_idx2 ON web_stats (DATE, FEATURE, USAGE.DB)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (DATE, FEATURE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (DATE, FEATURE, USAGE.DB)");
             stm.close();
 
             Date date = new Date(0);
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO web_stats(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -394,7 +406,7 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                 con.commit();
             }
             
-            psInsert = con.prepareStatement("DELETE FROM web_stats WHERE (HOST, DOMAIN, FEATURE, DATE) = (?,?,?,?)");
+            psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, DATE) = (?,?,?,?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -404,21 +416,21 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                 con.commit();
             }
             
-            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM web_stats");
+            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
 
-            rs = con.createStatement().executeQuery("SELECT count(*) FROM web_stats_idx");
+            rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName1);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
 
-            stm.execute("DROP INDEX web_stats_idx ON web_stats");
-            stm.execute("DROP INDEX web_stats_idx2 ON web_stats");
+            stm.execute("DROP INDEX " + indexName1 + " ON " + tableName);
+            stm.execute("DROP INDEX " + indexName2 + " ON " + tableName);
 
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX web_stats_idx ON web_stats (USAGE.DB)");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX web_stats_idx2 ON web_stats (USAGE.DB, DATE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (USAGE.DB)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, DATE)");
             try{
-                psInsert = con.prepareStatement("DELETE FROM web_stats WHERE  USAGE.DB=2");
+                psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE  USAGE.DB=2");
             } catch(Exception e) {
                 fail("There should not be any exception while deleting row");
             }
@@ -447,9 +459,11 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             con = DriverManager.getConnection(getUrl());
             con.setAutoCommit(autoCommit);
 
+            String tableName = generateRandomString();
+
             Statement stm = con.createStatement();
-            stm.execute("CREATE TABLE IF NOT EXISTS web_stats (" +
-                    "HOST CHAR(2) NOT NULL," +
+            stm.execute("CREATE TABLE IF NOT EXISTS " + tableName + "(" +
+                    " HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
                     "DATE DATE NOT NULL, \n" + 
@@ -460,7 +474,7 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
             stm.close();
 
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO web_stats(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -474,12 +488,12 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
                 con.commit();
             }
             
-            con.createStatement().execute("DELETE FROM web_stats");
+            con.createStatement().execute("DELETE FROM " + tableName);
             if (!autoCommit) {
                 con.commit();
             }
             
-            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM web_stats");
+            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
         } finally {
@@ -569,17 +583,19 @@ public class DeleteIT extends BaseHBaseManagedTimeIT {
     }
     
     private void testDeleteCount(boolean autoCommit, Integer limit) throws Exception {
-        String ddl = "CREATE TABLE IF NOT EXISTS TEST_TABLE (pk1 DECIMAL NOT NULL, v1 VARCHAR CONSTRAINT PK PRIMARY KEY (pk1))";
+        String tableName = generateRandomString();
+
+        String ddl = "CREATE TABLE IF NOT EXISTS " + tableName + " (pk1 DECIMAL NOT NULL, v1 VARCHAR CONSTRAINT PK PRIMARY KEY (pk1))";
         int numRecords = 1010;
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute(ddl);
             Statement stmt = conn.createStatement();
             for (int i = 0; i < numRecords ; i++) {
-                stmt.executeUpdate("UPSERT INTO TEST_TABLE (pk1, v1) VALUES (" + i + ",'value')");
+                stmt.executeUpdate("UPSERT INTO " + tableName + " (pk1, v1) VALUES (" + i + ",'value')");
             }
             conn.commit();
             conn.setAutoCommit(autoCommit);
-            String delete = "DELETE FROM TEST_TABLE WHERE (pk1) <= (" + numRecords + ")" + (limit == null ? "" : (" limit " + limit));
+            String delete = "DELETE FROM " + tableName + " WHERE (pk1) <= (" + numRecords + ")" + (limit == null ? "" : (" limit " + limit));
             try (PreparedStatement pstmt = conn.prepareStatement(delete)) {
                 int numberOfDeletes = pstmt.executeUpdate();
                 assertEquals(limit == null ? numRecords : limit, numberOfDeletes);