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 2014/09/23 11:22:27 UTC

[1/2] git commit: PHOENIX-1285 Override default for histogram depth in QueryServicesTestImpl

Repository: phoenix
Updated Branches:
  refs/heads/4.0 757738397 -> 53e2195dd


PHOENIX-1285 Override default for histogram depth in QueryServicesTestImpl


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

Branch: refs/heads/4.0
Commit: 768b9d22304dac02ed1ae0476bfbfdac9a9a9a78
Parents: 7577383
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Sep 23 01:35:46 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Sep 23 01:35:46 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/BaseQueryIT.java   |  1 -
 .../org/apache/phoenix/end2end/StatsCollectorIT.java   |  2 +-
 .../iterate/DefaultParallelIteratorRegionSplitter.java |  2 +-
 .../java/org/apache/phoenix/query/QueryServices.java   |  2 +-
 .../org/apache/phoenix/query/QueryServicesOptions.java | 13 ++++++-------
 .../phoenix/schema/stat/StatisticsCollector.java       |  2 +-
 .../test/java/org/apache/phoenix/query/BaseTest.java   |  2 +-
 .../apache/phoenix/query/QueryServicesTestImpl.java    |  3 +++
 8 files changed, 14 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index 9a66499..eb10512 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -75,7 +75,6 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
         props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
         // Make a small batch size to test multiple calls to reserve sequences
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
-        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, Integer.toString(20));
         // Must update config before starting server
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/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 407c128..a38abea 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
@@ -50,7 +50,7 @@ public class StatsCollectorIT extends BaseHBaseManagedTimeIT {
                 + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
         int histogramDepth = 60;
         Map<String, String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, Integer.toString(histogramDepth));
+        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_ATTRIB, Integer.toString(histogramDepth));
         props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(frequency));
         driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
index 2f568db..227163e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
@@ -64,7 +64,7 @@ public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRe
         this.context = context;
         this.tableRef = table;
         ReadOnlyProps props = context.getConnection().getQueryServices().getProps();
-        this.guidePostsDepth = props.getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY,
+        this.guidePostsDepth = props.getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_ATTRIB,
                 QueryServicesOptions.DEFAULT_HISTOGRAM_BYTE_DEPTH);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 2da52d3..cc2c7a3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -132,7 +132,7 @@ public interface QueryServices extends SQLCloseable {
 
     public static final String USE_REVERSE_SCAN_ATTRIB = "phoenix.query.useReverseScan";
 
-    public static final String HISTOGRAM_BYTE_DEPTH_CONF_KEY = "phoenix.guidepost.width";
+    public static final String HISTOGRAM_BYTE_DEPTH_ATTRIB = "phoenix.guidepost.width";
 
     /**
      * Get executor service used for parallel scans

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/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 a0bc4da..e3fefa8 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
@@ -24,7 +24,7 @@ import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 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.HISTOGRAM_BYTE_DEPTH_CONF_KEY;
+import static org.apache.phoenix.query.QueryServices.HISTOGRAM_BYTE_DEPTH_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.KEEP_ALIVE_MS_ATTRIB;
@@ -144,7 +144,7 @@ public class QueryServicesOptions {
     public static final String DEFAULT_TRACING_STATS_TABLE_NAME = "SYSTEM.TRACING_STATS";
     public static final String DEFAULT_TRACING_FREQ = Tracing.Frequency.NEVER.getKey();
     public static final double DEFAULT_TRACING_PROBABILITY_THRESHOLD = 0.05;
-    public static final long DEFAULT_HISTOGRAM_BYTE_DEPTH = 1024 * 1024;
+    public static final long DEFAULT_HISTOGRAM_BYTE_DEPTH = 1024 * 1024 * 30;
     
     
     public static final boolean DEFAULT_USE_REVERSE_SCAN = true;
@@ -201,7 +201,7 @@ public class QueryServicesOptions {
             .setIfUnset(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES)
             .setIfUnset(SEQUENCE_CACHE_SIZE_ATTRIB, DEFAULT_SEQUENCE_CACHE_SIZE)
             .setIfUnset(SCAN_RESULT_CHUNK_SIZE, DEFAULT_SCAN_RESULT_CHUNK_SIZE)
-            .setIfUnset(HISTOGRAM_BYTE_DEPTH_CONF_KEY, DEFAULT_HISTOGRAM_BYTE_DEPTH);
+            .setIfUnset(HISTOGRAM_BYTE_DEPTH_ATTRIB, DEFAULT_HISTOGRAM_BYTE_DEPTH);
             ;
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
@@ -300,10 +300,6 @@ public class QueryServicesOptions {
         return set(STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs);
     }
     
-    public QueryServicesOptions setHistogramDepthBytes(int depth) {
-        return set(HISTOGRAM_BYTE_DEPTH_CONF_KEY, depth);
-    }
-    
     public QueryServicesOptions setCallQueueRoundRobin(boolean isRoundRobin) {
         return set(CALL_QUEUE_PRODUCER_ATTRIB_NAME, isRoundRobin);
     }
@@ -441,4 +437,7 @@ public class QueryServicesOptions {
         return set(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, walEditCodec);
     }
 
+    public QueryServicesOptions setHistogramByteDepth(long byteDepth) {
+        return set(HISTOGRAM_BYTE_DEPTH_ATTRIB, byteDepth);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
index 17b5825..7552698 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
@@ -235,7 +235,7 @@ public class StatisticsCollector extends BaseRegionObserver implements Coprocess
         // Get the stats table associated with the current table on which the CP is
         // triggered
         stats = StatisticsTable.getStatisticsTableForCoprocessor(env, desc.getName());
-        guidepostDepth = env.getConfiguration().getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY,
+        guidepostDepth = env.getConfiguration().getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_ATTRIB,
                 QueryServicesOptions.DEFAULT_HISTOGRAM_BYTE_DEPTH);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/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 45dbb39..ab193f1 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
@@ -551,7 +551,7 @@ public abstract class BaseTest {
         conf.setInt("hbase.hlog.asyncer.number", 2);
         conf.setInt("hbase.assignment.zkevent.workers", 5);
         conf.setInt("hbase.assignment.threads.max", 5);
-        conf.setInt(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, 20);
+        conf.setInt(QueryServices.HISTOGRAM_BYTE_DEPTH_ATTRIB, 20);
         return conf;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/768b9d22/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 2e3c8f7..a65ca77 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
@@ -51,6 +51,8 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     private static final String DEFAULT_WAL_EDIT_CODEC = IndexedWALEditCodec.class.getName();
     public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE =  1024L*1024L*4L; // 4 Mb
     public static final long DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE =  1024L*1024L*2L; // 2 Mb
+    public static final long DEFAULT_HISTOGRAM_BYTE_DEPTH = 20;
+
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps) {
         this(defaultProps, ReadOnlyProps.EMPTY_PROPS);
@@ -58,6 +60,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     
     private static QueryServicesOptions getDefaultServicesOptions() {
     	return withDefaults()
+    	        .setHistogramByteDepth(DEFAULT_HISTOGRAM_BYTE_DEPTH)
                 .setThreadPoolSize(DEFAULT_THREAD_POOL_SIZE)
                 .setQueueSize(DEFAULT_QUEUE_SIZE)
                 .setMaxMemoryPerc(DEFAULT_MAX_MEMORY_PERC)


[2/2] git commit: PHOENIX-1284 Override config properties for unit tests not making it to server

Posted by ja...@apache.org.
PHOENIX-1284 Override config properties for unit tests not making it to server

Conflicts:
	phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
	phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
	phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java


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

Branch: refs/heads/4.0
Commit: 53e2195ddf7be5b0606f7338a8fb418bd81a93a1
Parents: 768b9d2
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Sep 23 01:23:54 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Sep 23 01:40:35 2014 -0700

----------------------------------------------------------------------
 .../end2end/BaseClientManagedTimeIT.java        | 31 +---------
 .../phoenix/end2end/BaseHBaseManagedTimeIT.java | 31 +---------
 .../BaseParallelIteratorsRegionSplitterIT.java  |  2 +-
 .../org/apache/phoenix/end2end/BaseQueryIT.java |  2 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |  2 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  |  2 +-
 .../phoenix/end2end/InMemoryOrderByIT.java      |  2 +-
 .../org/apache/phoenix/end2end/QueryPlanIT.java |  2 +-
 .../org/apache/phoenix/end2end/SequenceIT.java  |  2 +-
 ...ipRangeParallelIteratorRegionSplitterIT.java |  2 +-
 .../end2end/SkipScanAfterManualSplitIT.java     |  2 +-
 .../phoenix/end2end/SpillableGroupByIT.java     |  2 +-
 .../phoenix/end2end/SpooledOrderByIT.java       |  2 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |  2 +-
 .../phoenix/end2end/index/DropViewIT.java       |  2 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |  3 +-
 .../phoenix/end2end/index/SaltedIndexIT.java    | 17 +++---
 .../java/org/apache/phoenix/query/BaseTest.java | 62 ++++++++++++++++----
 18 files changed, 76 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
index 8b47e11..2f60ea1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
@@ -24,9 +24,6 @@ import java.sql.Date;
 import javax.annotation.concurrent.NotThreadSafe;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
-import org.apache.phoenix.jdbc.PhoenixTestDriver;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.After;
@@ -50,15 +47,6 @@ import org.junit.experimental.categories.Category;
 @NotThreadSafe
 @Category(ClientManagedTimeTest.class)
 public abstract class BaseClientManagedTimeIT extends BaseTest {
-    private static String url;
-    protected static PhoenixTestDriver driver;
-    protected static final Configuration config = HBaseConfiguration.create(); 
-    private static boolean clusterInitialized = false;
-    
-    protected final static String getUrl() {
-        return checkClusterInitialized();
-    }
-    
     protected static Configuration getTestClusterConfig() {
         // don't want callers to modify config.
         return new Configuration(config);
@@ -72,24 +60,7 @@ public abstract class BaseClientManagedTimeIT extends BaseTest {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        setUpTestDriver(getUrl(), ReadOnlyProps.EMPTY_PROPS);
-    }
-    
-    protected static void setUpTestDriver(String url, ReadOnlyProps props) throws Exception {
-        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
-            checkClusterInitialized();
-            if (driver == null) {
-                driver = initAndRegisterDriver(url, props);
-            }
-        }
-    }
-
-    private static String checkClusterInitialized() {
-        if (!clusterInitialized) {
-            url = setUpTestCluster(config);
-            clusterInitialized = true;
-        }
-        return url;
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
     }
     
     @AfterClass

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
index 047afb8..1d0f004 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseHBaseManagedTimeIT.java
@@ -22,10 +22,7 @@ import static org.junit.Assert.assertTrue;
 import javax.annotation.concurrent.NotThreadSafe;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
-import org.apache.phoenix.jdbc.PhoenixTestDriver;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.After;
@@ -49,15 +46,6 @@ import org.junit.experimental.categories.Category;
 @NotThreadSafe
 @Category(HBaseManagedTimeTest.class)
 public abstract class BaseHBaseManagedTimeIT extends BaseTest {
-    private static String url;
-    protected static PhoenixTestDriver driver;
-    private static final Configuration config = HBaseConfiguration.create(); 
-    private static boolean clusterInitialized = false;
-    
-    protected final static String getUrl() {
-        return checkClusterInitialized();
-    }
-    
     protected static Configuration getTestClusterConfig() {
         // don't want callers to modify config.
         return new Configuration(config);
@@ -70,24 +58,7 @@ public abstract class BaseHBaseManagedTimeIT extends BaseTest {
     
     @BeforeClass
     public static void doSetup() throws Exception {
-        setUpTestDriver(getUrl(), ReadOnlyProps.EMPTY_PROPS);
-    }
-    
-    protected static void setUpTestDriver(String url, ReadOnlyProps props) throws Exception {
-        if (PhoenixEmbeddedDriver.isTestUrl(url)) {
-            checkClusterInitialized();
-            if (driver == null) {
-                driver = initAndRegisterDriver(url, props);
-            }
-        }
-    }
-
-    private static String checkClusterInitialized() {
-        if (!clusterInitialized) {
-            url = setUpTestCluster(config);
-            clusterInitialized = true;
-        }
-        return url;
+        setUpTestDriver(ReadOnlyProps.EMPTY_PROPS);
     }
     
     @AfterClass

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
index cfaa8ee..514b36e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
@@ -59,7 +59,7 @@ public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTime
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     protected void initTableValues(long ts) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index eb10512..7a3e86e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -76,7 +76,7 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
         // Make a small batch size to test multiple calls to reserve sequences
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     protected long ts;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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 1cc9207..c255d00 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
@@ -43,7 +43,7 @@ public class BaseViewIT extends BaseHBaseManagedTimeIT {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     protected void testUpdatableViewWithIndex(Integer saltBuckets, boolean localIndex) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index 1d2ca4c..dcd96d3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -85,7 +85,7 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
         // 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(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Before

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
index 5e04f8e..48a0581 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InMemoryOrderByIT.java
@@ -38,6 +38,6 @@ public class InMemoryOrderByIT extends OrderByIT {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(1024*1024));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
index 60409b2..320ba72 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryPlanIT.java
@@ -48,7 +48,7 @@ public class QueryPlanIT extends BaseHBaseManagedTimeIT {
         // Override date format so we don't have a bunch of zeros
         props.put(QueryServices.DATE_FORMAT_ATTRIB, "yyyy-MM-dd");
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 3e0301c..f8673ce 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -67,7 +67,7 @@ public class SequenceIT extends BaseClientManagedTimeIT {
         // Make a small batch size to test multiple calls to reserve sequences
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @After

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
index d32441b..31e3a3b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
@@ -319,7 +319,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     private static List<KeyRange> getSplits(TableRef tableRef, final Scan scan, final List<HRegionLocation> regions,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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 f7a2448..9b3a3cd 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
@@ -73,7 +73,7 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
         props.put(QueryServices.ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, Boolean.toString(false));
         props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1000));
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     private static void initTable() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
index 3a0de85..9c7e4b9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpillableGroupByIT.java
@@ -66,7 +66,7 @@ public class SpillableGroupByIT extends BaseHBaseManagedTimeIT {
                 Integer.toString(1));
         // Large enough to not run out of memory, but small enough to spill
         props.put(QueryServices.MAX_MEMORY_SIZE_ATTRIB, Integer.toString(40000));
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     private long createTable() throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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
index 859a66a..2533a29 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SpooledOrderByIT.java
@@ -35,7 +35,7 @@ public class SpooledOrderByIT extends OrderByIT {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         props.put(QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB, Integer.toString(100));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index a64371d..642ba62 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -69,7 +69,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
       props.put(QueryServices.THREAD_POOL_SIZE_ATTRIB, Integer.toString(64));
 
       // Must update config before starting server
-      setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+      setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
   }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
index 2cb0572..87c4770 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropViewIT.java
@@ -54,7 +54,7 @@ public class DropViewIT extends BaseMutableIndexIT {
         // Drop the HBase table metadata for this test
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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 cdd44f1..fe24c35 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
@@ -61,8 +61,7 @@ public class MutableIndexIT extends BaseMutableIndexIT {
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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 8eae5f9..d5e9d42 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
@@ -57,8 +57,10 @@ public class SaltedIndexIT extends BaseIndexIT {
         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));
+        // Don't put guideposts in
+        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_ATTRIB, Long.toString(10000000));
         // Must update config before starting server
-        setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
     private static void makeImmutableAndDeleteData() throws Exception {
@@ -121,8 +123,6 @@ public class SaltedIndexIT extends BaseIndexIT {
         stmt.setString(2, "y");
         stmt.execute();
         conn.commit();
-        stmt = conn.prepareStatement("ANALYZE "+DATA_TABLE_FULL_NAME);
-        stmt.execute();
 
         query = "SELECT * FROM " + INDEX_TABLE_FULL_NAME;
         rs = conn.createStatement().executeQuery(query);
@@ -177,13 +177,14 @@ public class SaltedIndexIT extends BaseIndexIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-                "CLIENT PARALLEL 3-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" +
+                "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" +
                 "    SERVER SORTED BY [V]\n" + 
                 "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 2-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" + 
+                    "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" + 
                     "    SERVER SORTED BY [V]\n" + 
                     "CLIENT MERGE SORT";
-        assertEquals(expectedPlan,QueryUtil.getExplainPlan(rs));
+        String explainPlan2 = QueryUtil.getExplainPlan(rs);
+        assertEquals(expectedPlan,explainPlan2);
         
         // Will use data table now, since there's a LIMIT clause and
         // we're able to optimize out the ORDER BY, unless the data
@@ -199,11 +200,11 @@ public class SaltedIndexIT extends BaseIndexIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-             "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
+             "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
              "    SERVER FILTER BY V >= 'x'\n" + 
              "    SERVER 2 ROW LIMIT\n" + 
              "CLIENT 2 ROW LIMIT" :
-                 "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
+                 "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
                  "    SERVER FILTER BY V >= 'x'\n" + 
                  "    SERVER 2 ROW LIMIT\n" + 
                  "CLIENT MERGE SORT\n" + 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/53e2195d/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 ab193f1..100b4ad 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
@@ -107,6 +107,7 @@ import java.util.logging.Logger;
 import javax.annotation.Nonnull;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -442,19 +443,47 @@ public abstract class BaseTest {
         return conf.get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
     }
     
+    private static String url;
+    protected static PhoenixTestDriver driver;
+    private static boolean clusterInitialized = false;
+    protected static final Configuration config = HBaseConfiguration.create(); 
+    
+    protected static String getUrl() {
+        if (!clusterInitialized) {
+            throw new IllegalStateException("Cluster must be initialized before attempting to get the URL");
+        }
+        return url;
+    }
+    
+    protected static String checkClusterInitialized(ReadOnlyProps overrideProps) {
+        if (!clusterInitialized) {
+            url = setUpTestCluster(config, overrideProps);
+            clusterInitialized = true;
+        }
+        return url;
+    }
+    
     /**
      * Set up the test hbase cluster. 
+     * @param props TODO
      * @return url to be used by clients to connect to the cluster.
      */
-    protected static String setUpTestCluster(@Nonnull Configuration conf) {
+    protected static String setUpTestCluster(@Nonnull Configuration conf, ReadOnlyProps overrideProps) {
         boolean isDistributedCluster = isDistributedClusterModeEnabled(conf);
         if (!isDistributedCluster) {
-            return initMiniCluster(conf);
+            return initMiniCluster(conf, overrideProps);
        } else {
-            return initClusterDistributedMode(conf);
+            return initClusterDistributedMode(conf, overrideProps);
         }
     }
     
+    protected static void setUpTestDriver(ReadOnlyProps props) throws Exception {
+        String url = checkClusterInitialized(props);
+        if (driver == null) {
+            driver = initAndRegisterDriver(url, props);
+        }
+    }
+
     private static boolean isDistributedClusterModeEnabled(Configuration conf) {
         boolean isDistributedCluster = false;
         //check if the distributed mode was specified as a system property.
@@ -468,10 +497,11 @@ public abstract class BaseTest {
     
     /**
      * Initialize the mini cluster using phoenix-test specific configuration.
+     * @param overrideProps TODO
      * @return url to be used by clients to connect to the mini cluster.
      */
-    private static String initMiniCluster(Configuration conf) {
-        setUpConfigForMiniCluster(conf);
+    private static String initMiniCluster(Configuration conf, ReadOnlyProps overrideProps) {
+        setUpConfigForMiniCluster(conf, overrideProps);
         utility = new HBaseTestingUtility(conf);
         try {
             utility.startMiniCluster();
@@ -499,10 +529,11 @@ public abstract class BaseTest {
     
     /**
      * Initialize the cluster in distributed mode
+     * @param overrideProps TODO
      * @return url to be used by clients to connect to the mini cluster.
      */
-    private static String initClusterDistributedMode(Configuration conf) {
-        setTestConfigForDistribuedCluster(conf);
+    private static String initClusterDistributedMode(Configuration conf, ReadOnlyProps overrideProps) {
+        setTestConfigForDistribuedCluster(conf, overrideProps);
         try {
             IntegrationTestingUtility util =  new IntegrationTestingUtility(conf);
             utility = util;
@@ -513,11 +544,11 @@ public abstract class BaseTest {
         return JDBC_PROTOCOL + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
     }
 
-    private static void setTestConfigForDistribuedCluster(Configuration conf) {
-        setDefaultTestConfig(conf);
+    private static void setTestConfigForDistribuedCluster(Configuration conf, ReadOnlyProps overrideProps) {
+        setDefaultTestConfig(conf, overrideProps);
     }
     
-    private static void setDefaultTestConfig(Configuration conf) {
+    private static void setDefaultTestConfig(Configuration conf, ReadOnlyProps overrideProps) {
         ConfigUtil.setReplicationConfigIfAbsent(conf);
         QueryServices services = new PhoenixTestDriver().getQueryServices();
         for (Entry<String,String> entry : services.getProps()) {
@@ -525,11 +556,20 @@ public abstract class BaseTest {
         }
         //no point doing sanity checks when running tests.
         conf.setBoolean("hbase.table.sanity.checks", false);
+        
+        // override any defaults based on overrideProps
+        for (Entry<String,String> entry : overrideProps) {
+            conf.set(entry.getKey(), entry.getValue());
+        }
     }
     
     public static Configuration setUpConfigForMiniCluster(Configuration conf) {
+        return setUpConfigForMiniCluster(conf, ReadOnlyProps.EMPTY_PROPS);
+    }
+    
+    public static Configuration setUpConfigForMiniCluster(Configuration conf, ReadOnlyProps overrideProps) {
         assertNotNull(conf);
-        setDefaultTestConfig(conf);
+        setDefaultTestConfig(conf, overrideProps);
         /*
          * The default configuration of mini cluster ends up spawning a lot of threads
          * that are not really needed by phoenix for test purposes. Limiting these threads