You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2016/09/23 20:12:44 UTC

[1/3] phoenix git commit: PHOENIX-3327 Use same driver for all tests within a group

Repository: phoenix
Updated Branches:
  refs/heads/master bebb5cedf -> fedbca8bf


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 9b87361..ef3ac39 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
@@ -37,6 +37,7 @@ import static org.apache.phoenix.query.QueryServices.GLOBAL_METRICS_ENABLED;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_MAX_CACHE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.HBASE_CLIENT_SCANNER_TIMEOUT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.IMMUTABLE_ROWS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.INDEX_POPULATION_SLEEP_TIME;
@@ -59,7 +60,6 @@ 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.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;
@@ -78,6 +78,7 @@ import static org.apache.phoenix.query.QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB
 import static org.apache.phoenix.query.QueryServices.STATS_USE_CURRENT_TIME_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.THREAD_POOL_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.THREAD_TIMEOUT_MS_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.TRANSACTIONS_ENABLED;
 import static org.apache.phoenix.query.QueryServices.USE_BYTE_BASED_REGEX_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
 
@@ -598,6 +599,11 @@ public class QueryServicesOptions {
         return this;
     }
 
+    public QueryServicesOptions setTransactionsEnabled(boolean transactionsEnabled) {
+        config.setBoolean(TRANSACTIONS_ENABLED, transactionsEnabled);
+        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/fedbca8b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index bab52a4..58fb151 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -382,7 +382,7 @@ public final class QueryUtil {
                 HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
         String url = getUrl(server, port, znodeParent, principal);
         // 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);
+        String extraArgs = props.getProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, conf.get(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS));
         if (extraArgs.length() > 0) {
             url += extraArgs + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
index fde70d0..4ab75a9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
@@ -34,6 +34,7 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
@@ -89,6 +90,7 @@ public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
         }
     }
     
+    @Ignore
     @Test
     public void testDisallowIsolationLevel() throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 7008911..3de0b77 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
@@ -536,9 +536,7 @@ public abstract class BaseTest {
     }
     
     private static void checkTxManagerInitialized(ReadOnlyProps clientProps) throws SQLException, IOException {
-        if (txService == null
-                && clientProps.getBoolean(QueryServices.TRANSACTIONS_ENABLED,
-                        QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED)) {
+        if (txService == null) {
             setupTxManager();
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 f2b1519..d1d50d5 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
@@ -63,6 +63,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     public static final int DEFAULT_HCONNECTION_POOL_MAX_SIZE = 10;
     public static final int DEFAULT_HTABLE_MAX_THREADS = 10;
     public static final long DEFAULT_INDEX_POPULATION_WAIT_TIME = 0;
+    public static final boolean DEFAULT_TRANSACTIONS_ENABLED = true;
 
     
     /**
@@ -79,6 +80,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     
     private static QueryServicesOptions getDefaultServicesOptions() {
     	return withDefaults()
+    	        .setTransactionsEnabled(DEFAULT_TRANSACTIONS_ENABLED)
     	        .setExplainChunkCount(DEFAULT_EXPLAIN_CHUNK_COUNT)
                 .setExplainRowCount(DEFAULT_EXPLAIN_ROW_COUNT)
     	        .setSequenceSaltBuckets(DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS)


[3/3] phoenix git commit: PHOENIX-3327 Use same driver for all tests within a group

Posted by ja...@apache.org.
PHOENIX-3327 Use same driver for all tests within a group


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

Branch: refs/heads/master
Commit: fedbca8bf090baccaaed5d555daeba2ba9da5f32
Parents: bebb5ce
Author: James Taylor <ja...@apache.org>
Authored: Fri Sep 23 13:13:41 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Fri Sep 23 13:13:41 2016 -0700

----------------------------------------------------------------------
 .../StatisticsCollectionRunTrackerIT.java       | 15 ----
 .../apache/phoenix/end2end/AlterSessionIT.java  | 59 ++++++-------
 .../apache/phoenix/end2end/AlterTableIT.java    |  9 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 14 ----
 .../phoenix/end2end/ArrayFillFunctionIT.java    | 66 ++++++++-------
 .../apache/phoenix/end2end/BaseParallelIT.java  | 69 ----------------
 .../org/apache/phoenix/end2end/BaseViewIT.java  | 15 ----
 .../phoenix/end2end/CbrtFunctionEnd2EndIT.java  | 27 +++---
 .../phoenix/end2end/DisableLocalIndexIT.java    | 24 ++----
 .../phoenix/end2end/DistinctPrefixFilterIT.java | 24 +++---
 .../apache/phoenix/end2end/DynamicColumnIT.java | 19 +++--
 .../end2end/GetSetByteBitFunctionEnd2EndIT.java |  8 +-
 .../apache/phoenix/end2end/HashJoinMoreIT.java  | 16 ----
 .../phoenix/end2end/InMemoryOrderByIT.java      | 42 ----------
 .../apache/phoenix/end2end/IndexExtendedIT.java |  4 +-
 .../phoenix/end2end/LikeExpressionIT.java       | 16 ++--
 .../end2end/ParallelStatsDisabledIT.java        | 14 +++-
 .../phoenix/end2end/ParallelStatsEnabledIT.java | 12 +--
 .../phoenix/end2end/PowerFunctionEnd2EndIT.java | 19 +++--
 .../phoenix/end2end/QueryWithOffsetIT.java      | 14 ----
 .../phoenix/end2end/SerialIteratorsIT.java      | 19 ++---
 .../phoenix/end2end/SignFunctionEnd2EndIT.java  | 27 +++---
 .../end2end/SkipScanAfterManualSplitIT.java     | 33 +++-----
 .../phoenix/end2end/SortMergeJoinMoreIT.java    | 16 ----
 .../phoenix/end2end/SpooledOrderByIT.java       | 40 ---------
 .../phoenix/end2end/SpooledSortMergeJoinIT.java | 45 ----------
 .../phoenix/end2end/SpooledTmpFileDeleteIT.java | 62 +++++++-------
 .../phoenix/end2end/StatsCollectorIT.java       | 61 ++++----------
 .../end2end/StringToArrayFunctionIT.java        | 87 +++++++++++---------
 .../phoenix/end2end/TransactionalViewIT.java    | 15 ----
 .../end2end/index/AsyncIndexDisabledIT.java     | 13 ---
 .../phoenix/end2end/index/DropMetadataIT.java   | 50 +++++------
 .../index/GlobalIndexOptimizationIT.java        | 20 +----
 .../apache/phoenix/end2end/index/IndexIT.java   | 15 ----
 .../phoenix/end2end/index/LocalIndexIT.java     |  2 +-
 .../end2end/index/MutableIndexFailureIT.java    |  2 +-
 .../phoenix/end2end/index/MutableIndexIT.java   | 55 +++++--------
 .../phoenix/end2end/index/SaltedIndexIT.java    | 31 +++----
 .../phoenix/end2end/index/ViewIndexIT.java      |  2 +-
 .../end2end/index/txn/MutableRollbackIT.java    | 34 +++-----
 .../phoenix/end2end/index/txn/RollbackIT.java   | 25 ++----
 .../phoenix/iterate/PhoenixQueryTimeoutIT.java  | 52 ++++++------
 .../iterate/RoundRobinResultIteratorIT.java     | 38 ++++-----
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   | 15 ----
 .../phoenix/tx/NotThreadSafeTransactionIT.java  | 62 ++++++++++----
 .../org/apache/phoenix/tx/TransactionIT.java    | 54 ------------
 .../org/apache/phoenix/tx/TxCheckpointIT.java   | 29 +++----
 .../phoenix/query/QueryServicesOptions.java     |  8 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |  2 +-
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |  2 +
 .../java/org/apache/phoenix/query/BaseTest.java |  4 +-
 .../phoenix/query/QueryServicesTestImpl.java    |  2 +
 52 files changed, 481 insertions(+), 927 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
index 759ad4f..cf475f9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java
@@ -26,7 +26,6 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -35,33 +34,19 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.ParallelStatsEnabledIT;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class StatisticsCollectionRunTrackerIT extends ParallelStatsEnabledIT {
     private static final StatisticsCollectionRunTracker tracker = StatisticsCollectionRunTracker
             .getInstance(new Configuration());
 
     private String fullTableName;
 
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Must update config before starting server
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
-        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1024));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     @Before
     public void generateTableNames() {
         String schemaName = TestUtil.DEFAULT_SCHEMA_NAME;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterSessionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterSessionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterSessionIT.java
index ffd9656..bf666e4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterSessionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterSessionIT.java
@@ -17,13 +17,9 @@
  */
 package org.apache.phoenix.end2end;
 
-import org.apache.hadoop.hbase.client.Consistency;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.QueryUtil;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -31,9 +27,13 @@ import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.Properties;
 
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  *
@@ -42,53 +42,48 @@ import static org.junit.Assert.assertTrue;
  */
 public class AlterSessionIT extends ParallelStatsDisabledIT {
 
-    private static final String TABLE_NAME = generateUniqueName();
-    private static Connection testConn;
+    private String tableName;
 
-    @BeforeClass
-    public static void initTable() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        tableName = generateUniqueName();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        testConn = DriverManager.getConnection(getUrl(), props);
-        assertEquals(Consistency.STRONG, ((PhoenixConnection)testConn).getConsistency());
-        testConn.createStatement().execute(
-            "create table " + TABLE_NAME + " (col1 varchar primary key)");
-        testConn.commit();
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(
+                    "create table " + tableName + " (col1 varchar primary key)");
+        }
     }
 
     @Test
     public void testUpdateConsistency() throws Exception {
-        try {
-            Statement st = testConn.createStatement();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            Statement st = conn.createStatement();
             st.execute("alter session set Consistency = 'timeline'");
-            ResultSet rs = st.executeQuery("explain select * from " + TABLE_NAME);
-            assertEquals(Consistency.TIMELINE, ((PhoenixConnection)testConn).getConsistency());
+            ResultSet rs = st.executeQuery("explain select * from " + tableName);
+            assertEquals(Consistency.TIMELINE, conn.unwrap(PhoenixConnection.class).getConsistency());
             String queryPlan = QueryUtil.getExplainPlan(rs);
             assertTrue(queryPlan.indexOf("TIMELINE") > 0);
 
             // turn off timeline read consistency
             st.execute("alter session set Consistency = 'strong'");
-            rs = st.executeQuery("explain select * from " + TABLE_NAME);
+            rs = st.executeQuery("explain select * from " + tableName);
             queryPlan = QueryUtil.getExplainPlan(rs);
             assertTrue(queryPlan.indexOf("TIMELINE") < 0);
-        } finally {
-            this.testConn.close();
         }
     }
 
     @Test
     public void testSetConsistencyInURL() throws Exception {
-        try {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            Connection conn = DriverManager.getConnection(getUrl() + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR +
-                    "Consistency=TIMELINE", props);
+        try (Connection conn = DriverManager.getConnection(getUrl() + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR +
+                    "Consistency=TIMELINE", props)) {
             assertEquals(Consistency.TIMELINE, ((PhoenixConnection)conn).getConsistency());
             Statement st = conn.createStatement();
-            ResultSet rs = st.executeQuery("explain select * from " + TABLE_NAME);
+            ResultSet rs = st.executeQuery("explain select * from " + tableName);
             String queryPlan = QueryUtil.getExplainPlan(rs);
             assertTrue(queryPlan.indexOf("TIMELINE") > 0);
             conn.close();
-        } finally {
-            this.testConn.close();
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 031fd5c..0125a63 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -51,6 +51,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -2146,8 +2147,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             assertTrue(t.isMultiTenant());
             
             // check table metadata updated server side
-            ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM SYSTEM.CATALOG " +
- "WHERE table_name = '"
+            ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM SYSTEM.CATALOG "
+                            + "WHERE table_name = '"
                             + dataTableFullName + "' AND DISABLE_WAL IS NOT NULL AND MULTI_TENANT IS NOT NULL");
             assertTrue(rs.next());
             assertFalse(rs.getBoolean(1));
@@ -2234,7 +2235,9 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     
 	@Test
 	public void testCreatingTxnTableFailsIfTxnsDisabled() throws Exception {
-		try (Connection conn = DriverManager.getConnection(getUrl())) {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(false));
+		try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 			// creating a transactional table should fail if transactions are disabled
 			try {
 				conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR) TRANSACTIONAL=true");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 c9e2f26..e6bf2d2 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
@@ -32,7 +32,6 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -41,20 +40,15 @@ import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     
@@ -72,14 +66,6 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
         return Arrays.asList(false, true);
     }
 	
-	@BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-	
     private String generateDDL(String format) {
         return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
             isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
index 4680998..96bafad 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
@@ -23,22 +23,30 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.math.BigDecimal;
-import java.sql.*;
-
-import org.junit.BeforeClass;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+import org.junit.Before;
 import org.junit.Test;
 
 public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
-    private static final String REGIONS = generateUniqueName();
+    private String tableName;
 
-    @BeforeClass
-    public static void initTables() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        tableName = generateUniqueName();
         Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE " + REGIONS
+        String ddl = "CREATE TABLE " + tableName
             + " (region_name VARCHAR PRIMARY KEY,length1 INTEGER, length2 INTEGER,date DATE,time TIME,timestamp TIMESTAMP,varchar VARCHAR,integer INTEGER,double DOUBLE,bigint BIGINT,char CHAR(15),double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], varchars2 VARCHAR[])";
         conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO " + REGIONS
+        String dml = "UPSERT INTO " + tableName
             + "(region_name,length1,length2,date,time,timestamp,varchar,integer,double,bigint,char,double1,char1,nullcheck,chars2,varchars2) VALUES('SF Bay Area',"
             +
                 "0," +
@@ -68,7 +76,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(varchar,5) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(varchar,5) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"foo", "foo", "foo", "foo", "foo"};
@@ -85,7 +93,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(integer,4) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(integer,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{34, 34, 34, 34};
@@ -103,7 +111,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(double,4) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(double,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{23.45, 23.45, 23.45, 23.45};
@@ -121,7 +129,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(bigint,4) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(bigint,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{34567l, 34567l, 34567l, 34567l};
@@ -139,7 +147,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(char,4) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(char,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -156,7 +164,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(varchar,4) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(varchar,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -173,7 +181,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(date,3) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(date,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{new Date(1432102334184l), new Date(1432102334184l), new Date(1432102334184l)};
@@ -190,7 +198,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(time,3) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(time,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{new Time(1432102334184l), new Time(1432102334184l), new Time(1432102334184l)};
@@ -207,7 +215,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,3) FROM " + REGIONS + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(timestamp,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{new Timestamp(1432102334184l), new Timestamp(1432102334184l), new Timestamp(1432102334184l)};
@@ -224,7 +232,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,length2) FROM " + REGIONS
+            "SELECT ARRAY_FILL(timestamp,length2) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -242,7 +250,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,length1) FROM " + REGIONS
+            "SELECT ARRAY_FILL(timestamp,length1) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -260,7 +268,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(ARRAY_ELEM(ARRAY[23,45],1),3) FROM " + REGIONS
+            "SELECT ARRAY_FILL(ARRAY_ELEM(ARRAY[23,45],1),3) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -279,7 +287,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL('hello', ARRAY_LENGTH(ARRAY[34, 45])) FROM " + REGIONS
+            "SELECT ARRAY_FILL('hello', ARRAY_LENGTH(ARRAY[34, 45])) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -298,7 +306,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(3.4, ARRAY_LENGTH(ARRAY[34, 45])) FROM " + REGIONS
+            "SELECT ARRAY_FILL(3.4, ARRAY_LENGTH(ARRAY[34, 45])) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -491,7 +499,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT region_name FROM " + REGIONS + " WHERE ARRAY[12, 12, 12, 12]=ARRAY_FILL(12,4)");
+            "SELECT region_name FROM " + tableName + " WHERE ARRAY[12, 12, 12, 12]=ARRAY_FILL(12,4)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -505,7 +513,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT region_name FROM " + REGIONS + " WHERE varchar=ANY(ARRAY_FILL('foo',3))");
+            "SELECT region_name FROM " + tableName + " WHERE varchar=ANY(ARRAY_FILL('foo',3))");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -518,7 +526,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
         
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + REGIONS
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE ARRAY['2345', '2345', '2345', '2345']=ARRAY_FILL('2345', 4)");
         assertTrue(rs.next());
 
@@ -532,7 +540,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
         
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + REGIONS
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE ARRAY[23.45, 23.45, 23.45]=ARRAY_FILL(23.45, 3)");
         assertTrue(rs.next());
 
@@ -546,7 +554,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
         
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + REGIONS
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE ARRAY['foo','foo','foo','foo','foo']=ARRAY_FILL(varchar,5)");
         assertTrue(rs.next());
 
@@ -561,7 +569,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT region_name FROM " + REGIONS + " WHERE varchars2=ARRAY_FILL('hello',3)");
+            "SELECT region_name FROM " + tableName + " WHERE varchars2=ARRAY_FILL('hello',3)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -575,7 +583,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT region_name FROM " + REGIONS + " WHERE ARRAY[2,2,2,2]=ARRAY_FILL(2,4)");
+            "SELECT region_name FROM " + tableName + " WHERE ARRAY[2,2,2,2]=ARRAY_FILL(2,4)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIT.java
deleted file mode 100644
index 14473f1..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIT.java
+++ /dev/null
@@ -1,69 +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 org.apache.hadoop.conf.Configuration;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-/**
- * Base class for tests that create unique tables for
- * every test enabling them to be
- * 1) run in parallel
- * 2) not have to delete any tables
- * 
- * We need to separate these from tests that rely on clients
- * to set timestamps, because we create/destroy the Phoenix tables
- * between tests and only allow a table time stamp to increase.
- * Without this separation table deletion/creation would fail.
- *
- * All tests extending this class use the mini cluster that is
- * shared by all classes extending this class
- *
- * Remember to use BaseTest.generateRandomString() to generate table
- * names for your tests otherwise there might be naming collisions between
- * other tests.
- */
-public abstract class BaseParallelIT extends BaseTest {
-    protected static Configuration getTestClusterConfig() {
-        // don't want callers to modify config.
-        return new Configuration(config);
-    }
-
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
-    }
-
-    @AfterClass
-    public static void doTeardown() throws Exception {
-        // no teardown since we are creating unique table names
-        // just destroy our test driver
-        destroyDriver();
-    }
-
-    @After
-    public void cleanUpAfterTest() throws Exception {
-        // no cleanup since we are using unique table names
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 686b39f..478b234 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -31,7 +31,6 @@ import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -43,20 +42,15 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.BeforeClass;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public abstract class BaseViewIT extends ParallelStatsEnabledIT {
 	
@@ -66,15 +60,6 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
 	protected String tableDDLOptions;
 	protected boolean transactional;
 
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        // TODO: don't repeat this
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     public BaseViewIT( boolean transactional) {
 		StringBuilder optionBuilder = new StringBuilder();
 		this.transactional = transactional;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CbrtFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CbrtFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CbrtFunctionEnd2EndIT.java
index 98aa819..d34b6ae 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CbrtFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CbrtFunctionEnd2EndIT.java
@@ -28,7 +28,6 @@ import java.sql.ResultSet;
 
 import org.apache.phoenix.expression.function.CbrtFunction;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -38,20 +37,22 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private static final String KEY = "key";
     private static final double ZERO = 1e-8;
-    private static final String TEST_SIGNED = generateUniqueName();
-    private static final String TEST_UNSIGNED = generateUniqueName();
+    private String signedTableName;
+    private String unsignedTableName;
 
-    @BeforeClass
-    public static void initTable() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        signedTableName = generateUniqueName();
+        unsignedTableName = generateUniqueName();
         Connection conn = null;
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE " + TEST_SIGNED
+            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 " + TEST_UNSIGNED
+            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();
@@ -62,7 +63,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private void updateSignedTable(Connection conn, double data) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
-            "UPSERT INTO " + TEST_SIGNED + " VALUES (?, ?, ?, ?, ?, ?, ?)");
+            "UPSERT INTO " + signedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -77,7 +78,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private void updateUnsignedTable(Connection conn, double data) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
-            "UPSERT INTO " + TEST_UNSIGNED + " VALUES (?, ?, ?, ?, ?, ?, ?)");
+            "UPSERT INTO " + unsignedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -94,7 +95,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         updateSignedTable(conn, data);
         ResultSet rs = conn.createStatement().executeQuery(
             "SELECT CBRT(doub),CBRT(fl),CBRT(inte),CBRT(lon),CBRT(smalli),CBRT(tinyi) FROM "
-                + TEST_SIGNED);
+                + signedTableName);
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.cbrt(d.doubleValue())) < ZERO);
@@ -104,7 +105,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         assertTrue(Math.abs(rs.getDouble(5) - Math.cbrt(d.shortValue())) < ZERO);
         assertTrue(Math.abs(rs.getDouble(6) - Math.cbrt(d.byteValue())) < ZERO);
         assertTrue(!rs.next());
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + TEST_SIGNED
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + signedTableName
             + " WHERE CBRT(doub)>0 AND CBRT(fl)>0 AND CBRT(inte)>0 AND CBRT(lon)>0 AND CBRT(smalli)>0 AND CBRT(tinyi)>0");
         rs = stmt.executeQuery();
         if (data > 0) {
@@ -118,7 +119,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         updateUnsignedTable(conn, data);
         ResultSet rs = conn.createStatement().executeQuery(
             "SELECT CBRT(doub),CBRT(fl),CBRT(inte),CBRT(lon),CBRT(smalli),CBRT(tinyi) FROM "
-                + TEST_UNSIGNED);
+                + unsignedTableName);
         assertTrue(rs.next());
         Double d = Double.valueOf(data);
         assertTrue(Math.abs(rs.getDouble(1) - Math.cbrt(d.doubleValue())) < ZERO);
@@ -128,7 +129,7 @@ public class CbrtFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         assertTrue(Math.abs(rs.getDouble(5) - Math.cbrt(d.shortValue())) < ZERO);
         assertTrue(Math.abs(rs.getDouble(6) - Math.cbrt(d.byteValue())) < ZERO);
         assertTrue(!rs.next());
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + TEST_UNSIGNED
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + unsignedTableName
             + " WHERE CBRT(doub)>0 AND CBRT(fl)>0 AND CBRT(inte)>0 AND CBRT(lon)>0 AND CBRT(smalli)>0 AND CBRT(tinyi)>0");
         rs = stmt.executeQuery();
         if (data > 0) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 10f970f..8eab9e2 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
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -36,26 +35,16 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class DisableLocalIndexIT extends ParallelStatsDisabledIT {
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        // Must update config before starting server
-        props.put(QueryServices.ALLOW_LOCAL_INDEX_ATTRIB, Boolean.FALSE.toString());
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
 
     @Test
     public void testDisabledLocalIndexes() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(QueryServices.ALLOW_LOCAL_INDEX_ATTRIB, Boolean.FALSE.toString());
+        Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         String baseName = generateUniqueName();
         String tableName = baseName+ "_TABLE";
@@ -78,9 +67,10 @@ public class DisableLocalIndexIT extends ParallelStatsDisabledIT {
             admin.close();
         }
         
-        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "t1");
-        Connection tsconn = DriverManager.getConnection(getUrl(), props);
+        Properties tsconnProps = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        tsconnProps.setProperty(QueryServices.ALLOW_LOCAL_INDEX_ATTRIB, Boolean.FALSE.toString());
+        tsconnProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "t1");
+        Connection tsconn = DriverManager.getConnection(getUrl(), tsconnProps);
         
         tsconn.createStatement().execute("CREATE VIEW " + viewName + "(V1 VARCHAR) AS SELECT * FROM " + tableName);
         tsconn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + viewName + "(V1)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 87edc25..3d88a43 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
@@ -32,19 +32,21 @@ import java.util.Properties;
 
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 public class DistinctPrefixFilterIT extends ParallelStatsDisabledIT {
-    private static final String testTableF = generateUniqueName();
-    private static final String testTableV = generateUniqueName();
-    private static final String testSeq = testTableF + "_seq";
     private static final String PREFIX = "SERVER DISTINCT PREFIX";
-    private static Connection conn;
+    private String testTableF;
+    private String testTableV;
+    private String testSeq;
+    private Connection conn;
 
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        ParallelStatsDisabledIT.doSetup();
+    @Before
+    public void initTables() throws Exception {
+        testTableF = generateUniqueName();
+        testTableV = generateUniqueName();
+        testSeq = "SEQ_" + generateUniqueName();
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
@@ -367,7 +369,7 @@ public class DistinctPrefixFilterIT extends ParallelStatsDisabledIT {
         assertFalse(res.next());
     }
 
-    private static void insertPrefixF(int prefix1, int prefix2) throws SQLException {
+    private void insertPrefixF(int prefix1, int prefix2) throws SQLException {
         String query = "UPSERT INTO " + testTableF
                 + "(prefix1, prefix2, prefix3, col1, col2) VALUES(?,?,NEXT VALUE FOR "+testSeq+",rand(), trunc(rand()*1000))";
             PreparedStatement stmt = conn.prepareStatement(query);
@@ -376,7 +378,7 @@ public class DistinctPrefixFilterIT extends ParallelStatsDisabledIT {
             stmt.execute();
     }
 
-    private static void insertPrefixV(String prefix1, String prefix2) throws SQLException {
+    private void insertPrefixV(String prefix1, String prefix2) throws SQLException {
         String query = "UPSERT INTO " + testTableV
                 + "(prefix1, prefix2, prefix3, col1, col2) VALUES(?,?,NEXT VALUE FOR "+testSeq+",rand(), trunc(rand()*1000))";
             PreparedStatement stmt = conn.prepareStatement(query);
@@ -385,7 +387,7 @@ public class DistinctPrefixFilterIT extends ParallelStatsDisabledIT {
             stmt.execute();
     }
 
-    private static void multiply() throws SQLException {
+    private void multiply() throws SQLException {
         conn.prepareStatement("UPSERT INTO " + testTableF
                 + " SELECT prefix1,prefix2,NEXT VALUE FOR "+testSeq+",rand(), trunc(rand()*1000) FROM "+testTableF).execute();
         conn.prepareStatement("UPSERT INTO " + testTableV

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 3fc05a7..25e7230 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
@@ -18,7 +18,6 @@
 
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.HBASE_DYNAMIC_COLUMNS;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -48,7 +47,6 @@ import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -63,14 +61,15 @@ public class DynamicColumnIT extends ParallelStatsDisabledIT {
     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";
+    private String tableName;
 
-    @BeforeClass
-    public static void doBeforeTestSetup() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        tableName = generateUniqueName();
         try (PhoenixConnection pconn = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class)) {
             ConnectionQueryServices services = pconn.getQueryServices();
             try (HBaseAdmin admin = services.getAdmin()) {
-                HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TESTTBL"));
+                HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
                 htd.addFamily(new HColumnDescriptor(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES));
                 htd.addFamily(new HColumnDescriptor(FAMILY_NAME_A));
                 htd.addFamily(new HColumnDescriptor(FAMILY_NAME_B));
@@ -101,7 +100,13 @@ public class DynamicColumnIT extends ParallelStatsDisabledIT {
 
                 // 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(), tableName, HBASE_DYNAMIC_COLUMNS);
+                pconn.createStatement().execute("create table " + tableName + 
+                "   (entry varchar not null," +
+                "    F varchar," +
+                "    A.F1v1 varchar," +
+                "    A.F1v2 varchar," +
+                "    B.F2v1 varchar" +
+                "    CONSTRAINT pk PRIMARY KEY (entry))");
             }
 
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java
index 98d1478..31afb49 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GetSetByteBitFunctionEnd2EndIT.java
@@ -32,7 +32,6 @@ import org.apache.phoenix.expression.function.GetByteFunction;
 import org.apache.phoenix.expression.function.SetBitFunction;
 import org.apache.phoenix.expression.function.SetByteFunction;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -42,10 +41,11 @@ import org.junit.Test;
 public class GetSetByteBitFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private static final String KEY = "key";
-    private static final String TABLE_NAME = generateUniqueName();
+    private String TABLE_NAME;
 
-    @BeforeClass
-    public static void initTable() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        TABLE_NAME = generateUniqueName();
         Connection conn = null;
         PreparedStatement stmt = null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 5f04ead..52e8006 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
@@ -28,18 +28,12 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.Statement;
-import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class HashJoinMoreIT extends ParallelStatsDisabledIT {
     private final String[] plans = new String[] {
             /*
@@ -91,16 +85,6 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
             "    DYNAMIC SERVER FILTER BY (LHS.COL0, LHS.COL1, LHS.COL2) IN ((RHS.COL1, RHS.COL2, TO_INTEGER((RHS.COL3 - 1))))",            
     };
     
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Forces server cache to be used
-        props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     @Test
     public void testJoinOverSaltedTables() throws Exception {
         String tempTableNoSalting = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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
deleted file mode 100644
index 87a0b4b..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
+++ /dev/null
@@ -1,42 +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 java.util.Map;
-
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
-
-import com.google.common.collect.Maps;
-
-
-public class InMemoryOrderByIT extends OrderByIT {
-
-    public InMemoryOrderByIT() {
-    }
-
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.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));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
index e3a541d..cfbc9eb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexExtendedIT.java
@@ -374,9 +374,9 @@ public class IndexExtendedIT extends BaseOwnClusterIT {
         }
     }
 
-    private Connection getConnectionForLocalIndexTest() throws SQLException{
+    private static Connection getConnectionForLocalIndexTest() throws SQLException{
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.TRUE.toString());
         return DriverManager.getConnection(getUrl(),props);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 f522817..3dba4d5 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
@@ -30,21 +30,21 @@ import java.sql.SQLException;
 import java.sql.Statement;
 
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class LikeExpressionIT extends ParallelStatsDisabledIT {
 
-    private static final String TEST_TABLE = generateUniqueName();
+    private String tableName;
 
-    @BeforeClass
-    public static void doBeforeTestSetup() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        tableName = generateUniqueName();
         Connection conn = null;
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE " + TEST_TABLE + " (k VARCHAR NOT NULL PRIMARY KEY, i INTEGER)";
+            ddl = "CREATE TABLE " + tableName + " (k VARCHAR NOT NULL PRIMARY KEY, i INTEGER)";
             conn.createStatement().execute(ddl);
             conn.commit();
         } finally {
@@ -58,9 +58,9 @@ public class LikeExpressionIT extends ParallelStatsDisabledIT {
         insertRow(conn, "321n7-App-2-", 32);
     }
 
-    private static void insertRow(Connection conn, String k, int i) throws SQLException {
+    private void insertRow(Connection conn, String k, int i) throws SQLException {
         PreparedStatement stmt = conn.prepareStatement(
-            "UPSERT INTO " + TEST_TABLE + " VALUES (?, ?)");
+            "UPSERT INTO " + tableName + " VALUES (?, ?)");
         stmt.setString(1, k);
         stmt.setInt(2, i);
         stmt.executeUpdate();
@@ -69,7 +69,7 @@ public class LikeExpressionIT extends ParallelStatsDisabledIT {
 
     private void testLikeExpression(Connection conn, String likeStr, int numResult, int expectedSum)
             throws Exception {
-        String cmd = "select k, i from " + TEST_TABLE + " where k like '" + likeStr + "'";
+        String cmd = "select k, i from " + tableName + " where k like '" + likeStr + "'";
         Statement stmt = conn.createStatement();
         ResultSet rs = stmt.executeQuery(cmd);
         int sum = 0;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
index 4575867..19a208d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsDisabledIT.java
@@ -18,13 +18,21 @@
 
 package org.apache.phoenix.end2end;
 
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
 /**
- * Base class for tests that do not have statistics enabled.
- *
+ * Base class for tests whose methods run in parallel with statistics disabled.
+ * You must create unique names using {@link #generateUniqueName()} for each
+ * table and sequence used to prevent collisions.
  */
 @Category(ParallelStatsDisabledTest.class)
-public abstract class ParallelStatsDisabledIT extends BaseParallelIT {
+public abstract class ParallelStatsDisabledIT extends BaseTest {
 
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
index 322cb9e..312b49d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 
 import java.util.Map;
 
+import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
@@ -28,17 +29,16 @@ import org.junit.experimental.categories.Category;
 import com.google.common.collect.Maps;
 
 /**
- * 
- * Base class for tests that have statistics enabled.
- *
+ * Base class for tests whose methods run in parallel with statistics enabled.
+ * You must create unique names using {@link #generateUniqueName()} for each
+ * table and sequence used to prevent collisions.
  */
 @Category(ParallelStatsEnabledTest.class)
-public abstract class ParallelStatsEnabledIT extends BaseParallelIT {
+public abstract class ParallelStatsEnabledIT extends BaseTest {
     
     @BeforeClass
-    @Shadower(classBeingShadowed = BaseParallelIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(5);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PowerFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PowerFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PowerFunctionEnd2EndIT.java
index 1fb45c7..2d26be8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PowerFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PowerFunctionEnd2EndIT.java
@@ -27,7 +27,6 @@ import java.sql.ResultSet;
 
 import org.apache.phoenix.expression.function.PowerFunction;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -37,8 +36,8 @@ public class PowerFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private static final String KEY = "key";
     private static final double ZERO = 1e-9;
-    private static final String TEST_SIGNED = generateUniqueName();
-    private static final String TEST_UNSIGNED = generateUniqueName();
+    private String signedTableName;
+    private String unsignedTableName;
 
     private static boolean twoDoubleEquals(double a, double b) {
         if (Double.isNaN(a) ^ Double.isNaN(b)) return false;
@@ -55,17 +54,19 @@ public class PowerFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         }
     }
 
-    @BeforeClass
-    public static void initTable() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        signedTableName = generateUniqueName();
+        unsignedTableName = generateUniqueName();
         Connection conn = null;
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE " + TEST_SIGNED
+            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 " + TEST_UNSIGNED
+            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();
@@ -140,8 +141,8 @@ public class PowerFunctionEnd2EndIT extends ParallelStatsDisabledIT {
     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, TEST_SIGNED);
-            if (d >= 0) testNumberSpec(conn, d, TEST_UNSIGNED);
+            testNumberSpec(conn, d, signedTableName);
+            if (d >= 0) testNumberSpec(conn, d, unsignedTableName);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
index c60b403..04d1126 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryWithOffsetIT.java
@@ -31,22 +31,17 @@ import java.sql.SQLException;
 import java.sql.Types;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class QueryWithOffsetIT extends ParallelStatsDisabledIT {
     
@@ -57,15 +52,6 @@ public class QueryWithOffsetIT extends ParallelStatsDisabledIT {
     private String ddl;
     private String tableName;
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(true));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
     public QueryWithOffsetIT(String preSplit) {
         this.isSalted = preSplit.startsWith(" SALT_BUCKETS");
         this.preSplit = preSplit;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SerialIteratorsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SerialIteratorsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SerialIteratorsIT.java
index 37da39e..1360a08 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SerialIteratorsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SerialIteratorsIT.java
@@ -25,18 +25,13 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryServices;
 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 SerialIteratorsIT extends ParallelStatsDisabledIT {
     private String tableName = generateUniqueName();
     private final String[] strings = { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p",
@@ -45,19 +40,17 @@ public class SerialIteratorsIT extends ParallelStatsDisabledIT {
             + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
             + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) SPLIT ON ('e','i','o')";
 
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Don't force row key order
-        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
     
     @Test
     public void testConcatenatingSerialIterators() throws Exception {
         Connection conn;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        conn = getConnection();
         createTestTable(getUrl(), ddl);
         initTableValues(conn);
         String query = "SELECT t_id from " + tableName + " order by t_id desc limit " + 10;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SignFunctionEnd2EndIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SignFunctionEnd2EndIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SignFunctionEnd2EndIT.java
index 34ccdde..e3458dd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SignFunctionEnd2EndIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SignFunctionEnd2EndIT.java
@@ -29,7 +29,6 @@ import java.sql.ResultSet;
 
 import org.apache.phoenix.expression.function.SignFunction;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -39,20 +38,22 @@ import org.junit.Test;
 public class SignFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private static final String KEY = "key";
-    private static final String TEST_SIGNED = generateUniqueName();
-    private static final String TEST_UNSIGNED = generateUniqueName();
+    private String signedTableName;
+    private String unsignedTableName;
 
-    @BeforeClass
-    public static void initTable() throws Exception {
+    @Before
+    public void initTable() throws Exception {
+        signedTableName = generateUniqueName();
+        unsignedTableName = generateUniqueName();
         Connection conn = null;
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl());
             String ddl;
-            ddl = "CREATE TABLE " + TEST_SIGNED
+            ddl = "CREATE TABLE " + signedTableName
                 + " (k VARCHAR NOT NULL PRIMARY KEY, dec DECIMAL, doub DOUBLE, fl FLOAT, inte INTEGER, lon BIGINT, smalli SMALLINT, tinyi TINYINT)";
             conn.createStatement().execute(ddl);
-            ddl = "CREATE TABLE " + TEST_UNSIGNED
+            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();
@@ -63,7 +64,7 @@ public class SignFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private void updateSignedTable(Connection conn, double data) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
-            "UPSERT INTO " + TEST_SIGNED + " VALUES (?, ?, ?, ?, ?, ?, ?, ?)");
+            "UPSERT INTO " + signedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setBigDecimal(2, BigDecimal.valueOf(data));
@@ -79,7 +80,7 @@ public class SignFunctionEnd2EndIT extends ParallelStatsDisabledIT {
 
     private void updateUnsignedTable(Connection conn, double data) throws Exception {
         PreparedStatement stmt = conn.prepareStatement(
-            "UPSERT INTO " + TEST_UNSIGNED + " VALUES (?, ?, ?, ?, ?, ?, ?)");
+            "UPSERT INTO " + unsignedTableName + " VALUES (?, ?, ?, ?, ?, ?, ?)");
         stmt.setString(1, KEY);
         Double d = Double.valueOf(data);
         stmt.setDouble(2, d.doubleValue());
@@ -96,14 +97,14 @@ public class SignFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         updateSignedTable(conn, data);
         ResultSet rs = conn.createStatement().executeQuery(
             "SELECT SIGN(dec),SIGN(doub),SIGN(fl),SIGN(inte),SIGN(lon),SIGN(smalli),SIGN(tinyi) FROM "
-                + TEST_SIGNED);
+                + signedTableName);
         assertTrue(rs.next());
         for (int i = 1; i <= 7; ++i) {
             assertEquals(rs.getInt(i), expected);
         }
         assertTrue(!rs.next());
 
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + TEST_SIGNED
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + signedTableName
             + " WHERE SIGN(dec)=? AND SIGN(doub)=? AND SIGN(fl)=? AND SIGN(inte)=? AND SIGN(lon)=? AND SIGN(smalli)=? AND SIGN(tinyi)=?");
         for (int i = 1; i <= 7; ++i)
             stmt.setInt(i, expected);
@@ -117,14 +118,14 @@ public class SignFunctionEnd2EndIT extends ParallelStatsDisabledIT {
         updateUnsignedTable(conn, data);
         ResultSet rs = conn.createStatement().executeQuery(
             "SELECT SIGN(doub),SIGN(fl),SIGN(inte),SIGN(lon),SIGN(smalli),SIGN(tinyi) FROM "
-                + TEST_UNSIGNED);
+                + unsignedTableName);
         assertTrue(rs.next());
         for (int i = 1; i <= 6; ++i) {
             assertEquals(rs.getInt(i), expected);
         }
         assertTrue(!rs.next());
 
-        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + TEST_UNSIGNED
+        PreparedStatement stmt = conn.prepareStatement("SELECT k FROM " + unsignedTableName
             + " WHERE SIGN(doub)=? AND SIGN(fl)=? AND SIGN(inte)=? AND SIGN(lon)=? AND SIGN(smalli)=? AND SIGN(tinyi)=?");
         for (int i = 1; i <= 6; ++i)
             stmt.setInt(i, expected);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 efb662b..e4add9a 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
@@ -28,7 +28,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Map;
+import java.util.Properties;
 
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -36,12 +36,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 
 public class SkipScanAfterManualSplitIT extends ParallelStatsDisabledIT {
 
@@ -59,22 +57,15 @@ public class SkipScanAfterManualSplitIT extends ParallelStatsDisabledIT {
     private static final int MIN_CHAR = 'a';
     private static final int MAX_CHAR = 'z';
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
-        // needed for 64 region parallelization due to splitting
-        // props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(64));
-        props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(32));
-        // enables manual splitting on salted tables
-        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
-        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1000));
-        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
     
     private static void initTable(String tableName) throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         conn.createStatement().execute("CREATE TABLE " + tableName + "("
                 + "a VARCHAR PRIMARY KEY, b VARCHAR) " 
                 + HTableDescriptor.MAX_FILESIZE + "=" + MAX_FILESIZE + ","
@@ -109,7 +100,7 @@ public class SkipScanAfterManualSplitIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         byte[] tableNameBytes = Bytes.toBytes(tableName);
         initTable(tableName);
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         int nRegions = services.getAllTableRegions(tableNameBytes).size();
         int nInitialRegions = nRegions;
@@ -280,7 +271,7 @@ public class SkipScanAfterManualSplitIT extends ParallelStatsDisabledIT {
      */
     @Test
     public void testSkipScanInListOfRVCAfterManualSplit() throws SQLException {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + " ( "
             + "organization_id CHAR(15) NOT NULL, "
@@ -349,7 +340,7 @@ public class SkipScanAfterManualSplitIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testMinMaxRangeIntersection() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         String tableName = generateUniqueName();
         PreparedStatement stmt = conn.prepareStatement("create table " + tableName
             + "(pk1 UNSIGNED_TINYINT NOT NULL, pk2 UNSIGNED_TINYINT NOT NULL, kv VARCHAR "

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 df18093..a08ba39 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
@@ -27,30 +27,14 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.Statement;
-import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
     
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Forces server cache to be used
-        props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     @Test
     public void testJoinOverSaltedTables() throws Exception {
         String tempTableNoSalting = "TEMP_TABLE_NO_SALTING"  + generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
deleted file mode 100644
index 2f0bc61..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
+++ /dev/null
@@ -1,40 +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 java.util.Map;
-
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
-
-import com.google.common.collect.Maps;
-
-
-public class SpooledOrderByIT extends OrderByIT {
-
-    @BeforeClass
-    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(100));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledSortMergeJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledSortMergeJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledSortMergeJoinIT.java
deleted file mode 100644
index c646622..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledSortMergeJoinIT.java
+++ /dev/null
@@ -1,45 +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 java.util.Map;
-
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.junit.BeforeClass;
-
-import com.google.common.collect.Maps;
-
-
-public class SpooledSortMergeJoinIT extends SortMergeJoinIT {
-
-    public SpooledSortMergeJoinIT(String[] indexDDL, String[] plans) {
-        super(indexDDL, plans);
-    }
-
-    @BeforeClass
-    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);;
-        props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(100));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 51bcb31..9dc82bf 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
@@ -23,53 +23,50 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
 import com.google.common.io.Files;
 
-
-
 public class SpooledTmpFileDeleteIT extends ParallelStatsDisabledIT {
-	
-    private Connection conn = null;
-    private Properties props = null;
+	private static final String PRINCIPAL = "noRenewLease";
     private File spoolDir;
 	private String tableName;
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
-    public static void doSetup() throws Exception {
-        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
-        // disable renewing leases. This will force spooling to happen.
-        props.put(QueryServices.RENEW_LEASE_ENABLED, Boolean.toString(false));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private Connection getConnection() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(QueryServices.SPOOL_DIRECTORY, spoolDir.getPath());
+        props.setProperty(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(1));
+        props.setProperty(QueryServices.RENEW_LEASE_ENABLED, Boolean.toString(false));
+        // Ensures round robin off so that spooling is used.
+        // TODO: review with Samarth - should a Noop iterator be used if pacing is not possible?
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(true));
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        String url = QueryUtil.getConnectionUrl(props, config, PRINCIPAL);
+        return DriverManager.getConnection(url, props);
     }
-	
+    
 	@Before 
-	public void setup() throws SQLException {
+	public void setup() throws Exception {
 		tableName = generateUniqueName();
-		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 " + 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();
+        try (Connection conn = getConnection()) {
+    		Statement stmt = conn.createStatement();
+    		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();
+        }
 	}
 	
 	@After
@@ -100,7 +97,8 @@ public class SpooledTmpFileDeleteIT extends ParallelStatsDisabledIT {
 			file.delete();
 		}
 
-		String query = "select * from " + tableName + "";
+		String query = "select * from " + tableName;
+		Connection conn = getConnection();
 		Statement statement = conn.createStatement();
 		ResultSet rs = statement.executeQuery(query);
 		assertTrue(rs.next());
@@ -122,7 +120,7 @@ public class SpooledTmpFileDeleteIT extends ParallelStatsDisabledIT {
 			fileNames.add(file.getName());
 		}
 
-		Connection conn2 = DriverManager.getConnection(getUrl(), props);
+		Connection conn2 = getConnection();
 		String query2 = "select * from " + tableName + "";
 		Statement statement2 = conn2.createStatement();
 		ResultSet rs2 = statement2.executeQuery(query2);


[2/3] phoenix git commit: PHOENIX-3327 Use same driver for all tests within a group

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 0b5a92d..2445948 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -34,7 +34,6 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
@@ -50,18 +49,14 @@ 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.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class StatsCollectorIT extends ParallelStatsEnabledIT {
     private final String tableDDLOptions;
@@ -69,23 +64,18 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
     private String schemaName;
     private String fullTableName;
         
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsEnabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(10);
-        // Must update config before starting server
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
-        props.put(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.put(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(Long.MAX_VALUE));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     public StatsCollectorIT( boolean transactional) {
         this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
     }
     
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(Long.MAX_VALUE));
+        return DriverManager.getConnection(getUrl(), props);
+    }
+    
     @Before
     public void generateTableNames() {
         schemaName = TestUtil.DEFAULT_SCHEMA_NAME;
@@ -100,9 +90,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
 
     @Test
     public void testUpdateEmptyStats() throws Exception {
-        Connection conn;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(true);
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName +" ( k CHAR(1) PRIMARY KEY )"  + tableDDLOptions);
@@ -118,9 +106,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
     
     @Test
     public void testSomeUpdateEmptyStats() throws Exception {
-        Connection conn;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(true);
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName +" ( k VARCHAR PRIMARY KEY, a.v1 VARCHAR, b.v2 VARCHAR ) " + tableDDLOptions + (tableDDLOptions.isEmpty() ? "" : ",") + "SALT_BUCKETS = 3");
@@ -158,7 +144,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        conn = getConnection();
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
                         + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
@@ -178,9 +164,6 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        conn.close();
-        conn = DriverManager.getConnection(getUrl(), props);
-        // This analyze would not work
         stmt = conn.prepareStatement("UPDATE STATISTICS " + tableName);
         stmt.execute();
         rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName);
@@ -189,11 +172,9 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
     }
 
     private void testNoDuplicatesAfterUpdateStats(String splitKey) throws Throwable {
-        Connection conn;
+        Connection conn = getConnection();
         PreparedStatement stmt;
         ResultSet rs;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement()
                 .execute("CREATE TABLE " + fullTableName
                         + " ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))"+ tableDDLOptions
@@ -229,7 +210,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        conn = getConnection();
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
                         + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
@@ -264,7 +245,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         stmt.setArray(3, array);
         stmt.execute();
         conn.close();
-        conn = DriverManager.getConnection(getUrl(), props);
+        conn = getConnection();
         // This analyze would not work
         stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
@@ -277,7 +258,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
             InterruptedException {
         Connection conn;
         PreparedStatement stmt;
-        conn = DriverManager.getConnection(getUrl(), props);
+        conn = getConnection();
         stmt = upsertStmt(conn, tableName);
         stmt.setString(1, "a");
         String[] s = new String[] { "abc", "def", "ghi", "jkll", null, null, "xxx" };
@@ -363,10 +344,8 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
     
     private void testCompactUpdatesStats(Integer minStatsUpdateFreq, String tableName) throws Exception {
         int nRows = 10;
-        Connection conn;
+        Connection conn = getConnection();
         PreparedStatement stmt;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("CREATE TABLE " + tableName + "(k CHAR(1) PRIMARY KEY, v INTEGER, w INTEGER) "
                 + HColumnDescriptor.KEEP_DELETED_CELLS + "=" + Boolean.FALSE);
         stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
@@ -424,10 +403,8 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
     @Test
     public void testWithMultiCF() throws Exception {
         int nRows = 20;
-        Connection conn;
+        Connection conn = getConnection();
         PreparedStatement stmt;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName
                         + "(k VARCHAR PRIMARY KEY, a.v INTEGER, b.v INTEGER, c.v INTEGER NULL, d.v INTEGER NULL) ");
@@ -505,9 +482,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
 
     @Test
     public void testRowCountAndByteCounts() throws SQLException {
-        Connection conn;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         String ddl = "CREATE TABLE " + fullTableName + " (t_id VARCHAR NOT NULL,\n" + "k1 INTEGER NOT NULL,\n"
                 + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
                 + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) split on ('e','j','o')";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringToArrayFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringToArrayFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringToArrayFunctionIT.java
index c11d765..1afe6ef 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringToArrayFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StringToArrayFunctionIT.java
@@ -21,36 +21,41 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-import java.sql.*;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.types.PhoenixArray;
-import org.junit.BeforeClass;
+import org.junit.Before;
 import org.junit.Test;
 
 public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
-    private static final String TABLE_NAME = generateUniqueName();
-
-    @BeforeClass
-    public static void initTables() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE " + TABLE_NAME
-            + " (region_name VARCHAR PRIMARY KEY, string1 VARCHAR, string2 CHAR(50), delimiter1 VARCHAR, delimiter2 CHAR(20), nullstring1 VARCHAR, nullstring2 CHAR(20))";
-        conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO " + TABLE_NAME
-            + "(region_name, string1, string2, delimiter1, delimiter2, nullstring1, nullstring2) VALUES('SF Bay Area',"
-            +
-                "'a,b,c,d'," +
-                "'1.2.3.4'," +
-                "','," +
-                "'.'," +
-                "'c'," +
-                "'3'" +
-                ")";
-        PreparedStatement stmt = conn.prepareStatement(dml);
-        stmt.execute();
-        conn.commit();
+    private String tableName;
+
+    @Before
+    public void initTables() throws Exception {
+        tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE " + tableName
+                + " (region_name VARCHAR PRIMARY KEY, string1 VARCHAR, string2 CHAR(50), delimiter1 VARCHAR, delimiter2 CHAR(20), nullstring1 VARCHAR, nullstring2 CHAR(20))";
+            conn.createStatement().execute(ddl);
+            String dml = "UPSERT INTO " + tableName
+                + "(region_name, string1, string2, delimiter1, delimiter2, nullstring1, nullstring2) VALUES('SF Bay Area',"
+                +
+                    "'a,b,c,d'," +
+                    "'1.2.3.4'," +
+                    "','," +
+                    "'.'," +
+                    "'c'," +
+                    "'3'" +
+                    ")";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.execute();
+            conn.commit();
+        }
     }
 
     @Test
@@ -59,7 +64,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string1, delimiter1) FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string1, delimiter1) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -75,7 +80,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string1, delimiter1, nullstring1) FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string1, delimiter1, nullstring1) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -91,7 +96,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string1, delimiter1, 'a') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string1, delimiter1, 'a') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -107,7 +112,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string1, delimiter1, 'd') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string1, delimiter1, 'd') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -123,7 +128,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string2, delimiter2) FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string2, delimiter2) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -139,7 +144,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string2, delimiter2, nullstring2) FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string2, delimiter2, nullstring2) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -155,7 +160,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string2, delimiter2, '1') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string2, delimiter2, '1') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -171,7 +176,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(string2, delimiter2, '4') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(string2, delimiter2, '4') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -187,7 +192,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY(region_name, ' ', '4') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY(region_name, ' ', '4') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -203,7 +208,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY('hello,hello,hello', delimiter1) FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY('hello,hello,hello', delimiter1) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -219,7 +224,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY('a,hello,hello,hello,b', ',', 'hello') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY('a,hello,hello,hello,b', ',', 'hello') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -235,7 +240,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT STRING_TO_ARRAY('b.a.b', delimiter2, 'b') FROM " + TABLE_NAME
+            "SELECT STRING_TO_ARRAY('b.a.b', delimiter2, 'b') FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -251,7 +256,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_LENGTH(STRING_TO_ARRAY('a, b, c', ', ')) FROM " + TABLE_NAME
+            "SELECT ARRAY_LENGTH(STRING_TO_ARRAY('a, b, c', ', ')) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -266,7 +271,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
             "SELECT STRING_TO_ARRAY(ARRAY_TO_STRING(ARRAY['a', 'b', 'c'], delimiter2), delimiter2, 'b') FROM "
-                + TABLE_NAME + " WHERE region_name = 'SF Bay Area'");
+                + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         PhoenixArray expected = new PhoenixArray(PVarchar.INSTANCE, new Object[]{"a", null, "c"});
@@ -282,7 +287,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
             "SELECT STRING_TO_ARRAY(ARRAY_TO_STRING(ARRAY['a', 'b', 'c'], delimiter2), ARRAY_ELEM(ARRAY[',', '.'], 2), 'b') FROM "
-                + TABLE_NAME + " WHERE region_name = 'SF Bay Area'");
+                + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         PhoenixArray expected = new PhoenixArray(PVarchar.INSTANCE, new Object[]{"a", null, "c"});
@@ -435,7 +440,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl());
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + TABLE_NAME
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE ARRAY['a', 'b', 'c', 'd']=STRING_TO_ARRAY(string1, delimiter1)");
         assertTrue(rs.next());
 
@@ -448,7 +453,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl());
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + TABLE_NAME
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE 'a'=ANY(STRING_TO_ARRAY(string1, delimiter1))");
         assertTrue(rs.next());
 
@@ -461,7 +466,7 @@ public class StringToArrayFunctionIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl());
 
         ResultSet rs;
-        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + TABLE_NAME
+        rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
             + " WHERE 'a'=ALL(STRING_TO_ARRAY('a,a,a,', delimiter1))");
         assertTrue(rs.next());
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
index 18e6e5a..b8c028b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
@@ -25,33 +25,18 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class TransactionalViewIT extends ParallelStatsEnabledIT {
 
     private String fullTableName;
     private String fullViewName;
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsEnabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     @Before
     public void generateTableNames() {
         String schemaName = TestUtil.DEFAULT_SCHEMA_NAME;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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
index 5f02790..06edbcf 100644
--- 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
@@ -25,28 +25,15 @@ 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.ParallelStatsDisabledIT;
 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 ParallelStatsDisabledIT {
 
-    @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())) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 8b3c111..4e7d06a 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,7 +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.*;
+import static org.apache.phoenix.util.TestUtil.HBASE_NATIVE_SCHEMA_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -27,7 +27,6 @@ import static org.junit.Assert.fail;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -40,7 +39,6 @@ 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.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -52,31 +50,36 @@ import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.junit.BeforeClass;
+import org.apache.phoenix.util.StringUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class DropMetadataIT extends ParallelStatsDisabledIT {
+    private static final String PRINCIPAL = "dropMetaData";
     private static final byte[] FAMILY_NAME = Bytes.toBytes(SchemaUtil.normalizeIdentifier("1"));
     public static final String SCHEMA_NAME = "";
-    private final String TENANT_SPECIFIC_URL = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
+    private final String TENANT_ID = "tenant1";
+
+    private Connection getConnection() throws Exception {
+        return getConnection(PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES));
+    }
     
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    @BeforeClass 
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        // Drop the HBase table metadata for this test
-        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private Connection getConnection(Properties props) throws Exception {
+        props.setProperty(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        // Force real driver to be used as the test one doesn't handle creating
+        // more than one ConnectionQueryService
+        props.setProperty(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, StringUtil.EMPTY_STRING);
+        // Create new ConnectionQueryServices so that we can set DROP_METADATA_ATTRIB
+        String url = QueryUtil.getConnectionUrl(props, config, PRINCIPAL);
+        return DriverManager.getConnection(url, props);
     }
     
     @Test
     public void testDropViewKeepsHTable() throws Exception {
-        HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TEST_PROPERTIES).getAdmin();
+        Connection conn = getConnection();
+        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
         String hbaseNativeViewName = generateUniqueName();
 
         byte[] hbaseNativeBytes = SchemaUtil.getTableNameAsBytes(HBASE_NATIVE_SCHEMA_NAME, hbaseNativeViewName);
@@ -91,8 +94,6 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
             admin.close();
         }
         
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("create view " + hbaseNativeViewName+
                 "   (uint_key unsigned_int not null," +
                 "    ulong_key unsigned_long not null," +
@@ -102,17 +103,16 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
                 "    CONSTRAINT pk PRIMARY KEY (uint_key, ulong_key, string_key))\n" +
                      HColumnDescriptor.DATA_BLOCK_ENCODING + "='" + DataBlockEncoding.NONE + "'");
         conn.createStatement().execute("drop view " + hbaseNativeViewName);
-
+        conn.close();
     }
     
     @Test
     public void testDroppingIndexedColDropsIndex() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String indexTableName = generateUniqueName();
         String dataTableFullName = SchemaUtil.getTableName(SCHEMA_NAME, generateUniqueName());
         String localIndexTableName1 = "LOCAL_" + indexTableName + "_1";
         String localIndexTableName2 = "LOCAL_" + indexTableName + "_2";
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+        try (Connection conn = getConnection()) {
             conn.setAutoCommit(false);
             conn.createStatement().execute(
                 "CREATE TABLE " + dataTableFullName
@@ -200,8 +200,10 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
     }
     
     public void helpTestDroppingIndexedColDropsViewIndex(boolean isMultiTenant) throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : conn ) {
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(TENANT_ID_ATTRIB, TENANT_ID);
+        try (Connection conn = getConnection();
+                Connection viewConn = isMultiTenant ? getConnection(props) : conn ) {
             String tableWithView = generateUniqueName();
             String viewOfTable = generateUniqueName();
             String viewIndex1 = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 d3c3085..51e9de9 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
@@ -25,30 +25,16 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Map;
 import java.util.regex.Pattern;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.*;
-import org.junit.BeforeClass;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 public class GlobalIndexOptimizationIT extends ParallelStatsDisabledIT {
 
-    @BeforeClass 
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Drop the HBase table metadata for this test
-        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
     private void createBaseTable(String tableName, Integer saltBuckets, String splits, boolean multiTenant) throws SQLException {
         Connection conn = DriverManager.getConnection(getUrl());
         String ddl = "CREATE TABLE " + tableName + " (t_id VARCHAR NOT NULL,\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 e2bc873..9a2eec1 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
@@ -38,7 +38,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.Cell;
@@ -55,7 +54,6 @@ import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -63,24 +61,19 @@ import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
 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;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 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;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class IndexIT extends ParallelStatsDisabledIT {
 
@@ -105,14 +98,6 @@ public class IndexIT extends ParallelStatsDisabledIT {
         this.tableDDLOptions = optionBuilder.toString();
     }
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
     @Parameters(name="IndexIT_localIndex={0},mutable={1},transactional={2}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 214b7ba..9d3da8e 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
@@ -194,7 +194,7 @@ public class LocalIndexIT extends ParallelStatsDisabledIT {
     
     private Connection getConnection() throws SQLException{
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
         return DriverManager.getConnection(getUrl(),props);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 0657d54..d740013 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -96,7 +96,7 @@ public class MutableIndexFailureIT extends BaseOwnClusterIT {
         this.tableName = (localIndex ? "L_" : "") + TestUtil.DEFAULT_DATA_TABLE_NAME + (transactional ? "_TXN" : "")
                 + (isNamespaceMapped ? "_NM" : "");
         this.indexName = INDEX_NAME;
-        this.fullTableName = SchemaUtil.getTableName(schema, tableName);
+        fullTableName = SchemaUtil.getTableName(schema, tableName);
         this.fullIndexName = SchemaUtil.getTableName(schema, indexName);
         this.isNamespaceMapped = isNamespaceMapped;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 0a89eff..1bc01d2 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
@@ -33,7 +33,6 @@ import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 
 import jline.internal.Log;
@@ -45,7 +44,6 @@ 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.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
@@ -53,16 +51,13 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
 import com.google.common.primitives.Doubles;
 
 @RunWith(Parameterized.class)
@@ -80,16 +75,17 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
 		this.tableDDLOptions = optionBuilder.toString();
 	}
     
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        // Forces server cache to be used
-        props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(1));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private static Connection getConnection(Properties props) throws SQLException {
+        props.setProperty(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(1));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
-	
+    
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        return getConnection(props);
+    }
+    
 	@Parameters(name="MutableIndexIT_localIndex={0},transactional={1}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {
@@ -99,8 +95,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCoveredColumnUpdates() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+        try (Connection conn = getConnection()) {
 	        conn.setAutoCommit(false);
 			String tableName = "TBL_" + generateUniqueName();
 			String indexName = "IDX_" + generateUniqueName();
@@ -199,12 +194,11 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCoveredColumns() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		String tableName = "TBL_" + generateUniqueName();
 		String indexName = "IDX_" + generateUniqueName();
 		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)) {
+        try (Connection conn = getConnection()) {
 
 	        conn.setAutoCommit(false);
 	        String query;
@@ -309,12 +303,11 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testCompoundIndexKey() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		String tableName = "TBL_" + generateUniqueName();
 		String indexName = "IDX_" + generateUniqueName();
 		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)) {
+        try (Connection conn = getConnection()) {
 	        conn.setAutoCommit(false);
 	        String query;
 	        ResultSet rs;
@@ -428,12 +421,11 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
      */
     @Test
     public void testMultipleUpdatesToSingleRow() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		String tableName = "TBL_" + generateUniqueName();
 		String indexName = "IDX_" + generateUniqueName();
 		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)) {
+        try (Connection conn = getConnection()) {
 	        conn.setAutoCommit(false);
 	        String query;
 	        ResultSet rs;
@@ -512,12 +504,11 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testUpsertingNullForIndexedColumns() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		String tableName = "TBL_" + generateUniqueName();
 		String indexName = "IDX_" + generateUniqueName();
 		String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
         String testTableName = tableName + "_" + System.currentTimeMillis();
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+        try (Connection conn = getConnection()) {
 	        conn.setAutoCommit(false);
 	        ResultSet rs;
     		Statement stmt = conn.createStatement();
@@ -602,9 +593,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
 		String tableName = "TBL_" + generateUniqueName();
 		String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 
-
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+        try (Connection conn = getConnection()) {
 	        conn.setAutoCommit(false);
 	        conn.createStatement().execute(
 	            "CREATE TABLE " + fullTableName +" (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) " + tableDDLOptions);
@@ -634,10 +623,10 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     }
 
     private void testSplitDuringIndexScan(boolean isReverse) throws Exception {
-        Properties props = new Properties();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         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());
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
+        Connection conn1 = getConnection(props);
 		String tableName = "TBL_" + generateUniqueName();
         String indexName = "IDX_" + generateUniqueName();
 		HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
@@ -688,7 +677,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testIndexHalfStoreFileReader() throws Exception {
-        Connection conn1 = DriverManager.getConnection(getUrl());
+        Connection conn1 = getConnection();
         HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();
 		String tableName = "TBL_" + generateUniqueName();
 		String indexName = "IDX_" + generateUniqueName();
@@ -840,7 +829,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
 	  String indexName = "IDX_" + generateUniqueName();
 	  String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 	  Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-      try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+      try (Connection conn = getConnection()) {
           conn.setAutoCommit(false);
           // create data table
           conn.createStatement().execute(
@@ -857,7 +846,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
           String dml = "UPSERT INTO " + fullTableName + " (ENTITY_ID, TYPE) VALUES ( ?, ?)";
           props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "tenant1");
           // connection is tenant-specific
-          try (Connection tenantConn = DriverManager.getConnection(getUrl(), props)) {
+          try (Connection tenantConn = getConnection(props)) {
               // upsert one row
               upsertRow(dml, tenantConn, 0);
               tenantConn.commit();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 5bf0c58..06e7ede 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
@@ -26,39 +26,33 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.util.Map;
+import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-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.*;
-import org.junit.BeforeClass;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
 
 public class SaltedIndexIT extends ParallelStatsDisabledIT {
     private static final int TABLE_SPLITS = 3;
     private static final int INDEX_SPLITS = 4;
     
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Forces server cache to be used
-        props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
-        // Drop the HBase table metadata for this test
-        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        // Must update config before starting server
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(1));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
     
     private static void makeImmutableAndDeleteData(String tableName, String fullTableName) throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
+        Connection conn = getConnection();
         try {
             conn.setAutoCommit(true);
             conn.createStatement().execute("DELETE FROM " + fullTableName);
@@ -108,8 +102,7 @@ public class SaltedIndexIT extends ParallelStatsDisabledIT {
         String query;
         ResultSet rs;
         
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(false);
         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;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 99a8b4d..1abbc87 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
@@ -82,7 +82,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
     
     private Connection getConnection() throws SQLException{
         Properties props = new Properties();
-        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
         return DriverManager.getConnection(getUrl(),props);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 3ffade0..cbfe9a5 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
@@ -30,27 +30,21 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 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;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class MutableRollbackIT extends ParallelStatsDisabledIT {
 	
@@ -60,23 +54,20 @@ public class MutableRollbackIT extends ParallelStatsDisabledIT {
 		this.localIndex = localIndex;
 	}
 	
-	@BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
-        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-	
 	@Parameters(name="MutableRollbackIT_localIndex={0}") // name is used by failsafe as file name in reports
     public static Collection<Boolean> data() {
         return Arrays.asList(new Boolean[] { false, true});
     }
 	
-    public void testRollbackOfUncommittedExistingKeyValueIndexUpdate() throws Exception {
+	private static Connection getConnection() throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
         Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
+	}
+	
+    public void testRollbackOfUncommittedExistingKeyValueIndexUpdate() throws Exception {
+        Connection conn = getConnection();
         String tableName1 = "TBL1_" + generateUniqueName();
         String indexName1 = "IDX1_" + generateUniqueName();
         String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
@@ -215,8 +206,7 @@ public class MutableRollbackIT extends ParallelStatsDisabledIT {
         String tableName2 = "TBL2_" + generateUniqueName();
         String indexName2 = "IDX2_" + generateUniqueName();
         String fullTableName2 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName2);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -350,12 +340,10 @@ public class MutableRollbackIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testMultiRollbackOfUncommittedExistingRowKeyIndexUpdate() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         String tableName1 = "TBL1_" + generateUniqueName();
         String indexName1 = "IDX1_" + generateUniqueName();
         String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
-        String tableName2 = "TBL2_" + generateUniqueName();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -456,12 +444,10 @@ public class MutableRollbackIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testCheckpointAndRollback() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         String tableName1 = "TBL1_" + generateUniqueName();
         String indexName1 = "IDX1_" + generateUniqueName();
         String fullTableName1 = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName1);
-        String tableName2 = "TBL2_" + generateUniqueName();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 5153a68..8cafcfc 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
@@ -25,27 +25,22 @@ import static org.junit.Assert.assertTrue;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 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;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class RollbackIT extends ParallelStatsDisabledIT {
 	
@@ -57,15 +52,13 @@ public class RollbackIT extends ParallelStatsDisabledIT {
 		this.mutable = mutable;
 	}
 	
-	@BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
-	
+    
 	@Parameters(name="RollbackIT_localIndex={0},mutable={1}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     
@@ -79,8 +72,7 @@ public class RollbackIT extends ParallelStatsDisabledIT {
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IDX_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -121,8 +113,7 @@ public class RollbackIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testRollbackOfUncommittedRowKeyIndexInsert() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(false);
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IDX_" + generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 8fcdebb..956deba 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
@@ -17,7 +17,8 @@
  */
 package org.apache.phoenix.iterate;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -28,7 +29,8 @@ import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.jdbc.PhoenixStatement;
-import org.junit.BeforeClass;
+import org.apache.phoenix.query.QueryServices;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -37,13 +39,33 @@ import org.junit.Test;
  */
 public class PhoenixQueryTimeoutIT extends ParallelStatsDisabledIT {
 
-    private static final String QUERY_TIMEOUT_TEST = generateUniqueName();
+    private String tableName;
+
+    @Before
+    public void createTableAndInsertRows() throws Exception {
+        tableName = generateUniqueName();
+        int numRows = 1000;
+        String ddl =
+            "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            for (int i = 1; i <= numRows; i++) {
+                String key = "key" + i;
+                stmt.setString(1, key);
+                stmt.setString(2, "value" + i);
+                stmt.executeUpdate();
+            }
+            conn.commit();
+        }
+    }
 
-    @Test
     /**
      * This test validates that we timeout as expected. It does do by
      * setting the timeout value to 1 ms.
      */
+    @Test
     public void testCustomQueryTimeoutWithVeryLowTimeout() throws Exception {
         // Arrange
         PreparedStatement ps = loadDataAndPrepareQuery(1, 1);
@@ -85,30 +107,12 @@ public class PhoenixQueryTimeoutIT extends ParallelStatsDisabledIT {
     
     private PreparedStatement loadDataAndPrepareQuery(int timeoutMs, int timeoutSecs) throws Exception, SQLException {
         Properties props = new Properties();
-        props.setProperty("phoenix.query.timeoutMs", String.valueOf(timeoutMs));
+        props.setProperty(QueryServices.THREAD_TIMEOUT_MS_ATTRIB, String.valueOf(timeoutMs));
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement ps = conn.prepareStatement("SELECT * FROM " + QUERY_TIMEOUT_TEST);
+        PreparedStatement ps = conn.prepareStatement("SELECT * FROM " + tableName);
         PhoenixStatement phoenixStmt = ps.unwrap(PhoenixStatement.class);
         assertEquals(timeoutMs, phoenixStmt.getQueryTimeoutInMillis());
         assertEquals(timeoutSecs, phoenixStmt.getQueryTimeout());
         return ps;
     }
-
-    @BeforeClass
-    public static void createTableAndInsertRows() throws Exception {
-        int numRows = 1000;
-        String ddl =
-            "CREATE TABLE " + QUERY_TIMEOUT_TEST + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)";
-        Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute(ddl);
-        String dml = "UPSERT INTO " + QUERY_TIMEOUT_TEST + " VALUES (?, ?)";
-        PreparedStatement stmt = conn.prepareStatement(dml);
-        for (int i = 1; i <= numRows; i++) {
-            String key = "key" + i;
-            stmt.setString(1, key);
-            stmt.setString(2, "value" + i);
-            stmt.executeUpdate();
-        }
-        conn.commit();
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 47261ac..c1f7c88 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
@@ -32,7 +32,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
@@ -43,7 +42,6 @@ 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.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -51,37 +49,31 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 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;
 import com.google.common.collect.Sets;
 
 public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
 
     private static final int NUM_SALT_BUCKETS = 4; 
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(32));
+    private static Connection getConnection() throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         /*  
          * Don't force row key order. This causes RoundRobinResultIterator to be used if there was no order by specified
          * on the query.
          */
-        props.put(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
-        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1000));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        props.setProperty(QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
-
+    
     @Test
     public void testRoundRobinAfterTableSplit() throws Exception {
         String tableName = generateUniqueName();
         byte[] tableNameBytes = Bytes.toBytes(tableName);
         int numRows = setupTableForSplit(tableName);
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         int nRegions = services.getAllTableRegions(tableNameBytes).size();
         int nRegionsBeforeSplit = nRegions;
@@ -131,7 +123,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         int numRows = 9;
         Set<String> expectedKeys = Collections.unmodifiableSet(createTableAndInsertRows(tableName, numRows, salted, false));
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         PreparedStatement stmt = conn.prepareStatement("SELECT K, V FROM " + tableName);
         tryWithFetchSize(new HashSet<>(expectedKeys), 1, stmt, 0);
         tryWithFetchSize(new HashSet<>(expectedKeys), 2, stmt, salted ? 2 : 5);
@@ -155,7 +147,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         int numRows = 6;
         Set<String> insertedKeys = createTableAndInsertRows(tableName, numRows, salted, false);
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         PreparedStatement stmt = conn.prepareStatement("SELECT K, V FROM " + tableName + " WHERE K = ?");
         stmt.setString(1, "key1"); // will return only 1 row
         int numRowsFiltered = 1;
@@ -187,7 +179,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
 
     private Set<String> createTableAndInsertRows(String tableName, int numRows, boolean salted, boolean addTableNameToKey) throws Exception {
         String ddl = "CREATE TABLE " + tableName + " (K VARCHAR NOT NULL PRIMARY KEY, V VARCHAR)" + (salted ? "SALT_BUCKETS=" + NUM_SALT_BUCKETS : "");
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         conn.createStatement().execute(ddl);
         String dml = "UPSERT INTO " + tableName + " VALUES (?, ?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
@@ -207,7 +199,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
     public void testFetchSizesAndRVCExpression() throws Exception {
         String tableName = generateUniqueName();
         Set<String> insertedKeys = Collections.unmodifiableSet(createTableAndInsertRows(tableName, 4, false, false));
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         PreparedStatement stmt = conn.prepareStatement("SELECT K FROM " + tableName + " WHERE (K, V)  > (?, ?)");
         stmt.setString(1, "key0");
         stmt.setString(2, "value0");
@@ -244,7 +236,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
 
         int MIN_CHAR = 'a';
         int MAX_CHAR = 'z';
-        Connection conn = DriverManager.getConnection(getUrl());
+        Connection conn = getConnection();
         conn.createStatement().execute("CREATE TABLE " + tableName + "("
                 + "a VARCHAR PRIMARY KEY, b VARCHAR) " 
                 + HTableDescriptor.MAX_FILESIZE + "=" + maxFileSize + ","
@@ -288,7 +280,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
         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());
+        Connection conn = getConnection();
         PreparedStatement stmt = conn.prepareStatement(query);
         stmt.setFetchSize(2); // force parallel fetch of scanner cache
         ResultSet rs = stmt.executeQuery();
@@ -317,7 +309,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
     @Test
     public void testBug2074() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         try {
             conn.createStatement().execute("CREATE TABLE EVENTS" 
                     + "   (id VARCHAR(10) PRIMARY KEY, " 
@@ -407,7 +399,7 @@ public class RoundRobinResultIteratorIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testIteratorsPickedInRoundRobinFashionForSaltedTable() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
+        try (Connection conn = getConnection()) {
             String testTable = "testIteratorsPickedInRoundRobinFashionForSaltedTable".toUpperCase();
             Statement stmt = conn.createStatement();
             stmt.execute("CREATE TABLE " + testTable + "(K VARCHAR PRIMARY KEY) SALT_BUCKETS = 8");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 930f454..ceb8714 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
@@ -33,29 +33,22 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.PhoenixRuntime;
 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.mockito.Mockito;
 
-import com.google.common.collect.Maps;
-
 /**
  * Verifies the number of rpcs calls from {@link MetaDataClient} updateCache() 
  * for transactional and non-transactional tables.
@@ -64,14 +57,6 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
 	
 	public static final int NUM_MILLIS_IN_DAY = 86400000;
 
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-    
     private static void setupSystemTable(String fullTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/phoenix-core/src/it/java/org/apache/phoenix/tx/NotThreadSafeTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/NotThreadSafeTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/NotThreadSafeTransactionIT.java
index 6fea975..b50f424 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/NotThreadSafeTransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/NotThreadSafeTransactionIT.java
@@ -27,32 +27,62 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.util.Map;
 
 import javax.annotation.concurrent.NotThreadSafe;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-
-@NotThreadSafe
+/**
+ * 
+ * Transaction related tests that flap when run in parallel.
+ * TODO: review with Tephra community
+ *
+ */
+@NotThreadSafe // Prevents test methods from running in parallel
 public class NotThreadSafeTransactionIT extends ParallelStatsDisabledIT {
-    
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    @Test
+    public void testDelete() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        String selectSQL = "SELECT * FROM " + fullTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            TestUtil.createTransactionalTable(conn, fullTableName);
+            conn1.setAutoCommit(false);
+            ResultSet rs = conn1.createStatement().executeQuery(selectSQL);
+            assertFalse(rs.next());
+            
+            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);
+            stmt.execute();
+            conn1.commit();
+            
+            TestUtil.setRowKeyColumns(stmt, 2);
+            stmt.execute();
+            
+            // verify rows can be read even though commit has not been called
+            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 " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(1, rs.getInt(1));
+            
+            conn1.commit();
+            
+            // verify rows are deleted after commit
+            rs = conn1.createStatement().executeQuery(selectSQL);
+            assertFalse(rs.next());
+        }
     }
-
+        
     @Test
     public void testInflightUpdateNotSeen() throws Exception {
         String transTableName = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 bf99c4b..809225a 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
@@ -33,7 +33,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -47,7 +46,6 @@ 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.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -59,31 +57,19 @@ import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.apache.tephra.TransactionContext;
 import org.apache.tephra.TransactionSystemClient;
 import org.apache.tephra.TxConstants;
 import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 
 public class TransactionIT extends ParallelStatsDisabledIT {
     
-
-    @BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-    }
-
     @Test
     public void testReadOwnWrites() throws Exception {
         String transTableName = generateUniqueName();
@@ -154,46 +140,6 @@ public class TransactionIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testDelete() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        String selectSQL = "SELECT * FROM " + fullTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Connection conn1 = DriverManager.getConnection(getUrl()); 
-                Connection conn2 = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
-            conn1.setAutoCommit(false);
-            ResultSet rs = conn1.createStatement().executeQuery(selectSQL);
-            assertFalse(rs.next());
-            
-            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);
-            stmt.execute();
-            conn1.commit();
-            
-            TestUtil.setRowKeyColumns(stmt, 2);
-            stmt.execute();
-            
-            // verify rows can be read even though commit has not been called
-            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 " + fullTableName);
-            assertTrue(rs.next());
-            assertEquals(1, rs.getInt(1));
-            
-            conn1.commit();
-            
-            // verify rows are deleted after commit
-            rs = conn1.createStatement().executeQuery(selectSQL);
-            assertFalse(rs.next());
-        }
-    }
-    
-    @Test
     public void testAutoCommitQuerySingleTable() throws Exception {
         String transTableName = generateUniqueName();
         String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fedbca8b/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 21ea072..14bcd70 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
@@ -30,26 +30,20 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 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.tephra.Transaction.VisibilityLevel;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 public class TxCheckpointIT extends ParallelStatsDisabledIT {
 	
@@ -62,13 +56,14 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
 
 	}
 	
-	@BeforeClass
-    @Shadower(classBeingShadowed = ParallelStatsDisabledIT.class)
-    public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
-        props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    private static Connection getConnection() throws SQLException {
+        return getConnection(PropertiesUtil.deepCopy(TEST_PROPERTIES));
+    }
+    
+    private static Connection getConnection(Properties props) throws SQLException {
+        props.setProperty(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        return conn;
     }
 	
 	@Parameters(name="TxCheckpointIT_localIndex={0},mutable={1}") // name is used by failsafe as file name in reports
@@ -88,7 +83,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection(props);
         conn.setAutoCommit(true);
         conn.createStatement().execute("CREATE SEQUENCE "+seqName);
         conn.createStatement().execute("CREATE TABLE " + fullTableName + "(pk INTEGER PRIMARY KEY, val INTEGER)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
@@ -123,7 +118,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
     
     private void testRollbackOfUncommittedDelete(String indexDDL, String fullTableName) throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
@@ -212,7 +207,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         String indexName = "IDX_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(tableName, tableName);
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-		try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+		try (Connection conn = getConnection()) {
 			conn.setAutoCommit(false);
 			Statement stmt = conn.createStatement();
 
@@ -303,7 +298,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         String fullTableName = SchemaUtil.getTableName(tableName, tableName);
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		ResultSet rs;
-		try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+		try (Connection conn = getConnection()) {
 			conn.setAutoCommit(false);
 			Statement stmt = conn.createStatement();
 			stmt.execute("CREATE TABLE " + fullTableName + "1(ID1 BIGINT NOT NULL PRIMARY KEY, FK1A INTEGER, FK1B INTEGER)"