You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/12/02 23:10:34 UTC

phoenix git commit: PHOENIX-2458 Refactoring tests that start the txn manager to use BaseOwnClusterIT

Repository: phoenix
Updated Branches:
  refs/heads/master 6aa4cc61a -> 0701e06d4


PHOENIX-2458 Refactoring tests that start the txn manager to use BaseOwnClusterIT


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

Branch: refs/heads/master
Commit: 0701e06d48a90c372a073d6aac2c3259e5500bb6
Parents: 6aa4cc6
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Tue Dec 1 13:58:49 2015 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Dec 2 14:10:30 2015 -0800

----------------------------------------------------------------------
 .../end2end/index/MutableIndexFailureIT.java    | 92 +++++---------------
 .../end2end/index/txn/TxWriteFailureIT.java     | 49 +++--------
 .../apache/phoenix/execute/PartialCommitIT.java | 41 +++------
 .../apache/phoenix/rpc/PhoenixClientRpcIT.java  |  1 +
 .../java/org/apache/phoenix/query/BaseTest.java |  7 ++
 5 files changed, 55 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0701e06d/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 801d6c3..9032ee9 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
@@ -17,11 +17,6 @@
  */
 package org.apache.phoenix.end2end.index;
 
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
-import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -34,6 +29,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -43,11 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseCluster;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -55,14 +47,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
-import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
-import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
-import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTableType;
@@ -73,9 +60,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -95,11 +80,7 @@ import com.google.common.collect.Maps;
 
 @Category(NeedsOwnMiniClusterTest.class)
 @RunWith(Parameterized.class)
-@Ignore // see PHOENIX-2458 
-public class MutableIndexFailureIT extends BaseTest {
-    private static final int NUM_SLAVES = 4;
-    private static PhoenixTestDriver driver;
-    private static HBaseTestingUtility util;
+public class MutableIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
     private Timer scheduleTimer;
 
     private String tableName;
@@ -113,63 +94,34 @@ public class MutableIndexFailureIT extends BaseTest {
     public MutableIndexFailureIT(boolean transactional) {
         this.transactional = transactional;
         this.tableDDLOptions = transactional ? " TRANSACTIONAL=true " : "";
-        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME;
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + (transactional ? "_TXN" : "");
         this.indexName = "IDX";
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
     }
     
-    @Before
-    public void doSetup() throws Exception {
-        Configuration conf = HBaseConfiguration.create();
-        setUpConfigForMiniCluster(conf);
-        conf.setInt("hbase.client.retries.number", 2);
-        conf.setInt("hbase.client.pause", 5000);
-        conf.setInt("hbase.balancer.period", Integer.MAX_VALUE);
-        conf.setLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB, 0);
-        conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, IndexMasterObserver.class.getName());
-        conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, IndexLoadBalancer.class,
-            LoadBalancer.class);
-        util = new HBaseTestingUtility(conf);
-        util.startMiniCluster(NUM_SLAVES);
-        String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
-        url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
-                + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
-        clusterInitialized = true;
-        setupTxManager();
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(10);
+        serverProps.put("hbase.client.retries.number", "2");
+        serverProps.put("hbase.client.pause", "5000");
+        serverProps.put("hbase.balancer.period", String.valueOf(Integer.MAX_VALUE));
+        serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB, "0");
+        Map<String, String> clientProps = Collections.singletonMap(QueryServices.TRANSACTIONS_ENABLED, "true");
+        NUM_SLAVES_BASE = 4;
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
     
     @Parameters(name = "transactional = {0}")
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] { { false }, { true } });
     }
-    
-    @After
-    public void tearDown() throws Exception {
-        try {
-            destroyDriver(driver);
-        } finally {
-            try {
-                if(scheduleTimer != null){
-                    scheduleTimer.cancel();
-                    scheduleTimer = null;
-                }
-            } finally {
-                util.shutdownMiniCluster();
-            }
-        }
-    }
 
-    @Ignore("See PHOENIX-2331")
     @Test(timeout=300000)
     public void testWriteFailureDisablesLocalIndex() throws Exception {
         helpTestWriteFailureDisablesIndex(true);
     }
  
-    @Ignore("See PHOENIX-2332")
     @Test(timeout=300000)
     public void testWriteFailureDisablesIndex() throws Exception {
         helpTestWriteFailureDisablesIndex(false);
@@ -219,7 +171,7 @@ public class MutableIndexFailureIT extends BaseTest {
             TableName indexTable =
                     TableName.valueOf(localIndex ? MetaDataUtil
                             .getLocalIndexTableName(fullTableName) : fullIndexName);
-            HBaseAdmin admin = this.util.getHBaseAdmin();
+            HBaseAdmin admin = this.getUtility().getHBaseAdmin();
             HTableDescriptor indexTableDesc = admin.getTableDescriptor(indexTable);
             try{
               admin.disableTable(indexTable);
@@ -382,9 +334,9 @@ public class MutableIndexFailureIT extends BaseTest {
                 // find a RS which doesn't has CATALOG table
                 TableName catalogTable = TableName.valueOf("SYSTEM.CATALOG");
                 TableName indexTable = TableName.valueOf(fullIndexName);
-                final HBaseCluster cluster = this.util.getHBaseCluster();
+                final HBaseCluster cluster = this.getUtility().getHBaseCluster();
                 Collection<ServerName> rss = cluster.getClusterStatus().getServers();
-                HBaseAdmin admin = this.util.getHBaseAdmin();
+                HBaseAdmin admin = this.getUtility().getHBaseAdmin();
                 List<HRegionInfo> regions = admin.getTableRegions(catalogTable);
                 ServerName catalogRS = cluster.getServerHoldingRegion(regions.get(0).getTable(),
                         regions.get(0).getRegionName());
@@ -404,7 +356,7 @@ public class MutableIndexFailureIT extends BaseTest {
                 final HRegionInfo indexRegion = regions.get(0);
                 final ServerName dstRS = rsToBeKilled;
                 admin.move(indexRegion.getEncodedNameAsBytes(), Bytes.toBytes(rsToBeKilled.getServerName()));
-                this.util.waitFor(30000, 200, new Waiter.Predicate<Exception>() {
+                this.getUtility().waitFor(30000, 200, new Waiter.Predicate<Exception>() {
                     @Override
                     public boolean evaluate() throws Exception {
                       ServerName sn = cluster.getServerHoldingRegion(indexRegion.getTable(),
@@ -420,10 +372,10 @@ public class MutableIndexFailureIT extends BaseTest {
                 Thread.sleep(100);
                 
                 // kill RS hosting index table
-                this.util.getHBaseCluster().killRegionServer(rsToBeKilled);
+                this.getUtility().getHBaseCluster().killRegionServer(rsToBeKilled);
                 
                 // wait for index table completes recovery
-                this.util.waitUntilAllRegionsAssigned(indexTable);
+                this.getUtility().waitUntilAllRegionsAssigned(indexTable);
                 
                 // Verify the metadata for index is correct.       
                 do {
@@ -436,8 +388,6 @@ public class MutableIndexFailureIT extends BaseTest {
                   }
                 } while(true);
                 this.scheduleTimer.cancel();
-                
-                assertEquals(cluster.getClusterStatus().getDeadServers(), 1);
             }
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0701e06d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
index 04bf1dc..8e36bf0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/txn/TxWriteFailureIT.java
@@ -17,11 +17,6 @@
  */
 package org.apache.phoenix.end2end.index.txn;
 
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
-import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -37,21 +32,17 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseOwnClusterHBaseManagedTimeIT;
 import org.apache.phoenix.hbase.index.Indexer;
-import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
@@ -66,11 +57,8 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class TxWriteFailureIT extends BaseTest {
+public class TxWriteFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
 	
-    private static PhoenixTestDriver driver;
-    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-
     private static final String SCHEMA_NAME = "S";
     private static final String DATA_TABLE_NAME = "T";
     private static final String INDEX_TABLE_NAME = "I";
@@ -85,29 +73,18 @@ public class TxWriteFailureIT extends BaseTest {
 		this.localIndex = localIndex;
 		this.mutable = mutable;
 	}
-
+	
 	@BeforeClass
-	public static void setupCluster() throws Exception {
-		Configuration conf = TEST_UTIL.getConfiguration();
-		setUpConfigForMiniCluster(conf);
-		conf.setClass("hbase.coprocessor.region.classes", FailingRegionObserver.class, RegionObserver.class);
-		conf.setBoolean("hbase.coprocessor.abortonerror", false);
-		conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-		TEST_UTIL.startMiniCluster();
-		String clientPort = TEST_UTIL.getConfiguration().get(
-				QueryServices.ZOOKEEPER_PORT_ATTRIB);
-		url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST
-				+ JDBC_PROTOCOL_SEPARATOR + clientPort
-				+ JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
-
-		Map<String, String> props = Maps.newHashMapWithExpectedSize(2);
-		// Must update config before starting server
-		props.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, Boolean.toString(true));
-		props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-		driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
-		clusterInitialized = true;
-		setupTxManager();
-	}
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(3);
+        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put("hbase.coprocessor.abortonerror", "false");
+        serverProps.put(Indexer.CHECK_VERSION_CONF_KEY, "false");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(10);
+        clientProps.put(QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB, "true");
+        clientProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
 	
 	@Parameters(name="localIndex = {0} , mutable = {1}")
     public static Collection<Boolean[]> data() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0701e06d/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 4b06834..61aae62 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -22,29 +22,23 @@ package org.apache.phoenix.execute;
 import static com.google.common.collect.Lists.newArrayList;
 import static com.google.common.collect.Sets.newHashSet;
 import static java.util.Collections.singletonList;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
-import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
-import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.sql.Connection;
-import java.sql.Driver;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -54,14 +48,13 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.end2end.BaseOwnClusterIT;
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -76,7 +69,9 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public class PartialCommitIT extends BaseTest {
+// Needs to extend BaseOwnClusterIT (and not BaseOwnClusterHBaseManagedTimeIT since table data 
+// is created and populated once in the constructor 
+public class PartialCommitIT extends BaseOwnClusterIT {
     
 	private final String A_SUCESS_TABLE;
 	private final String B_FAILURE_TABLE;
@@ -88,26 +83,16 @@ public class PartialCommitIT extends BaseTest {
     private static final byte[] ROW_TO_FAIL_UPSERT_BYTES = Bytes.toBytes("fail me upsert");
     private static final byte[] ROW_TO_FAIL_DELETE_BYTES = Bytes.toBytes("fail me delete");
     private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-    private static Driver driver;
     
     @BeforeClass
-    public static void setupCluster() throws Exception {
-      Configuration conf = TEST_UTIL.getConfiguration();
-      setUpConfigForMiniCluster(conf);
-      conf.setClass("hbase.coprocessor.region.classes", FailingRegionObserver.class, RegionObserver.class);
-      conf.setBoolean("hbase.coprocessor.abortonerror", false);
-      conf.setBoolean(Indexer.CHECK_VERSION_CONF_KEY, false);
-      TEST_UTIL.startMiniCluster();
-      String clientPort = TEST_UTIL.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
-      url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
-              + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
-
-      Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
-      // Must update config before starting server
-      driver = initAndRegisterDriver(url, ReadOnlyProps.EMPTY_PROPS);
-      clusterInitialized = true;
-      setupTxManager();
-      createTablesWithABitOfData();
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(10);
+        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put("hbase.coprocessor.abortonerror", "false");
+        serverProps.put(Indexer.CHECK_VERSION_CONF_KEY, "false");
+        Map<String, String> clientProps = Collections.singletonMap(QueryServices.TRANSACTIONS_ENABLED, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+        createTablesWithABitOfData();
     }
     
     @Parameters(name="transactional = {0}")

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0701e06d/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
index 0c61b55..2ccf490 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/PhoenixClientRpcIT.java
@@ -50,6 +50,7 @@ public class PhoenixClientRpcIT extends BaseOwnClusterHBaseManagedTimeIT {
     @AfterClass
     public static void cleanUpAfterTestSuite() throws Exception {
         TestPhoenixIndexRpcSchedulerFactory.reset();
+        tearDownMiniCluster();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0701e06d/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 fa5df31..0c1c71b 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
@@ -121,16 +121,20 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ServerRpcControllerFactory;
+import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.regionserver.LocalIndexMerger;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
+import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
@@ -698,6 +702,9 @@ public abstract class BaseTest {
         conf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 5);
         conf.setInt("hbase.regionserver.metahandler.count", 2);
         conf.setInt(HConstants.MASTER_HANDLER_COUNT, 2);
+        conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, IndexMasterObserver.class.getName());
+        conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, IndexLoadBalancer.class,
+            LoadBalancer.class);
         conf.setClass("hbase.coprocessor.regionserver.classes", LocalIndexMerger.class,
             RegionServerObserver.class);
         conf.setInt("dfs.namenode.handler.count", 2);