You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2018/07/13 20:41:10 UTC

[1/5] helix git commit: Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.

Repository: helix
Updated Branches:
  refs/heads/master fe970550b -> c0d5792b7


http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
index 3d039aa..43d80fe 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureDependence.java
@@ -62,7 +62,7 @@ public class TestJobFailureDependence extends TaskTestBase {
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
 
     // all jobs after failed job should fail too.
     for (int i = 2; i < _numDbs; i++) {
@@ -90,7 +90,7 @@ public class TestJobFailureDependence extends TaskTestBase {
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(2));
     _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);
@@ -115,7 +115,7 @@ public class TestJobFailureDependence extends TaskTestBase {
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(2));
     String namedSpaceJob2 = String.format("%s_%s", queueName, currentJobNames.get(2));
     _driver.pollForJobState(queueName, namedSpaceJob2, TaskState.FAILED);
 
@@ -145,7 +145,7 @@ public class TestJobFailureDependence extends TaskTestBase {
     }
 
     _driver.start(queueBuilder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
 
     String namedSpaceJob1 = String.format("%s_%s", queueName, currentJobNames.get(1));
     _driver.pollForJobState(queueName, namedSpaceJob1, TaskState.FAILED);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureHighThreshold.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureHighThreshold.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureHighThreshold.java
index 9e67d46..062d277 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureHighThreshold.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureHighThreshold.java
@@ -43,7 +43,6 @@ import org.testng.annotations.Test;
 
 public class TestJobFailureHighThreshold extends TaskSynchronizedTestBase {
 
-  private ClusterControllerManager _controller;
   private static final String DB_NAME = WorkflowGenerator.DEFAULT_TGT_DB;
 
   @BeforeClass
@@ -54,13 +53,7 @@ public class TestJobFailureHighThreshold extends TaskSynchronizedTestBase {
     _numParitions = 5;
     _numReplicas = 1;
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipants();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureTaskNotStarted.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureTaskNotStarted.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureTaskNotStarted.java
index d1f8250..ace98e0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureTaskNotStarted.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailureTaskNotStarted.java
@@ -56,8 +56,6 @@ import org.testng.annotations.Test;
 
 
 public class TestJobFailureTaskNotStarted extends TaskSynchronizedTestBase {
-
-  private ClusterControllerManager _controller;
   private static final String DB_NAME = WorkflowGenerator.DEFAULT_TGT_DB;
   private static final String UNBALANCED_DB_NAME = "UnbalancedDB";
   private MockParticipantManager _blockedParticipant;
@@ -71,13 +69,7 @@ public class TestJobFailureTaskNotStarted extends TaskSynchronizedTestBase {
     _numParitions = 2;
     _numReplicas = 1;
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipantsWithStuckTaskStateModelFactory();
@@ -101,7 +93,7 @@ public class TestJobFailureTaskNotStarted extends TaskSynchronizedTestBase {
       }
     });
 
-    List<String> instances = _setupTool.getClusterManagementTool().getInstancesInCluster(CLUSTER_NAME);
+    List<String> instances = _gSetupTool.getClusterManagementTool().getInstancesInCluster(CLUSTER_NAME);
 
     _participants[0] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instances.get(0));
     StateMachineEngine stateMachine = _participants[0].getStateMachineEngine();
@@ -180,12 +172,12 @@ public class TestJobFailureTaskNotStarted extends TaskSynchronizedTestBase {
 
   private void setupUnbalancedDB() throws InterruptedException {
     // Start with Full-Auto mode to create the partitions, Semi-Auto won't create partitions.
-    _setupTool.addResourceToCluster(CLUSTER_NAME, UNBALANCED_DB_NAME, 50, MASTER_SLAVE_STATE_MODEL,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, UNBALANCED_DB_NAME, 50, MASTER_SLAVE_STATE_MODEL,
         IdealState.RebalanceMode.FULL_AUTO.toString());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, UNBALANCED_DB_NAME, 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, UNBALANCED_DB_NAME, 1);
 
     // Set preference list to put all partitions to one instance.
-    IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME,
+    IdealState idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME,
         UNBALANCED_DB_NAME);
     Set<String> partitions = idealState.getPartitionSet();
     for (String partition : partitions) {
@@ -193,7 +185,7 @@ public class TestJobFailureTaskNotStarted extends TaskSynchronizedTestBase {
     }
     idealState.setRebalanceMode(IdealState.RebalanceMode.SEMI_AUTO);
 
-    _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, UNBALANCED_DB_NAME, idealState);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, UNBALANCED_DB_NAME, idealState);
 
     HelixClusterVerifier clusterVerifier =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkClient(_gZkClient).build();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeout.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeout.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeout.java
index 8482f69..dffb613 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeout.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeout.java
@@ -41,8 +41,6 @@ import com.google.common.collect.Sets;
 
 public final class TestJobTimeout extends TaskSynchronizedTestBase {
 
-  private ClusterControllerManager _controller;
-
   @BeforeClass
   public void beforeClass() throws Exception {
     _numNodes = 2;
@@ -50,13 +48,8 @@ public final class TestJobTimeout extends TaskSynchronizedTestBase {
     _numReplicas = 1; // only Master, no Slave
     _numDbs = 1;
     _participants =  new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipants();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeoutTaskNotStarted.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeoutTaskNotStarted.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeoutTaskNotStarted.java
index 6129946..32d5d52 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeoutTaskNotStarted.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobTimeoutTaskNotStarted.java
@@ -52,8 +52,6 @@ import com.google.common.collect.Sets;
 
 public class TestJobTimeoutTaskNotStarted extends TaskSynchronizedTestBase {
 
-  private ClusterControllerManager _controller;
-
   @BeforeClass
   public void beforeClass() throws Exception {
     _numDbs = 1;
@@ -61,13 +59,7 @@ public class TestJobTimeoutTaskNotStarted extends TaskSynchronizedTestBase {
     _numParitions = 50;
     _numReplicas = 1;
     _participants =  new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipantsWithStuckTaskStateModelFactory();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestRebalanceRunningTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRebalanceRunningTask.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRebalanceRunningTask.java
index f518d5c..f9fa2bc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRebalanceRunningTask.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRebalanceRunningTask.java
@@ -42,8 +42,6 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public final class TestRebalanceRunningTask extends TaskSynchronizedTestBase {
-
-  private ClusterControllerManager _controller;
   private final String JOB = "test_job";
   private String WORKFLOW;
   private final String DATABASE = WorkflowGenerator.DEFAULT_TGT_DB;
@@ -57,13 +55,7 @@ public final class TestRebalanceRunningTask extends TaskSynchronizedTestBase {
     _numReplicas = 1; // only Master, no Slave
     _numDbs = 1;
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
index b50bf12..23da9dc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestRunJobsWithMissingTarget.java
@@ -65,7 +65,7 @@ public class TestRunJobsWithMissingTarget extends TaskTestBase {
       currentJobNames.add(jobName);
     }
 
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(1));
     _driver.start(builder.build());
 
     String namedSpaceJob = String.format("%s_%s", workflowName, currentJobNames.get(1));
@@ -129,7 +129,7 @@ public class TestRunJobsWithMissingTarget extends TaskTestBase {
     }
 
     _driver.start(builder.build());
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(0));
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, _testDbs.get(0));
 
     String namedSpaceJob1 = String.format("%s_%s", workflowName, currentJobNames.get(0));
     _driver.pollForJobState(workflowName, namedSpaceJob1, TaskState.FAILED);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
index c78b43c..6045812 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskAssignment.java
@@ -45,7 +45,7 @@ public class TestTaskAssignment extends TaskTestBase {
 
   @Test
   public void testTaskAssignment() throws InterruptedException {
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + 0), false);
     String jobResource = TestHelper.getTestMethodName();
     JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
index 225a75c..62aca00 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerParallel.java
@@ -121,9 +121,5 @@ public class TestTaskRebalancerParallel extends TaskTestBase {
     _driver.resume(queueName);
     Thread.sleep(2000);
     Assert.assertTrue(TaskTestUtil.pollForWorkflowParallelState(_driver, queueName));
-
-    for (int i = 1; i < _numNodes; i++) {
-      _participants[i].syncStart();
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThreadLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThreadLeak.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThreadLeak.java
index 95ac55b..420c534 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThreadLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThreadLeak.java
@@ -50,9 +50,9 @@ public class TestTaskThreadLeak extends TaskTestBase {
     String lastJob = null;
     for (int i = 0; i < 5; i++) {
       String db = TestHelper.getTestMethodName() + "_" + i;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, 20, MASTER_SLAVE_STATE_MODEL,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, 20, MASTER_SLAVE_STATE_MODEL,
           IdealState.RebalanceMode.FULL_AUTO.name());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, 1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, 1);
       JobConfig.Builder jobBuilder =
           new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND).setTargetResource(db)
               .setNumConcurrentTasksPerInstance(100);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThrottling.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThrottling.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThrottling.java
index bec9505..d25397a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThrottling.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskThrottling.java
@@ -78,7 +78,7 @@ public class TestTaskThrottling extends TaskTestBase {
     Map<String, String> properties = new HashMap<String, String>();
     properties.put(ClusterConfig.ClusterConfigProperty.MAX_CONCURRENT_TASK_PER_INSTANCE.name(),
         new Integer(perNodeTaskLimitation).toString());
-    _setupTool.getClusterManagementTool().setConfig(scope, properties);
+    _gSetupTool.getClusterManagementTool().setConfig(scope, properties);
 
     String jobName2 = "Job2";
     flow = WorkflowGenerator.generateSingleJobWorkflowBuilder(jobName2, jobConfig).build();
@@ -173,10 +173,10 @@ public class TestTaskThrottling extends TaskTestBase {
 
   private void setParticipantsCapacity(int perNodeTaskLimitation) {
     for (int i = 0; i < _numNodes; i++) {
-      InstanceConfig instanceConfig = _setupTool.getClusterManagementTool()
+      InstanceConfig instanceConfig = _gSetupTool.getClusterManagementTool()
           .getInstanceConfig(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + i));
       instanceConfig.setMaxConcurrentTask(perNodeTaskLimitation);
-      _setupTool.getClusterManagementTool()
+      _gSetupTool.getClusterManagementTool()
           .setInstanceConfig(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + i), instanceConfig);
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
index a95dfca..6bea3e7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskWithInstanceDisabled.java
@@ -41,7 +41,7 @@ public class TestTaskWithInstanceDisabled extends TaskTestBase {
   }
   @Test
   public void testTaskWithInstanceDisabled() throws InterruptedException {
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_" + (_startPort + 0), false);
     String jobResource = TestHelper.getTestMethodName();
     JobConfig.Builder jobBuilder = new JobConfig.Builder().setCommand(MockTask.TASK_COMMAND)

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
index 0f3ab51..b656cdc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestUserContentStore.java
@@ -55,17 +55,12 @@ public class TestUserContentStore extends TaskTestBase {
   @BeforeClass
   public void beforeClass() throws Exception {
     _participants = new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
 
     // Setup cluster and instances
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     for (int i = 0; i < _numNodes; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // start dummy participants

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java
index 2c84d22..d960315 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestHandleNewSession.java
@@ -23,12 +23,12 @@ import java.util.Date;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestHandleNewSession extends ZkIntegrationTestBase {
+public class TestHandleNewSession extends ZkTestBase {
   @Test
   public void testHandleNewSession() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);
@@ -72,6 +72,7 @@ public class TestHandleNewSession extends ZkIntegrationTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("Disconnecting ...");
     participant.syncStop();
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java
index 021fb39..fb7b973 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBaseDataAccessor.java
@@ -36,9 +36,26 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor.AccessResult;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor.RetCode;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
 
 public class TestZkBaseDataAccessor extends ZkUnitTestBase {
+  String _rootPath = TestHelper.getTestClassName();
+
+
+  @AfterMethod
+  public void afterMethod() {
+    String path = "/" + _rootPath;
+    if (_gZkClient.exists(path)) {
+      _gZkClient.deleteRecursively(path);
+    }
+  }
+
+  @AfterClass
+  public void after() {
+    int a =1;
+  }
 
   @Test
   public void testSyncSet() {
@@ -48,7 +65,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     BaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -59,7 +76,6 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     Assert.assertEquals(getRecord.getId(), "msg_0");
 
     System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test
@@ -70,7 +86,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     BaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -91,7 +107,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     Assert.assertEquals(getRecord.getId(), "msg_0");
 
     // set ephemeral
-    path = String.format("/%s/%s", testName, "msg_1");
+    path = String.format("/%s/%s", _rootPath, "msg_1");
     record = new ZNRecord("msg_1");
     success = accessor.set(path, record, 0, AccessOption.EPHEMERAL);
     Assert.assertFalse(success);
@@ -128,19 +144,17 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s/%s", testName, "msg_0", "submsg_0");
+    String path = String.format("/%s/%s/%s", _rootPath, "msg_0", "submsg_0");
     ZNRecord record = new ZNRecord("submsg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
     AccessResult result = accessor.doSet(path, record, -1, AccessOption.PERSISTENT);
     Assert.assertEquals(result._retCode, RetCode.OK);
     Assert.assertEquals(result._pathCreated.size(), 3);
-    Assert.assertTrue(result._pathCreated.contains(String.format("/%s", testName)));
-    Assert.assertTrue(result._pathCreated.contains(String.format("/%s/%s", testName, "msg_0")));
+    Assert.assertTrue(result._pathCreated.contains(String.format("/%s/%s", _rootPath, "msg_0")));
     Assert.assertTrue(result._pathCreated.contains(path));
 
-    Assert.assertTrue(_gZkClient.exists(String.format("/%s", testName)));
-    Assert.assertTrue(_gZkClient.exists(String.format("/%s/%s", testName, "msg_0")));
+    Assert.assertTrue(_gZkClient.exists(String.format("/%s/%s", _rootPath, "msg_0")));
     ZNRecord getRecord = _gZkClient.readData(path);
     Assert.assertNotNull(getRecord);
     Assert.assertEquals(getRecord.getId(), "submsg_0");
@@ -156,9 +170,9 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
-    ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+    ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<>(_gZkClient);
 
     boolean success = accessor.create(path, record, AccessOption.PERSISTENT);
     Assert.assertTrue(success);
@@ -184,7 +198,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -227,7 +241,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -255,7 +269,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -314,7 +328,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -339,7 +353,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
 
-    String path = String.format("/%s/%s", testName, "msg_0");
+    String path = String.format("/%s/%s", _rootPath, "msg_0");
     ZNRecord record = new ZNRecord("msg_0");
     ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
 
@@ -360,20 +374,17 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
 
   @Test
   public void testAsyncZkBaseDataAccessor() {
-    System.out.println("START TestZkBaseDataAccessor.async at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println(
+        "START TestZkBaseDataAccessor.async at " + new Date(System.currentTimeMillis()));
 
-    String root = "TestZkBaseDataAccessor_asyn";
-    ZkClient zkClient = new ZkClient(ZK_ADDR);
-    zkClient.setZkSerializer(new ZNRecordSerializer());
-    zkClient.deleteRecursively("/" + root);
+    String root = _rootPath;
+    _gZkClient.deleteRecursively("/" + root);
 
-    ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<ZNRecord>(zkClient);
+    ZkBaseDataAccessor<ZNRecord> accessor = new ZkBaseDataAccessor<>(_gZkClient);
 
     // test async createChildren
-    String parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
-    List<ZNRecord> records = new ArrayList<ZNRecord>();
-    List<String> paths = new ArrayList<String>();
+    List<ZNRecord> records = new ArrayList<>();
+    List<String> paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -389,14 +400,13 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       String path = PropertyPathBuilder.instanceMessage(root, "host_1", msgId);
-      ZNRecord record = zkClient.readData(path);
+      ZNRecord record = _gZkClient.readData(path);
       Assert.assertEquals(record.getId(), msgId, "Should get what we created");
     }
 
     // test async setChildren
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
-    records = new ArrayList<ZNRecord>();
-    paths = new ArrayList<String>();
+    records = new ArrayList<>();
+    paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -414,16 +424,15 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       String path = PropertyPathBuilder.instanceMessage(root, "host_1", msgId);
-      ZNRecord record = zkClient.readData(path);
+      ZNRecord record = _gZkClient.readData(path);
       Assert.assertEquals(record.getSimpleFields().size(), 1, "Should have 1 simple field set");
       Assert.assertEquals(record.getSimpleField("key1"), "value1", "Should have value1 set");
     }
 
     // test async updateChildren
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
     // records = new ArrayList<ZNRecord>();
     List<DataUpdater<ZNRecord>> znrecordUpdaters = new ArrayList<DataUpdater<ZNRecord>>();
-    paths = new ArrayList<String>();
+    paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -442,13 +451,13 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       String path = PropertyPathBuilder.instanceMessage(root, "host_1", msgId);
-      ZNRecord record = zkClient.readData(path);
+      ZNRecord record = _gZkClient.readData(path);
       Assert.assertEquals(record.getSimpleFields().size(), 2, "Should have 2 simple fields set");
       Assert.assertEquals(record.getSimpleField("key2"), "value2", "Should have value2 set");
     }
 
     // test async getChildren
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
+    String parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
     records = accessor.getChildren(parentPath, null, 0);
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
@@ -459,8 +468,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     }
 
     // test async exists
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
-    paths = new ArrayList<String>();
+    paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -472,8 +480,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     }
 
     // test async getStats
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
-    paths = new ArrayList<String>();
+    paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -487,8 +494,7 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     }
 
     // test async remove
-    parentPath = PropertyPathBuilder.instanceMessage(root, "host_1");
-    paths = new ArrayList<String>();
+    paths = new ArrayList<>();
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       paths.add(PropertyPathBuilder.instanceMessage(root, "host_1", msgId));
@@ -503,11 +509,10 @@ public class TestZkBaseDataAccessor extends ZkUnitTestBase {
     for (int i = 0; i < 10; i++) {
       String msgId = "msg_" + i;
       String path = PropertyPathBuilder.instanceMessage(root, "host_1", msgId);
-      boolean pathExists = zkClient.exists(path);
+      boolean pathExists = _gZkClient.exists(path);
       Assert.assertFalse(pathExists, "Should be removed " + msgId);
     }
 
-    zkClient.close();
     System.out.println("END TestZkBaseDataAccessor.async at "
         + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java
index bdad048..d175515 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkManagerFlappingDetection.java
@@ -22,13 +22,13 @@ package org.apache.helix.manager.zk;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase {
+public class TestZkManagerFlappingDetection extends ZkTestBase {
   @Test
   public void testDisconnectHistory() throws Exception {
     String className = TestHelper.getTestClassName();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestBatchMessageModeConfigs.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestBatchMessageModeConfigs.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestBatchMessageModeConfigs.java
index f0f49c5..de7c7da 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestBatchMessageModeConfigs.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestBatchMessageModeConfigs.java
@@ -46,30 +46,30 @@ public class TestBatchMessageModeConfigs extends ZkStandAloneCMTestBase {
 
   @Test
   public void testEnableBatchModeForCluster() throws InterruptedException {
-    _setupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, true);
+    _gSetupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, true);
     String dbName = TEST_DB_PREFIX + "Cluster";
     setupResource(dbName);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
     Thread.sleep(2000L);
     verify();
-    _setupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, false);
+    _gSetupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, false);
   }
 
   @Test
   public void testEnableBatchModeForResource() throws InterruptedException {
     String dbName = TEST_DB_PREFIX + "Resource";
     setupResource(dbName);
-    _setupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, dbName, true);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
+    _gSetupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, dbName, true);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
     Thread.sleep(2000L);
     verify();
-    _setupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, dbName, false);
+    _gSetupTool.getClusterManagementTool().enableBatchMessageMode(CLUSTER_NAME, dbName, false);
   }
 
   private void setupResource(String dbName) throws InterruptedException {
     IdealState idealState = new FullAutoModeISBuilder(dbName).setStateModel("OnlineOffline")
         .setStateModelFactoryName("TestFactory").setNumPartitions(10).setNumReplica(1).build();
-    _setupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
+    _gSetupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
   }
 
   private void verify() {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java
index 3b8e8f0..6ee11ca 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestResourceThreadpoolSize.java
@@ -52,8 +52,8 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
   public void TestThreadPoolSizeConfig() {
     setResourceThreadPoolSize("NextDB", 12);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "NextDB", 64, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "NextDB", 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "NextDB", 64, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "NextDB", 3);
 
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -84,26 +84,26 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
     }
 
     // add db with default thread pool
-    _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "1", 64,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "1", 64,
         STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "1", 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "1", 3);
 
     // add db with customized thread pool
     IdealState idealState = new FullAutoModeISBuilder(WorkflowGenerator.DEFAULT_TGT_DB + "2")
         .setStateModel(ONLINE_OFFLINE).setStateModelFactoryName(TEST_FACTORY).setNumPartitions(10)
         .setNumReplica(1).build();
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .addResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "2", idealState);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "2", 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "2", 1);
 
     // add db with configured pool size
     idealState = new FullAutoModeISBuilder(WorkflowGenerator.DEFAULT_TGT_DB + "3")
         .setStateModel(ONLINE_OFFLINE).setStateModelFactoryName(TEST_FACTORY).setNumPartitions(10)
         .setNumReplica(1).build();
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .addResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "3", idealState);
     setResourceThreadPoolSize(WorkflowGenerator.DEFAULT_TGT_DB + "3", configuredPoolSize);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "3", 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "3", 1);
 
     boolean result = ClusterStateVerifier.verifyByPolling(
         new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
@@ -141,9 +141,9 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
     IdealState idealState = new FullAutoModeISBuilder(WorkflowGenerator.DEFAULT_TGT_DB + "4")
         .setStateModel(MASTER_SLAVE).setStateModelFactoryName(TEST_FACTORY).setNumPartitions(10)
         .setNumReplica(1).build();
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .addResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "4", idealState);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "4", 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "4", 1);
 
     Thread.sleep(2000);
 
@@ -182,8 +182,8 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
       IdealState idealState = new FullAutoModeISBuilder(dbName).setStateModel("OnlineOffline")
           .setStateModelFactoryName("TestFactory").setNumPartitions(10).setNumReplica(1).build();
       idealState.setBatchMessageMode(true);
-      _setupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
+      _gSetupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
     }
     Thread.sleep(2000L);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
index ccaba4e..995e0eb 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterStatusMonitorLifecycle.java
@@ -21,7 +21,7 @@ package org.apache.helix.monitoring;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
@@ -43,15 +43,15 @@ import java.util.Date;
 import java.util.HashSet;
 import java.util.Set;
 
-public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestClusterStatusMonitorLifecycle.class);
+public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(TestClusterStatusMonitorLifecycle.class);
 
   MockParticipantManager[] _participants;
   ClusterDistributedController[] _controllers;
   String _controllerClusterName;
   String _clusterNamePrefix;
   String _firstClusterName;
+  Set<String> _clusters = new HashSet<>();
 
   final int n = 5;
   final int clusterNb = 10;
@@ -77,6 +77,8 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
           n, // number of nodes
           3, // replicas
           "MasterSlave", true); // do rebalance
+
+      _clusters.add(clusterName);
     }
 
     // setup controller cluster
@@ -142,11 +144,18 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
   }
 
   @AfterClass
-  public void afterClass() {
+  public void afterClass() throws Exception {
     System.out.println("Cleaning up...");
     for (int i = 0; i < _participants.length; i++) {
       _participants[i].syncStop();
     }
+    cleanupControllers();
+    _gSetupTool.deleteCluster(_controllerClusterName);
+
+    for (String cluster : _clusters) {
+      TestHelper.dropCluster(cluster, _gZkClient);
+    }
+
     System.out.println("END " + _clusterNamePrefix + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -178,7 +187,9 @@ public class TestClusterStatusMonitorLifecycle extends ZkIntegrationTestBase {
 
   private void cleanupControllers() {
     for (int i = 0; i < _controllers.length; i++) {
-      _controllers[i].syncStop();
+      if (_controllers[i] != null && _controllers[i].isConnected()) {
+        _controllers[i].syncStop();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAggregateMetrics.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAggregateMetrics.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAggregateMetrics.java
index dfe5016..ea7a680 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAggregateMetrics.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestClusterAggregateMetrics.java
@@ -35,7 +35,7 @@ import javax.management.QueryExp;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
@@ -54,7 +54,7 @@ import org.testng.annotations.Test;
  * when a Participant is disabled.
  *
  */
-public class TestClusterAggregateMetrics extends ZkIntegrationTestBase {
+public class TestClusterAggregateMetrics extends ZkTestBase {
 
   // Configurable values for test setup
   private static final int NUM_PARTICIPANTS = 3;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
index 143f3c0..633a472 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
@@ -99,6 +99,7 @@ public class TestDisableResourceMbean extends ZkUnitTestBase {
     for (MockParticipantManager participant : participants) {
       participant.syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDropResourceMetricsReset.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDropResourceMetricsReset.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDropResourceMetricsReset.java
index 2529360..fa7abf0 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDropResourceMetricsReset.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDropResourceMetricsReset.java
@@ -98,6 +98,7 @@ public class TestDropResourceMetricsReset extends ZkUnitTestBase {
     for (MockParticipantManager participant : participants) {
       participant.syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResetClusterMetrics.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResetClusterMetrics.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResetClusterMetrics.java
index 409a10a..d4c64e0 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResetClusterMetrics.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestResetClusterMetrics.java
@@ -72,6 +72,7 @@ public class TestResetClusterMetrics extends ZkUnitTestBase {
 
     // Stop the controller
     controller.syncStop();
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     // Check the metrics
     Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java
index 90990ee..608e2e1 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerElection.java
@@ -53,10 +53,6 @@ public class TestDistControllerElection extends ZkUnitTestBase {
     String className = getShortClassName();
 
     final String clusterName = CLUSTER_PREFIX + "_" + className + "_" + "testController";
-    String path = "/" + clusterName;
-    if (_gZkClient.exists(path)) {
-      _gZkClient.deleteRecursively(path);
-    }
 
     ZKHelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient));
@@ -96,6 +92,8 @@ public class TestDistControllerElection extends ZkUnitTestBase {
     // AssertJUnit.assertNull(election.getController());
     // AssertJUnit.assertNull(election.getLeader());
 
+    TestHelper.dropCluster(clusterName, _gZkClient);
+
     System.out.println("END TestDistControllerElection at " + new Date(System.currentTimeMillis()));
   }
 
@@ -106,10 +104,6 @@ public class TestDistControllerElection extends ZkUnitTestBase {
 
     final String clusterName =
         CONTROLLER_CLUSTER_PREFIX + "_" + className + "_" + "testControllerParticipant";
-    String path = "/" + clusterName;
-    if (_gZkClient.exists(path)) {
-      _gZkClient.deleteRecursively(path);
-    }
 
     ZKHelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient));
@@ -156,6 +150,7 @@ public class TestDistControllerElection extends ZkUnitTestBase {
     // AssertJUnit.assertNull(election.getController());
     // AssertJUnit.assertNull(election.getLeader());
 
+    TestHelper.dropCluster(clusterName, _gZkClient);
     LOG.info("END " + getShortClassName() + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -166,9 +161,6 @@ public class TestDistControllerElection extends ZkUnitTestBase {
 
     final String clusterName = CLUSTER_PREFIX + "_" + className + "_" + "testParticipant";
     String path = "/" + clusterName;
-    if (_gZkClient.exists(path)) {
-      _gZkClient.deleteRecursively(path);
-    }
     TestHelper.setupEmptyCluster(_gZkClient, clusterName);
 
     final String controllerName = "participant_0";
@@ -188,6 +180,8 @@ public class TestDistControllerElection extends ZkUnitTestBase {
     AssertJUnit.assertNull(leaderRecord);
     // AssertJUnit.assertNull(election.getController());
     // AssertJUnit.assertNull(election.getLeader());
+
+    TestHelper.dropCluster(clusterName, _gZkClient);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
index 0642d67..804a0d7 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
@@ -28,6 +28,7 @@ import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.participant.DistClusterControllerStateModel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
@@ -46,6 +47,13 @@ public class TestDistControllerStateModel extends ZkUnitTestBase {
     TestHelper.setupEmptyCluster(_gZkClient, clusterName);
   }
 
+  @AfterMethod
+  public void afterMethod() throws Exception {
+    if (_gZkClient.exists("/" + clusterName)) {
+      TestHelper.dropCluster(clusterName, _gZkClient);
+    }
+  }
+
   @Test()
   public void testOnBecomeStandbyFromOffline() {
     stateModel.onBecomeStandbyFromOffline(new Message(new ZNRecord("test")), null);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/spectator/TestRoutingDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/spectator/TestRoutingDataCache.java b/helix-core/src/test/java/org/apache/helix/spectator/TestRoutingDataCache.java
index 7ff3f67..6cf70bd 100644
--- a/helix-core/src/test/java/org/apache/helix/spectator/TestRoutingDataCache.java
+++ b/helix-core/src/test/java/org/apache/helix/spectator/TestRoutingDataCache.java
@@ -77,8 +77,8 @@ public class TestRoutingDataCache extends ZkStandAloneCMTestBase {
     Assert.assertEquals(accessor.getReadCount(PropertyType.EXTERNALVIEW), 0);
 
     // add new resources
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_1", 1, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_1", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_1", 1, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_1", _replica);
 
     Thread.sleep(100);
     HelixClusterVerifier _clusterVerifier =
@@ -95,10 +95,10 @@ public class TestRoutingDataCache extends ZkStandAloneCMTestBase {
     // Add more resources
     accessor.clearReadCounters();
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_2", 1, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_2", _replica);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_3", 1, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_3", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_2", 1, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_2", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_3", 1, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_3", _replica);
 
     Thread.sleep(100);
     Assert.assertTrue(_clusterVerifier.verify());
@@ -111,7 +111,7 @@ public class TestRoutingDataCache extends ZkStandAloneCMTestBase {
     // update one resource
     accessor.clearReadCounters();
 
-    _setupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, "TestDB_2", false);
+    _gSetupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, "TestDB_2", false);
 
     Thread.sleep(100);
     Assert.assertTrue(_clusterVerifier.verify());

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java b/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
index cab60ce..056e34c 100644
--- a/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
@@ -28,17 +28,20 @@ import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.integration.task.MockTask;
 import org.apache.helix.integration.task.WorkflowGenerator;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 
-public class TaskSynchronizedTestBase extends ZkIntegrationTestBase {
+public class TaskSynchronizedTestBase extends ZkTestBase {
   protected int _numNodes = 5;
   protected int _startPort = 12918;
   protected int _numParitions = 20;
@@ -48,9 +51,9 @@ public class TaskSynchronizedTestBase extends ZkIntegrationTestBase {
   protected Boolean _partitionVary = true;
   protected Boolean _instanceGroupTag = false;
 
+  protected ClusterControllerManager _controller;
   protected HelixManager _manager;
   protected TaskDriver _driver;
-  protected ClusterSetup _setupTool;
 
   protected List<String> _testDbs = new ArrayList<String>();
 
@@ -58,64 +61,85 @@ public class TaskSynchronizedTestBase extends ZkIntegrationTestBase {
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + getShortClassName();
   protected MockParticipantManager[] _participants;
 
+  protected HelixClusterVerifier _clusterVerifier;
+
   @BeforeClass
   public void beforeClass() throws Exception {
     _participants =  new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipants();
     createManagers();
+    _clusterVerifier =
+        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkClient(_gZkClient).build();
   }
 
 
   @AfterClass
   public void afterClass() throws Exception {
-    _manager.disconnect();
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
+    if (_manager != null && _manager.isConnected()) {
+      _manager.disconnect();
+    }
     stopParticipants();
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      try {
+        _gSetupTool.deleteCluster(CLUSTER_NAME);
+      } catch (Exception ex) {
+        System.err.println(
+            "Failed to delete cluster " + CLUSTER_NAME + ", error: " + ex.getLocalizedMessage());
+      }
+    }
   }
 
   protected void setupDBs() {
+    setupDBs(_gSetupTool);
+  }
+
+  protected void setupDBs(ClusterSetup clusterSetup) {
     // Set up target db
     if (_numDbs > 1) {
       for (int i = 0; i < _numDbs; i++) {
         int varyNum = _partitionVary == true ? 10 * i : 0;
         String db = WorkflowGenerator.DEFAULT_TGT_DB + i;
-        _setupTool
+        clusterSetup
             .addResourceToCluster(CLUSTER_NAME, db, _numParitions + varyNum, MASTER_SLAVE_STATE_MODEL,
                 IdealState.RebalanceMode.FULL_AUTO.toString());
-        _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
+        clusterSetup.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
         _testDbs.add(db);
       }
     } else {
       if (_instanceGroupTag) {
-        _setupTool
+        clusterSetup
             .addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numParitions,
                 "OnlineOffline", IdealState.RebalanceMode.FULL_AUTO.name());
-        IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+        IdealState idealState = clusterSetup.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
         idealState.setInstanceGroupTag("TESTTAG0");
-        _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, idealState);
+        clusterSetup.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, idealState);
       } else {
-        _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB,
+        clusterSetup.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB,
             _numParitions, MASTER_SLAVE_STATE_MODEL, IdealState.RebalanceMode.FULL_AUTO.name());
       }
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
+      clusterSetup.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _numReplicas);
     }
   }
 
   protected void setupParticipants() {
+    setupParticipants(_gSetupTool);
+  }
+
+  protected void setupParticipants(ClusterSetup setupTool) {
     _participants = new MockParticipantManager[_numNodes];
     for (int i = 0; i < _numNodes; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       if (_instanceGroupTag) {
-        _setupTool.addInstanceTag(CLUSTER_NAME, storageNodeName, "TESTTAG" + i);
+        setupTool.addInstanceTag(CLUSTER_NAME, storageNodeName, "TESTTAG" + i);
       }
     }
   }
@@ -173,7 +197,7 @@ public class TaskSynchronizedTestBase extends ZkIntegrationTestBase {
           + "were set up.", i, _participants.length));
     }
     if (_participants[i] != null && _participants[i].isConnected()) {
-      _participants[i].reset();
+      _participants[i].syncStop();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/task/TestAssignableInstanceManagerControllerSwitch.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TestAssignableInstanceManagerControllerSwitch.java b/helix-core/src/test/java/org/apache/helix/task/TestAssignableInstanceManagerControllerSwitch.java
index f971820..21278f2 100644
--- a/helix-core/src/test/java/org/apache/helix/task/TestAssignableInstanceManagerControllerSwitch.java
+++ b/helix-core/src/test/java/org/apache/helix/task/TestAssignableInstanceManagerControllerSwitch.java
@@ -58,7 +58,7 @@ public class TestAssignableInstanceManagerControllerSwitch extends TaskTestBase
       String instanceName = liveInstance.getInstanceName();
       liveInstanceMap.put(instanceName, liveInstance);
       instanceConfigMap.put(instanceName,
-          _setupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instanceName));
+          _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instanceName));
     }
 
     // Get ClusterConfig

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/task/TestJobStateOnCreation.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TestJobStateOnCreation.java b/helix-core/src/test/java/org/apache/helix/task/TestJobStateOnCreation.java
index c454879..c5707bc 100644
--- a/helix-core/src/test/java/org/apache/helix/task/TestJobStateOnCreation.java
+++ b/helix-core/src/test/java/org/apache/helix/task/TestJobStateOnCreation.java
@@ -50,12 +50,8 @@ public class TestJobStateOnCreation extends TaskSynchronizedTestBase {
     _resource = new Resource(WORKFLOW_NAME);
     _currStateOutput = new CurrentStateOutput();
     _participants =  new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     createManagers();
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/task/TestSemiAutoStateTransition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TestSemiAutoStateTransition.java b/helix-core/src/test/java/org/apache/helix/task/TestSemiAutoStateTransition.java
index 2f75c09..9d34980 100644
--- a/helix-core/src/test/java/org/apache/helix/task/TestSemiAutoStateTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/task/TestSemiAutoStateTransition.java
@@ -47,22 +47,16 @@ public class TestSemiAutoStateTransition extends TaskTestBase {
     _participants =  new MockParticipantManager[_numNodes];
     _numParitions = 1;
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     _accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
     _keyBuilder = _accessor.keyBuilder();
     setupParticipants();
 
     for (int i = 0; i < _numDbs; i++) {
       String db = WorkflowGenerator.DEFAULT_TGT_DB + i;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _numParitions, MASTER_SLAVE_STATE_MODEL,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _numParitions, MASTER_SLAVE_STATE_MODEL,
           IdealState.RebalanceMode.SEMI_AUTO.toString());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _numReplicas);
       _testDbs.add(db);
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java b/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java
index a959127..ec2c5a9 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestClusterSetup.java
@@ -372,6 +372,9 @@ public class TestClusterSetup extends ZkUnitTestBase {
     exists = _gZkClient.exists(keyBuilder.pause().getPath());
     Assert.assertFalse(exists, "pause node under controller should be removed");
 
+    // clean up
+    TestHelper.dropCluster(clusterName, _gZkClient);
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
   }
@@ -435,11 +438,13 @@ public class TestClusterSetup extends ZkUnitTestBase {
 
     Assert.assertNull(accessor.getProperty(keyBuilder.instanceConfig("localhost_12918")),
         "Instance config should be dropped");
-    Assert.assertFalse(_gZkClient.exists(PropertyPathBuilder.instance(clusterName, "localhost_12918")),
+    Assert.assertFalse(
+        _gZkClient.exists(PropertyPathBuilder.instance(clusterName, "localhost_12918")),
         "Instance/host should be dropped");
 
-    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test
@@ -471,6 +476,8 @@ public class TestClusterSetup extends ZkUnitTestBase {
     });
     idealState = accessor.getProperty(keyBuilder.idealStates("TestDB0"));
     Assert.assertTrue(idealState.isEnabled());
+
+    TestHelper.dropCluster(clusterName, _gZkClient);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
index caa8696..842fcfa 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
@@ -29,7 +29,7 @@ import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -45,7 +45,7 @@ import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestHelixAdminCli extends ZkIntegrationTestBase {
+public class TestHelixAdminCli extends ZkTestBase {
   @Test
   public void testAddCluster() throws Exception {
     String command = "--zkSvr localhost:2183 -addCluster clusterTest";


[3/5] helix git commit: Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
deleted file mode 100644
index 85b4d45..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
+++ /dev/null
@@ -1,322 +0,0 @@
-package org.apache.helix.integration.common;
-
-/*
- * 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.
- */
-
-import java.lang.reflect.Method;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Level;
-import org.I0Itec.zkclient.ZkServer;
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.pipeline.AbstractAsyncBaseStage;
-import org.apache.helix.controller.pipeline.Stage;
-import org.apache.helix.controller.pipeline.StageContext;
-import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
-import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy;
-import org.apache.helix.controller.stages.AttributeName;
-import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.BuiltInStateModelDefinitions;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.ConfigScope;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.OnlineOfflineSMD;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.util.ZKClientPool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.Assert;
-import org.testng.AssertJUnit;
-import org.testng.ITestContext;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.AfterSuite;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.BeforeSuite;
-
-public class ZkIntegrationTestBase {
-  private static Logger LOG = LoggerFactory.getLogger(ZkIntegrationTestBase.class);
-
-  protected static ZkServer _zkServer;
-  protected static ZkClient _gZkClient;
-  protected static ClusterSetup _gSetupTool;
-  protected static BaseDataAccessor<ZNRecord> _baseAccessor;
-
-  public static final String ZK_ADDR = "localhost:2183";
-  protected static final String CLUSTER_PREFIX = "CLUSTER";
-  protected static final String CONTROLLER_CLUSTER_PREFIX = "CONTROLLER_CLUSTER";
-
-  protected final String CONTROLLER_PREFIX = "controller";
-  protected final String PARTICIPANT_PREFIX = "localhost";
-
-  @BeforeSuite
-  public void beforeSuite() throws Exception {
-    // TODO: use logging.properties file to config java.util.logging.Logger levels
-    java.util.logging.Logger topJavaLogger = java.util.logging.Logger.getLogger("");
-    topJavaLogger.setLevel(Level.WARNING);
-
-    // Due to ZOOKEEPER-2693 fix, we need to specify whitelist for execute zk commends
-    System.setProperty("zookeeper.4lw.commands.whitelist", "*");
-
-    _zkServer = TestHelper.startZkServer(ZK_ADDR);
-    AssertJUnit.assertTrue(_zkServer != null);
-    ZKClientPool.reset();
-
-    _gZkClient = new ZkClient(ZK_ADDR);
-    _gZkClient.setZkSerializer(new ZNRecordSerializer());
-    _gSetupTool = new ClusterSetup(_gZkClient);
-    _baseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
-  }
-
-  @AfterSuite
-  public void afterSuite() {
-    ZKClientPool.reset();
-    _gZkClient.close();
-    TestHelper.stopZkServer(_zkServer);
-  }
-
-  @BeforeMethod
-  public void beforeTest(Method testMethod, ITestContext testContext){
-    long startTime = System.currentTimeMillis();
-    System.out.println("START " + testMethod.getName() + " at " + new Date(startTime));
-    testContext.setAttribute("StartTime", System.currentTimeMillis());
-  }
-
-  @AfterMethod
-  public void endTest(Method testMethod, ITestContext testContext) {
-    Long startTime = (Long) testContext.getAttribute("StartTime");
-    long endTime = System.currentTimeMillis();
-    System.out.println(
-        "END " + testMethod.getName() + " at " + new Date(endTime) + ", took: " + (endTime - startTime) + "ms.");
-  }
-
-  protected String getShortClassName() {
-    return this.getClass().getSimpleName();
-  }
-
-  protected String getCurrentLeader(ZkClient zkClient, String clusterName) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader());
-    if (leader == null) {
-      return null;
-    }
-    return leader.getInstanceName();
-  }
-
-  protected void enableHealthCheck(String clusterName) {
-    ConfigScope scope = new ConfigScopeBuilder().forCluster(clusterName).build();
-    new ConfigAccessor(_gZkClient).set(scope, "healthChange" + ".enabled", "" + true);
-  }
-
-  protected void enablePersistBestPossibleAssignment(ZkClient zkClient, String clusterName,
-      Boolean enabled) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
-    clusterConfig.setPersistBestPossibleAssignment(enabled);
-    configAccessor.setClusterConfig(clusterName, clusterConfig);
-  }
-
-  protected void enablePersistIntermediateAssignment(ZkClient zkClient, String clusterName,
-      Boolean enabled) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
-    clusterConfig.setPersistIntermediateAssignment(enabled);
-    configAccessor.setClusterConfig(clusterName, clusterConfig);
-  }
-
-  protected void enableTopologyAwareRebalance(ZkClient zkClient, String clusterName,
-      Boolean enabled) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
-    clusterConfig.setTopologyAwareEnabled(enabled);
-    configAccessor.setClusterConfig(clusterName, clusterConfig);
-  }
-
-  protected void enableDelayRebalanceInCluster(ZkClient zkClient, String clusterName,
-      boolean enabled) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
-    clusterConfig.setDelayRebalaceEnabled(enabled);
-    configAccessor.setClusterConfig(clusterName, clusterConfig);
-  }
-
-  protected void enableDelayRebalanceInInstance(ZkClient zkClient, String clusterName,
-      String instanceName, boolean enabled) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    InstanceConfig instanceConfig = configAccessor.getInstanceConfig(clusterName, instanceName);
-    instanceConfig.setDelayRebalanceEnabled(enabled);
-    configAccessor.setInstanceConfig(clusterName, instanceName, instanceConfig);
-  }
-
-  protected void setDelayTimeInCluster(ZkClient zkClient, String clusterName, long delay) {
-    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
-    clusterConfig.setRebalanceDelayTime(delay);
-    configAccessor.setClusterConfig(clusterName, clusterConfig);
-  }
-
-  protected IdealState createResourceWithDelayedRebalance(String clusterName, String db,
-      String stateModel, int numPartition, int replica, int minActiveReplica, long delay) {
-    return createResourceWithDelayedRebalance(clusterName, db, stateModel, numPartition, replica,
-        minActiveReplica, delay, AutoRebalanceStrategy.class.getName());
-  }
-
-  protected IdealState createResourceWithDelayedRebalance(String clusterName, String db,
-      String stateModel, int numPartition, int replica, int minActiveReplica, long delay,
-      String rebalanceStrategy) {
-    IdealState idealState =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
-    if (idealState == null) {
-      _gSetupTool.addResourceToCluster(clusterName, db, numPartition, stateModel,
-          IdealState.RebalanceMode.FULL_AUTO + "", rebalanceStrategy);
-    }
-
-    idealState =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
-    idealState.setMinActiveReplicas(minActiveReplica);
-    if (!idealState.isDelayRebalanceEnabled()) {
-      idealState.setDelayRebalanceEnabled(true);
-    }
-    if (delay > 0) {
-      idealState.setRebalanceDelay(delay);
-    }
-    idealState.setRebalancerClassName(DelayedAutoRebalancer.class.getName());
-    _gSetupTool.getClusterManagementTool().setResourceIdealState(clusterName, db, idealState);
-    _gSetupTool.rebalanceStorageCluster(clusterName, db, replica);
-    idealState =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
-
-    return idealState;
-  }
-
-  protected IdealState createIdealState(String resourceGroupName, String instanceGroupTag,
-      List<String> instanceNames, int numPartition, int replica, String rebalanceMode,
-      String stateModelDef) {
-    IdealState is = _gSetupTool
-        .createIdealStateForResourceGroup(resourceGroupName, instanceGroupTag, numPartition,
-            replica, rebalanceMode, stateModelDef);
-
-    // setup initial partition->instance mapping.
-    int nodeIdx = 0;
-    int numNode = instanceNames.size();
-    assert (numNode >= replica);
-    for (int i = 0; i < numPartition; i++) {
-      String partitionName = resourceGroupName + "_" + i;
-      for (int j = 0; j < replica; j++) {
-        is.setPartitionState(partitionName, instanceNames.get((nodeIdx + j) % numNode),
-            OnlineOfflineSMD.States.ONLINE.toString());
-      }
-      nodeIdx++;
-    }
-
-    return is;
-  }
-
-  protected void createDBInSemiAuto(ClusterSetup clusterSetup, String clusterName, String dbName,
-      List<String> preferenceList, String stateModelDef, int numPartition, int replica) {
-    clusterSetup.addResourceToCluster(clusterName, dbName, numPartition, stateModelDef,
-        IdealState.RebalanceMode.SEMI_AUTO.toString());
-    clusterSetup.rebalanceStorageCluster(clusterName, dbName, replica);
-
-    IdealState is =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, dbName);
-    for (String p : is.getPartitionSet()) {
-      is.setPreferenceList(p, preferenceList);
-    }
-    clusterSetup.getClusterManagementTool().setResourceIdealState(clusterName, dbName, is);
-  }
-
-  /**
-   * Validate there should be always minimal active replica and top state replica for each partition.
-   * Also make sure there is always some partitions with only active replica count.
-   */
-  protected void validateMinActiveAndTopStateReplica(IdealState is, ExternalView ev,
-      int minActiveReplica, int numNodes) {
-    StateModelDefinition stateModelDef =
-        BuiltInStateModelDefinitions.valueOf(is.getStateModelDefRef()).getStateModelDefinition();
-    String topState = stateModelDef.getStatesPriorityList().get(0);
-    int replica = Integer.valueOf(is.getReplicas());
-
-    Map<String, Integer> stateCount =
-        stateModelDef.getStateCountMap(numNodes, replica);
-    Set<String> activeStates = stateCount.keySet();
-
-    for (String partition : is.getPartitionSet()) {
-      Map<String, String> assignmentMap = ev.getRecord().getMapField(partition);
-      Assert.assertNotNull(assignmentMap,
-          is.getResourceName() + "'s best possible assignment is null for partition " + partition);
-      Assert.assertTrue(!assignmentMap.isEmpty(),
-          is.getResourceName() + "'s partition " + partition + " has no best possible map in IS.");
-
-      boolean hasTopState = false;
-      int activeReplica = 0;
-      for (String state : assignmentMap.values()) {
-        if (topState.equalsIgnoreCase(state)) {
-          hasTopState = true;
-        }
-        if (activeStates.contains(state)) {
-          activeReplica++;
-        }
-      }
-
-      if (activeReplica < minActiveReplica) {
-        int a = 0;
-      }
-
-      Assert.assertTrue(hasTopState, String.format("%s missing %s replica", partition, topState));
-      Assert.assertTrue(activeReplica >= minActiveReplica, String
-          .format("%s has less active replica %d then required %d", partition, activeReplica,
-              minActiveReplica));
-    }
-  }
-
-  protected void runStage(HelixManager manager, ClusterEvent event, Stage stage) throws Exception {
-    event.addAttribute(AttributeName.helixmanager.name(), manager);
-    StageContext context = new StageContext();
-    stage.init(context);
-    stage.preProcess();
-
-    // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in
-    // execute() function call
-    if (stage instanceof AbstractAsyncBaseStage) {
-      ((AbstractAsyncBaseStage) stage).execute(event);
-    } else {
-      stage.process(event);
-    }
-    stage.postProcess();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/common/ZkStandAloneCMTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/common/ZkStandAloneCMTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/common/ZkStandAloneCMTestBase.java
index 4027984..3cbffe1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/common/ZkStandAloneCMTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/common/ZkStandAloneCMTestBase.java
@@ -24,12 +24,14 @@ import java.util.Date;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -41,7 +43,7 @@ import org.testng.annotations.BeforeClass;
  * start 5 dummy participants verify the current states at end
  */
 
-public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
+public class ZkStandAloneCMTestBase extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(ZkStandAloneCMTestBase.class);
 
   protected static final int NODE_NR = 5;
@@ -50,11 +52,12 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
   protected static final String TEST_DB = "TestDB";
   protected static final int _PARTITIONS = 20;
 
-  protected ClusterSetup _setupTool;
   protected HelixManager _manager;
   protected final String CLASS_NAME = getShortClassName();
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
+  HelixClusterVerifier _clusterVerifier;
+
   protected MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
   protected ClusterControllerManager _controller;
 
@@ -65,20 +68,14 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, _replica);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, _replica);
 
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
@@ -96,6 +93,8 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
         ClusterStateVerifier
             .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
 
+    _clusterVerifier = new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+
     Assert.assertTrue(result);
 
     result =
@@ -114,7 +113,6 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
     /**
      * shutdown order: 1) disconnect the controller 2) disconnect participants
      */
-
     if (_controller != null && _controller.isConnected()) {
       _controller.syncStop();
     }
@@ -127,6 +125,16 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
       _manager.disconnect();
     }
 
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      try {
+        _gSetupTool.deleteCluster(CLUSTER_NAME);
+      } catch (Exception ex) {
+        System.err.println(
+            "Failed to delete cluster " + CLUSTER_NAME + ", error: " + ex.getLocalizedMessage());
+      }
+    }
+
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterDataCacheSelectiveUpdate.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterDataCacheSelectiveUpdate.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterDataCacheSelectiveUpdate.java
index b032513..6202f84 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterDataCacheSelectiveUpdate.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterDataCacheSelectiveUpdate.java
@@ -100,8 +100,8 @@ public class TestClusterDataCacheSelectiveUpdate extends ZkStandAloneCMTestBase
     Assert.assertEquals(accessor.getReadCount(PropertyType.CONFIGS), 1);
 
     // add a new resource
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_1", _PARTITIONS, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_1", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_1", _PARTITIONS, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_1", _replica);
 
     Thread.sleep(100);
     HelixClusterVerifier _clusterVerifier =
@@ -119,10 +119,10 @@ public class TestClusterDataCacheSelectiveUpdate extends ZkStandAloneCMTestBase
     // Add more resources
     accessor.clearReadCounters();
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_2", _PARTITIONS, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_2", _replica);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_3", _PARTITIONS, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_3", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_2", _PARTITIONS, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_2", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "TestDB_3", _PARTITIONS, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "TestDB_3", _replica);
 
     // Totally four resources. Two of them are newly added.
     cache.notifyDataChange(HelixConstants.ChangeType.IDEAL_STATE);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
index 3949183..e643c9a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
@@ -7,10 +7,13 @@ import org.apache.helix.integration.task.WorkflowGenerator;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestClusterMaintenanceMode extends TaskTestBase {
+  MockParticipantManager _newInstance;
+
   @BeforeClass
   public void beforeClass() throws Exception {
     _numDbs = 1;
@@ -20,6 +23,14 @@ public class TestClusterMaintenanceMode extends TaskTestBase {
     super.beforeClass();
   }
 
+  @AfterClass
+  public void afterClass() throws Exception {
+    if (_newInstance != null && _newInstance.isConnected()) {
+      _newInstance.syncStop();
+    }
+    super.afterClass();
+  }
+
   @Test
   public void testMaintenanceModeAddNewInstance() throws InterruptedException {
     _gSetupTool.getClusterManagementTool().enableMaintenanceMode(CLUSTER_NAME, true, "Test");
@@ -27,10 +38,10 @@ public class TestClusterMaintenanceMode extends TaskTestBase {
     ExternalView prevExternalView = _gSetupTool.getClusterManagementTool()
         .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
     String instanceName = PARTICIPANT_PREFIX + "_" + (_startPort + 10);
-    _setupTool.addInstanceToCluster(CLUSTER_NAME, instanceName);
-    MockParticipantManager newInstance =
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instanceName);
+    _newInstance =
         new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-    newInstance.syncStart();
+    _newInstance.syncStart();
     _gSetupTool.getClusterManagementTool()
         .rebalance(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, 3);
     Thread.sleep(3000);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLeadershipChange.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLeadershipChange.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLeadershipChange.java
index c44c924..6c0236f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLeadershipChange.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLeadershipChange.java
@@ -28,7 +28,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyPathBuilder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
@@ -36,7 +36,7 @@ import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestControllerLeadershipChange extends ZkIntegrationTestBase{
+public class TestControllerLeadershipChange extends ZkTestBase {
 
   @Test
   public void testMissingTopStateDurationMonitoring() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLiveLock.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLiveLock.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLiveLock.java
index b2b291b..0dff764 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLiveLock.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestControllerLiveLock.java
@@ -124,6 +124,7 @@ public class TestControllerLiveLock extends ZkUnitTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/controller/TestSkipBestPossibleCalculation.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestSkipBestPossibleCalculation.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestSkipBestPossibleCalculation.java
index 79ae4c4..df5bf10 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/controller/TestSkipBestPossibleCalculation.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestSkipBestPossibleCalculation.java
@@ -40,9 +40,9 @@ public class TestSkipBestPossibleCalculation extends ZkStandAloneCMTestBase {
     int numResource = 5;
     for (int i = 0; i < numResource; i++) {
       String dbName = "TestDB_" + i;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, dbName, _PARTITIONS, STATE_MODEL,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, dbName, _PARTITIONS, STATE_MODEL,
           IdealState.RebalanceMode.CUSTOMIZED.name());
-      _setupTool.rebalanceResource(CLUSTER_NAME, dbName, 3);
+      _gSetupTool.rebalanceResource(CLUSTER_NAME, dbName, 3);
     }
 
     ClusterDataCache cache =

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
index 207044a..ef59f31 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
@@ -28,7 +28,7 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.manager.zk.CallbackHandler;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKHelixManager;
@@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestDistributedControllerManager extends ZkIntegrationTestBase {
+public class TestDistributedControllerManager extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestDistributedControllerManager.class);
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/manager/TestHelixDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestHelixDataAccessor.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestHelixDataAccessor.java
index 39fdb2a..7e6d931 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestHelixDataAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestHelixDataAccessor.java
@@ -6,15 +6,12 @@ import java.util.Map;
 import java.util.TreeMap;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.exceptions.HelixMetaDataAccessException;
-import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.ClusterDataCache;
-import org.apache.helix.controller.stages.ReadClusterDataStage;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.MockZkClient;
@@ -22,7 +19,7 @@ import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestHelixDataAccessor extends ZkIntegrationTestBase {
+public class TestHelixDataAccessor extends ZkTestBase {
   private MockZkClient _zkClient;
   BaseDataAccessor<ZNRecord> baseDataAccessor;
   HelixDataAccessor accessor;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
index d2ab0bb..d21f46e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
@@ -32,7 +32,7 @@ import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
@@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestParticipantManager extends ZkIntegrationTestBase {
+public class TestParticipantManager extends ZkTestBase {
 
   private static Logger LOG = LoggerFactory.getLogger(TestParticipantManager.class);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessage.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessage.java
index bd44ca3..f2c4622 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessage.java
@@ -30,7 +30,7 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.HelixProperty.HelixPropertyAttribute;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -44,7 +44,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestBatchMessage extends ZkIntegrationTestBase {
+public class TestBatchMessage extends ZkTestBase {
   class TestZkChildListener implements IZkChildListener {
     int _maxNbOfChilds = 0;
 
@@ -117,6 +117,7 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -199,6 +200,7 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -270,8 +272,14 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     // verify "TestDB0_0", masterOfPartition0 is in ERROR state
     TestHelper.verifyState(clusterName, ZK_ADDR, errorStateMap, "ERROR");
 
-    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
+    _gSetupTool.deleteCluster(clusterName);
 
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test
@@ -351,6 +359,7 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessageWrapper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessageWrapper.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessageWrapper.java
index 1a9b65a..f3f469e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessageWrapper.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestBatchMessageWrapper.java
@@ -135,6 +135,7 @@ public class TestBatchMessageWrapper extends ZkUnitTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle.java
index 5f6c6b4..01fb0d8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle.java
@@ -28,7 +28,7 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -43,7 +43,7 @@ import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestMessageThrottle extends ZkIntegrationTestBase {
+public class TestMessageThrottle extends ZkTestBase {
   @Test()
   public void testMessageThrottle() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle2.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle2.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle2.java
index d9edaf1..83ac0bc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle2.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestMessageThrottle2.java
@@ -41,7 +41,7 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKHelixManager;
@@ -67,7 +67,7 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 // test case from Ming Fang
-public class TestMessageThrottle2 extends ZkIntegrationTestBase {
+public class TestMessageThrottle2 extends ZkTestBase {
   final static String clusterName = "TestMessageThrottle2";
   final static String resourceName = "MyResource";
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
index 43fedad..d1fa2c0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
@@ -28,7 +28,7 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.api.config.HelixConfigProperty;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.integration.DelayedTransitionBase;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -41,10 +41,11 @@ import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
+public class TestP2PMessageSemiAuto extends ZkTestBase {
   final String CLASS_NAME = getShortClassName();
   final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
@@ -57,7 +58,7 @@ public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
   static final int PARTITION_NUMBER = 20;
   static final int REPLICA_NUMBER = 3;
 
-  List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  List<MockParticipantManager> _participants = new ArrayList<>();
   List<String> _instances = new ArrayList<>();
   ClusterControllerManager _controller;
 
@@ -106,6 +107,18 @@ public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
     _accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
   }
 
+  @AfterClass
+  public void afterClass() throws Exception {
+    _controller.syncStop();
+    for (MockParticipantManager p : _participants) {
+      if (p.isConnected()) {
+        p.syncStop();
+      }
+    }
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+  }
+
   @Test
   public void testP2PStateTransitionDisabled() {
     // disable the master instance

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestInstanceAutoJoin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestInstanceAutoJoin.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestInstanceAutoJoin.java
index 007335d..f0ffc63 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestInstanceAutoJoin.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestInstanceAutoJoin.java
@@ -38,10 +38,10 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
     HelixManager manager = _participants[0];
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "OnlineOffline", RebalanceMode.FULL_AUTO
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "OnlineOffline", RebalanceMode.FULL_AUTO
         + "");
 
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 1);
     String instance2 = "localhost_279699";
     // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2);
     MockParticipantManager newParticipant =

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNonOfflineInitState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNonOfflineInitState.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNonOfflineInitState.java
index bd9e168..a937c41 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNonOfflineInitState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNonOfflineInitState.java
@@ -22,12 +22,11 @@ package org.apache.helix.integration.paticipant;
 import java.util.Date;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockBootstrapModelFactory;
 import org.apache.helix.participant.StateMachineEngine;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.slf4j.Logger;
@@ -35,7 +34,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestNonOfflineInitState extends ZkIntegrationTestBase {
+public class TestNonOfflineInitState extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestNonOfflineInitState.class);
 
   @Test
@@ -81,6 +80,7 @@ public class TestNonOfflineInitState extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END testNonOfflineInitState at " + new Date(System.currentTimeMillis()));
   }
@@ -88,26 +88,21 @@ public class TestNonOfflineInitState extends ZkIntegrationTestBase {
   private static void setupCluster(String clusterName, String ZkAddr, int startPort,
       String participantNamePrefix, String resourceNamePrefix, int resourceNb, int partitionNb,
       int nodesNb, int replica, String stateModelDef, boolean doRebalance) throws Exception {
-    if (_gZkClient.exists("/" + clusterName)) {
-      LOG.warn("Cluster already exists:" + clusterName + ". Deleting it");
-      _gZkClient.deleteRecursively("/" + clusterName);
-    }
 
-    ClusterSetup setupTool = new ClusterSetup(ZkAddr);
-    setupTool.addCluster(clusterName, true);
-    setupTool.addStateModelDef(clusterName, "Bootstrap",
+    _gSetupTool.addCluster(clusterName, true);
+    _gSetupTool.addStateModelDef(clusterName, "Bootstrap",
         TestHelper.generateStateModelDefForBootstrap());
 
     for (int i = 0; i < nodesNb; i++) {
       int port = startPort + i;
-      setupTool.addInstanceToCluster(clusterName, participantNamePrefix + "_" + port);
+      _gSetupTool.addInstanceToCluster(clusterName, participantNamePrefix + "_" + port);
     }
 
     for (int i = 0; i < resourceNb; i++) {
       String dbName = resourceNamePrefix + i;
-      setupTool.addResourceToCluster(clusterName, dbName, partitionNb, stateModelDef);
+      _gSetupTool.addResourceToCluster(clusterName, dbName, partitionNb, stateModelDef);
       if (doRebalance) {
-        setupTool.rebalanceStorageCluster(clusterName, dbName, replica);
+        _gSetupTool.rebalanceStorageCluster(clusterName, dbName, replica);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestRestartParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestRestartParticipant.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestRestartParticipant.java
index 1fe742e..cf9f240 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestRestartParticipant.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestRestartParticipant.java
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
@@ -34,7 +34,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestRestartParticipant extends ZkIntegrationTestBase {
+public class TestRestartParticipant extends ZkTestBase {
   public class KillOtherTransition extends MockTransition {
     final AtomicReference<MockParticipantManager> _other;
 
@@ -111,6 +111,7 @@ public class TestRestartParticipant extends ZkIntegrationTestBase {
       participants[i].syncStop();
     }
     participant.syncStop();
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeout.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeout.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeout.java
index 438050d..83b97a5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeout.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeout.java
@@ -60,25 +60,19 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
 
     // Set the timeout values
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     String stateTransition = "SLAVE" + "-" + "MASTER" + "_" + Message.Attributes.TIMEOUT;
     idealState.getRecord().setSimpleField(stateTransition, "300");
 
@@ -145,7 +139,7 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
   public void testStateTransitionTimeOut() throws Exception {
     Map<String, SleepStateModelFactory> factories = new HashMap<String, SleepStateModelFactory>();
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       SleepStateModelFactory factory = new SleepStateModelFactory(1000);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeoutWithResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeoutWithResource.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeoutWithResource.java
index b7491f2..2e9af04 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeoutWithResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestStateTransitionTimeoutWithResource.java
@@ -70,18 +70,12 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     _manager = HelixManagerFactory
@@ -155,9 +149,9 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
 
   @Test
   public void testStateTransitionTimeOut() throws Exception {
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
-    _setupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB, false);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
+    _gSetupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB, false);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
 
     // Set the timeout values
     StateTransitionTimeoutConfig stateTransitionTimeoutConfig =
@@ -171,7 +165,7 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
     setParticipants(TEST_DB);
 
 
-    _setupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB, true);
+    _gSetupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB, true);
     boolean result =
         ClusterStateVerifier
             .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
@@ -182,9 +176,9 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
 
   @Test
   public void testStateTransitionTimeoutByClusterLevel() throws InterruptedException {
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB + 1, _PARTITIONS, STATE_MODEL);
-    _setupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB + 1, false);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB  + 1, 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB + 1, _PARTITIONS, STATE_MODEL);
+    _gSetupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB + 1, false);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB  + 1, 3);
 
     StateTransitionTimeoutConfig stateTransitionTimeoutConfig =
         new StateTransitionTimeoutConfig(new ZNRecord(TEST_DB + 1));
@@ -195,7 +189,7 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
 
     setParticipants(TEST_DB + 1);
 
-    _setupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB + 1, true);
+    _gSetupTool.getClusterManagementTool().enableResource(CLUSTER_NAME, TEST_DB + 1, true);
     boolean result =
         ClusterStateVerifier
             .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
@@ -205,7 +199,7 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
 
   private void verify(String dbName) {
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView(dbName));
     for (String p : idealState.getPartitionSet()) {
@@ -221,7 +215,7 @@ public class TestStateTransitionTimeoutWithResource extends ZkStandAloneCMTestBa
   private void setParticipants(String dbName) throws InterruptedException {
     _factories = new HashMap<>();
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
     for (int i = 0; i < NODE_NR; i++) {
       if (_participants[i] != null) {
         _participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
index 0b90af6..05286a7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
@@ -28,7 +28,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.controller.rebalancer.strategy.*;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
@@ -36,7 +36,6 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
@@ -46,7 +45,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
-public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
+public class TestCrushAutoRebalance extends ZkTestBase {
   final int NUM_NODE = 6;
   protected static final int START_PORT = 12918;
   protected static final int _PARTITIONS = 20;
@@ -55,12 +54,11 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
   protected ClusterControllerManager _controller;
 
-  protected ClusterSetup _setupTool = null;
-  List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
-  Map<String, String> _nodeToZoneMap = new HashMap<String, String>();
-  Map<String, String> _nodeToTagMap = new HashMap<String, String>();
-  List<String> _nodes = new ArrayList<String>();
-  Set<String> _allDBs = new HashSet<String>();
+  List<MockParticipantManager> _participants = new ArrayList<>();
+  Map<String, String> _nodeToZoneMap = new HashMap<>();
+  Map<String, String> _nodeToTagMap = new HashMap<>();
+  List<String> _nodes = new ArrayList<>();
+  Set<String> _allDBs = new HashSet<>();
   int _replica = 3;
 
   String[] _testModels = { BuiltInStateModelDefinitions.OnlineOffline.name(),
@@ -72,20 +70,15 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       String zone = "zone-" + i % 3;
       String tag = "tag-" + i % 2;
-      _setupTool.getClusterManagementTool().setInstanceZoneId(CLUSTER_NAME, storageNodeName, zone);
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
+      _gSetupTool.getClusterManagementTool().setInstanceZoneId(CLUSTER_NAME, storageNodeName, zone);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
       _nodeToZoneMap.put(storageNodeName, zone);
       _nodeToTagMap.put(storageNodeName, tag);
       _nodes.add(storageNodeName);
@@ -123,9 +116,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -136,9 +129,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
 
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateZoneAndTagIsolation(is, ev, _replica);
     }
   }
@@ -150,13 +143,13 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     int i = 0;
     for (String tag : tags) {
       String db = "Test-DB-Tag-" + rebalanceStrategyName + "-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS,
           BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO + "",
           rebalanceStrategyClass);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       is.setInstanceGroupTag(tag);
-      _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -167,9 +160,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
 
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateZoneAndTagIsolation(is, ev, _replica);
     }
   }
@@ -191,9 +184,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-CrushRebalanceStrategy-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", CrushRebalanceStrategy.class.getName());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -204,9 +197,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
 
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateZoneAndTagIsolation(is, ev, 2);
     }
   }
@@ -222,19 +215,19 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
       MockParticipantManager p = _participants.get(i);
       if (_nodeToZoneMap.get(p.getInstanceName()).equals(zone)){
         p.syncStop();
-        _setupTool.getClusterManagementTool()
+        _gSetupTool.getClusterManagementTool()
             .enableInstance(CLUSTER_NAME, p.getInstanceName(), false);
         Thread.sleep(50);
-        _setupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
+        _gSetupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
       }
     }
 
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-CrushRebalanceStrategy-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", CrushRebalanceStrategy.class.getName());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -245,9 +238,9 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
 
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateZoneAndTagIsolation(is, ev, 2);
     }
   }
@@ -255,7 +248,7 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
   @AfterMethod
   public void afterMethod() throws Exception {
     for (String db : _allDBs) {
-      _setupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
     }
     _allDBs.clear();
     // waiting for all DB be dropped.
@@ -278,7 +271,7 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
         assignedZones.add(_nodeToZoneMap.get(instance));
         if (tag != null) {
           InstanceConfig config =
-              _setupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
+              _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
           Assert.assertTrue(config.containsTag(tag));
         }
       }
@@ -307,10 +300,12 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
      * shutdown order: 1) disconnect the controller 2) disconnect participants
      */
     _controller.syncStop();
-    for (MockParticipantManager participant : _participants) {
-      participant.syncStop();
+    for (MockParticipantManager p : _participants) {
+      if (p.isConnected()) {
+        p.syncStop();
+      }
     }
-    _setupTool.deleteCluster(CLUSTER_NAME);
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
index 1a41e08..5005858 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
@@ -31,6 +31,7 @@ import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
@@ -47,10 +48,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
   protected ClusterControllerManager _controller;
 
-  protected ClusterSetup _setupTool = null;
   List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
   Map<String, String> _nodeToTagMap = new HashMap<String, String>();
-  List<String> _nodes = new ArrayList<String>();
+  List<String> _nodes = new ArrayList<>();
   Set<String> _allDBs = new HashSet<>();
   int _replica = 3;
 
@@ -63,12 +63,7 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
     ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
@@ -78,10 +73,10 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       _nodes.add(storageNodeName);
       String tag = "tag-" + i % 2;
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
       _nodeToTagMap.put(storageNodeName, tag);
       InstanceConfig instanceConfig =
           configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
@@ -105,6 +100,18 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     //enableTopologyAwareRebalance(_gZkClient, CLUSTER_NAME, true);
   }
 
+  @AfterClass
+  public void afterClass() throws Exception {
+    _controller.syncStop();
+    for (MockParticipantManager p : _participants) {
+      if (p.isConnected()) {
+        p.syncStop();
+      }
+    }
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+    super.afterClass();
+  }
+
   @DataProvider(name = "rebalanceStrategies")
   public static String[][] rebalanceStrategies() {
     return new String[][] { { "CrushRebalanceStrategy", CrushRebalanceStrategy.class.getName() },
@@ -118,9 +125,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -130,9 +137,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, _replica);
     }
   }
@@ -144,13 +151,13 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     int i = 3;
     for (String tag : tags) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS,
           BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO + "",
           rebalanceStrategyClass);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       is.setInstanceGroupTag(tag);
-      _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -160,9 +167,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, _replica);
     }
   }
@@ -183,9 +190,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     int j = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + j++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -195,9 +202,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, 2);
     }
 
@@ -218,18 +225,18 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     for (int i = 2; i < _participants.size(); i++) {
       MockParticipantManager p = _participants.get(i);
       p.syncStop();
-      _setupTool.getClusterManagementTool()
+      _gSetupTool.getClusterManagementTool()
           .enableInstance(CLUSTER_NAME, p.getInstanceName(), false);
       Thread.sleep(200);
-      _setupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
+      _gSetupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
     }
 
     int j = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + j++;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
@@ -238,9 +245,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _allDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, 2);
     }
 
@@ -248,7 +255,7 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
     for (int i = 2; i < _participants.size(); i++) {
       String storageNodeName = _participants.get(i).getInstanceName();
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
       InstanceConfig instanceConfig =
           configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
@@ -274,7 +281,7 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
       for (String instance : instancesInEV) {
         if (tag != null) {
           InstanceConfig config =
-              _setupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
+              _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
           Assert.assertTrue(config.containsTag(tag));
         }
       }
@@ -284,7 +291,7 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
   @AfterMethod
   public void afterMethod() throws Exception {
     for (String db : _allDBs) {
-      _setupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
     }
     _allDBs.clear();
     // waiting for all DB be dropped.

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceTopoplogyAwareDisabled.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceTopoplogyAwareDisabled.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceTopoplogyAwareDisabled.java
index c7d88e9..2b89164 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceTopoplogyAwareDisabled.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceTopoplogyAwareDisabled.java
@@ -20,10 +20,9 @@ package org.apache.helix.integration.rebalancer.CrushRebalancers;
  */
 
 import java.util.Date;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.tools.ClusterSetup;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
@@ -33,36 +32,31 @@ public class TestCrushAutoRebalanceTopoplogyAwareDisabled extends TestCrushAutoR
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (ZkIntegrationTestBase._gZkClient.exists(namespace)) {
-      ZkIntegrationTestBase._gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZkIntegrationTestBase._gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (TestCrushAutoRebalanceNonRack.START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       _nodes.add(storageNodeName);
       String tag = "tag-" + i % 2;
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
       _nodeToTagMap.put(storageNodeName, tag);
     }
 
     // start dummy participants
     for (String node : _nodes) {
       MockParticipantManager participant =
-          new MockParticipantManager(ZkIntegrationTestBase.ZK_ADDR, CLUSTER_NAME, node);
+          new MockParticipantManager(ZkTestBase.ZK_ADDR, CLUSTER_NAME, node);
       participant.syncStart();
       _participants.add(participant);
     }
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZkIntegrationTestBase.ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller = new ClusterControllerManager(ZkTestBase.ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    enablePersistBestPossibleAssignment(ZkIntegrationTestBase._gZkClient, CLUSTER_NAME, true);
+    enablePersistBestPossibleAssignment(ZkTestBase._gZkClient, CLUSTER_NAME, true);
   }
 
   @Test(dataProvider = "rebalanceStrategies")


[2/5] helix git commit: Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
index 6f6c0b8..92da126 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
@@ -28,16 +28,14 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
-import org.omg.PortableServer.THREAD_POLICY_ID;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
@@ -45,7 +43,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
+public class TestDelayedAutoRebalance extends ZkTestBase {
   final int NUM_NODE = 5;
   protected static final int START_PORT = 12918;
   protected static final int _PARTITIONS = 5;
@@ -54,7 +52,6 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
   protected ClusterControllerManager _controller;
 
-  protected ClusterSetup _setupTool = null;
   List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
   int _replica = 3;
   int _minActiveReplica = _replica - 1;
@@ -65,16 +62,11 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
       // start dummy participants
       MockParticipantManager participant =
@@ -136,8 +128,8 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     Thread.sleep(500);
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
     }
     setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, -1);
@@ -159,8 +151,8 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     // after delay time, it should maintain required number of replicas.
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _replica, NUM_NODE);
     }
   }
@@ -172,10 +164,10 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
 
     // disable delay rebalance for one db, partition should be moved immediately
     String testDb = _testDBs.get(0);
-    IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(
+    IdealState idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(
         CLUSTER_NAME, testDb);
     idealState.setDelayRebalanceEnabled(false);
-    _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
     Thread.sleep(1000);
 
 
@@ -183,9 +175,9 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     // replica for other dbs should not be moved.
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       IdealState is =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
 
       if (db.equals(testDb)) {
         validateMinActiveAndTopStateReplica(idealState, ev, _replica, NUM_NODE);
@@ -212,8 +204,8 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _replica, NUM_NODE);
     }
 
@@ -230,7 +222,7 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     Thread.sleep(1000);
 
     for (String db : _testDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       Map<String, List<String>> preferenceLists = is.getPreferenceLists();
       for (List<String> instances : preferenceLists.values()) {
         Assert.assertFalse(instances.contains(disabledInstanceName));
@@ -243,7 +235,7 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
   public void afterTest() throws InterruptedException {
     // delete all DBs create in last test
     for (String db : _testDBs) {
-      _setupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
     }
     _testDBs.clear();
     Thread.sleep(50);
@@ -275,7 +267,7 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       externalViews.put(db, ev);
     }
     return externalViews;
@@ -320,8 +312,8 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), false);
@@ -337,7 +329,7 @@ public class TestDelayedAutoRebalance extends ZkIntegrationTestBase {
     for (MockParticipantManager participant : _participants) {
       participant.syncStop();
     }
-    _setupTool.deleteCluster(CLUSTER_NAME);
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
index e36204b..0c2b24c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
@@ -61,8 +61,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev, instance, true);
     }
@@ -84,8 +84,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -111,8 +111,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -125,8 +125,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
     }
     setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, -1);
@@ -148,8 +148,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -162,8 +162,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
     }
     setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, -1);
@@ -186,8 +186,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -197,8 +197,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     // after delay time, it should maintain required number of replicas.
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _replica, NUM_NODE);
     }
   }
@@ -215,8 +215,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -224,10 +224,10 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable delay rebalance for one db, partition should be moved immediately
     String testDb = _testDBs.get(0);
-    IdealState idealState = _setupTool.getClusterManagementTool().getResourceIdealState(
+    IdealState idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(
         CLUSTER_NAME, testDb);
     idealState.setDelayRebalanceEnabled(false);
-    _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
     Thread.sleep(2000);
     Assert.assertTrue(_clusterVerifier.verify());
 
@@ -235,9 +235,9 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     // replica for other dbs should not be moved.
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       IdealState is =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
 
       if (db.equals(testDb)) {
         validateMinActiveAndTopStateReplica(idealState, ev, _replica, NUM_NODE);
@@ -263,8 +263,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _minActiveReplica, NUM_NODE);
       validateNoPartitionMove(is, externalViewsBefore.get(db), ev,
           _participants.get(0).getInstanceName(), true);
@@ -278,8 +278,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _testDBs) {
       ExternalView ev =
-          _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(
           CLUSTER_NAME, db);
       validateMinActiveAndTopStateReplica(is, ev, _replica, NUM_NODE);
     }
@@ -308,7 +308,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
   private void enableInstance(String instance, boolean enabled) {
     // Disable one node, no partition should be moved.
     long currentTime = System.currentTimeMillis();
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instance, enabled);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instance, enabled);
     InstanceConfig instanceConfig = _configAccessor.getInstanceConfig(CLUSTER_NAME, instance);
     Assert.assertEquals(instanceConfig.getInstanceEnabled(), enabled);
     Assert.assertTrue(instanceConfig.getInstanceEnabledTime() >= currentTime);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
index 116a756..2c096f5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
@@ -41,18 +41,13 @@ public class TestDelayedAutoRebalanceWithRackaware extends TestDelayedAutoRebala
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       String zone = "zone-" + i % 3;
-      _setupTool.getClusterManagementTool().setInstanceZoneId(CLUSTER_NAME, storageNodeName, zone);
+      _gSetupTool.getClusterManagementTool().setInstanceZoneId(CLUSTER_NAME, storageNodeName, zone);
 
       // start dummy participants
       MockParticipantManager participant =

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestExpandCluster.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestExpandCluster.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestExpandCluster.java
index 0e6c69b..134ab14 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestExpandCluster.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestExpandCluster.java
@@ -80,7 +80,7 @@ public class TestExpandCluster extends TestPartitionMigrationBase {
       config.getRecord().getSimpleFields()
           .remove(InstanceConfig.InstanceConfigProperty.HELIX_ENABLED_TIMESTAMP.name());
 
-      _setupTool.getClusterManagementTool().addInstance(CLUSTER_NAME, config);
+      _gSetupTool.getClusterManagementTool().addInstance(CLUSTER_NAME, config);
 
       // start dummy participants
       MockParticipantManager participant =
@@ -92,7 +92,7 @@ public class TestExpandCluster extends TestPartitionMigrationBase {
     // enable new instance one by one
     for (int i = numNodes; i < numNodes + NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, storageNodeName, true);
+      _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, storageNodeName, true);
       Thread.sleep(100);
     }
 
@@ -120,7 +120,7 @@ public class TestExpandCluster extends TestPartitionMigrationBase {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       MockParticipantManager participant = _participants.get(i);
       participant.syncStop();
-      _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, storageNodeName, false);
+      _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, storageNodeName, false);
       Assert.assertTrue(_clusterVerifier.verify());
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestFullAutoMigration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestFullAutoMigration.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestFullAutoMigration.java
index e93445d..8dfe195 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestFullAutoMigration.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestFullAutoMigration.java
@@ -139,10 +139,10 @@ public class TestFullAutoMigration extends TestPartitionMigrationBase {
           delayTime);
       _testDBs.add(db);
     }
-    Thread.sleep(800);
+    Thread.sleep(100);
     Assert.assertTrue(_clusterVerifier.verify());
     for (String db : _testDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       idealStateMap.put(db, is);
     }
     return idealStateMap;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
index 2b7996c..4e9638a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
@@ -34,21 +34,20 @@ import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
 import org.apache.helix.integration.DelayedTransitionBase;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 
 
-public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
+public class TestPartitionMigrationBase extends ZkTestBase {
   final int NUM_NODE = 6;
   protected static final int START_PORT = 12918;
   protected static final int _PARTITIONS = 50;
@@ -57,7 +56,6 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
   protected ClusterControllerManager _controller;
 
-  protected ClusterSetup _setupTool = null;
   List<MockParticipantManager> _participants = new ArrayList<>();
   int _replica = 3;
   int _minActiveReplica = _replica - 1;
@@ -73,12 +71,7 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
@@ -103,12 +96,12 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
   }
 
   protected MockParticipantManager createAndStartParticipant(String instancename) {
-    _setupTool.addInstanceToCluster(CLUSTER_NAME, instancename);
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instancename);
 
     // start dummy participants
     MockParticipantManager participant =
-        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instancename, 50);
-    participant.setTransition(new DelayedTransitionBase(50));
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instancename, 10);
+    participant.setTransition(new DelayedTransitionBase(10));
     participant.syncStart();
     return participant;
   }
@@ -129,7 +122,7 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
       _testDBs.add(db);
     }
     for (String db : _testDBs) {
-      IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       idealStateMap.put(db, is);
     }
     ClusterConfig clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
@@ -210,23 +203,23 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
     private void verifyPartitionCount(String resource, String partition,
         Map<String, String> stateMap, int replica, String warningPrefix, int minActiveReplica) {
       if (stateMap.size() < replica) {
-        System.out.println(
-            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
-                + " replicas in " + warningPrefix);
+//        System.out.println(
+//            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
+//                + " replicas in " + warningPrefix);
         _hasLessReplica = true;
       }
 
       if (stateMap.size() > replica + EXTRA_REPLICA) {
-        System.out.println(
-            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
-                + " replicas in " + warningPrefix);
-        _hasMoreReplica = true;
+//        System.out.println(
+//            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
+//                + " replicas in " + warningPrefix);
+//        _hasMoreReplica = true;
       }
 
       if (stateMap.size() < minActiveReplica) {
-        System.out.println(
-            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
-                + " min active replicas in " + warningPrefix);
+//        System.out.println(
+//            "resource " + resource + ", partition " + partition + " has " + stateMap.size()
+//                + " min active replicas in " + warningPrefix);
         _hasMinActiveReplica = true;
       }
     }
@@ -260,7 +253,7 @@ public class TestPartitionMigrationBase extends ZkIntegrationTestBase {
       participant.syncStop();
     }
     _manager.disconnect();
-    _setupTool.deleteCluster(CLUSTER_NAME);
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoIsWithEmptyMap.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoIsWithEmptyMap.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoIsWithEmptyMap.java
index 6a77336..0181139 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoIsWithEmptyMap.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoIsWithEmptyMap.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
@@ -35,7 +35,7 @@ import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestAutoIsWithEmptyMap extends ZkIntegrationTestBase {
+public class TestAutoIsWithEmptyMap extends ZkTestBase {
   @Test
   public void testAutoIsWithEmptyMap() throws Exception {
     String className = TestHelper.getTestClassName();
@@ -96,6 +96,7 @@ public class TestAutoIsWithEmptyMap extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalance.java
index 0619359..7c06e9d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalance.java
@@ -55,33 +55,27 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL,
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL,
         RebalanceMode.FULL_AUTO + "");
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, db2, _PARTITIONS, "OnlineOffline",
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, db2, _PARTITIONS, "OnlineOffline",
         RebalanceMode.FULL_AUTO + "");
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, _replica);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, _replica);
 
     for (int i = 0; i < 3; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, _tag);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, _tag);
     }
 
-    _setupTool.rebalanceCluster(CLUSTER_NAME, db2, 1, "ucpx", _tag);
+    _gSetupTool.rebalanceCluster(CLUSTER_NAME, db2, 1, "ucpx", _tag);
 
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
@@ -108,10 +102,10 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
   @Test()
   public void testDropResourceAutoRebalance() throws Exception {
     // add a resource to be dropped
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", _PARTITIONS, "OnlineOffline",
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", _PARTITIONS, "OnlineOffline",
         RebalanceMode.FULL_AUTO + "");
 
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 1);
 
     boolean result =
         ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
@@ -126,10 +120,10 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
             "localhost_12921", "localhost_12922"), ZK_ADDR);
 
     // add a resource to be dropped
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB2", _PARTITIONS, "MasterSlave",
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "MyDB2", _PARTITIONS, "MasterSlave",
         RebalanceMode.FULL_AUTO + "");
 
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 1);
 
     result =
         ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
@@ -157,7 +151,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
     // add 2 nodes
     for (int i = 0; i < 2; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
       MockParticipantManager participant =
           new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_'));

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalancePartitionLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalancePartitionLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalancePartitionLimit.java
index a53b6c6..ba30964 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalancePartitionLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalancePartitionLimit.java
@@ -54,22 +54,16 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, 100, "OnlineOffline",
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, 100, "OnlineOffline",
         RebalanceMode.FULL_AUTO + "", 0, 25);
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 1);
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
@@ -134,7 +128,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBase {
     // add 2 nodes
     for (int i = 0; i < 2; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
       String newInstanceName = storageNodeName.replace(':', '_');
       MockParticipantManager participant =

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalanceWithDisabledInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalanceWithDisabledInstance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalanceWithDisabledInstance.java
index a0a53b9..13e156f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalanceWithDisabledInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestAutoRebalanceWithDisabledInstance.java
@@ -41,9 +41,9 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
   @Override
   public void beforeClass() throws Exception {
     super.beforeClass();
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB_2, _PARTITIONS, STATE_MODEL,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB_2, _PARTITIONS, STATE_MODEL,
         RebalanceMode.FULL_AUTO + "");
-    _setupTool.rebalanceResource(CLUSTER_NAME, TEST_DB_2, _replica);
+    _gSetupTool.rebalanceResource(CLUSTER_NAME, TEST_DB_2, _replica);
 
     Thread.sleep(2000);
 
@@ -70,7 +70,7 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
     Assert.assertFalse(currentPartitions.isEmpty());
 
     // disable instance
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, disabledInstance, false);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, disabledInstance, false);
     Thread.sleep(4000);
 
     // TODO: preference list is not persisted in IS for full-auto,
@@ -84,7 +84,7 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
     Assert.assertTrue(currentPartitions.isEmpty());
 
     //enable instance
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, disabledInstance, true);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, disabledInstance, true);
     Thread.sleep(4000);
 
     // TODO: preference list is not persisted in IS for full-auto,
@@ -102,10 +102,10 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
   public void testAddDisabledInstanceAutoRebalance() throws Exception {
     // add disabled instance.
     String nodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + NODE_NR);
-    _setupTool.addInstanceToCluster(CLUSTER_NAME, nodeName);
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, nodeName);
     MockParticipantManager participant =
           new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, nodeName);
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, nodeName, false);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, nodeName, false);
 
     participant.syncStart();
 
@@ -121,7 +121,7 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
     Assert.assertTrue(currentPartitions.isEmpty());
 
     //enable instance
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, nodeName, true);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, nodeName, true);
     Thread.sleep(2000);
     // TODO: preference list is not persisted in IS for full-auto,
     // Need a way to find how helix assigns partitions to nodes.
@@ -135,7 +135,7 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
   }
 
   private Set<String> getPartitionsAssignedtoInstance(String cluster, String dbName, String instance) {
-    HelixAdmin admin = _setupTool.getClusterManagementTool();
+    HelixAdmin admin = _gSetupTool.getClusterManagementTool();
     Set<String> partitionSet = new HashSet<String>();
     IdealState is = admin.getResourceIdealState(cluster, dbName);
     for (String partition : is.getRecord().getListFields().keySet()) {
@@ -151,7 +151,7 @@ public class TestAutoRebalanceWithDisabledInstance extends ZkStandAloneCMTestBas
   }
 
   private Set<String> getCurrentPartitionsOnInstance(String cluster, String dbName, String instance) {
-    HelixAdmin admin = _setupTool.getClusterManagementTool();
+    HelixAdmin admin = _gSetupTool.getClusterManagementTool();
     Set<String> partitionSet = new HashSet<String>();
 
     ExternalView ev = admin.getResourceExternalView(cluster, dbName);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
index 9769021..c7ef8fa 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
@@ -24,14 +24,13 @@ import java.util.Date;
 import java.util.List;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.MaintenanceSignal;
-import org.apache.helix.model.PauseSignal;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
@@ -39,7 +38,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestClusterInMaintenanceModeWhenReachingMaxPartition extends ZkIntegrationTestBase {
+public class TestClusterInMaintenanceModeWhenReachingMaxPartition extends ZkTestBase {
   final int NUM_NODE = 5;
   protected static final int START_PORT = 12918;
   protected static final int _PARTITIONS = 5;
@@ -58,10 +57,6 @@ public class TestClusterInMaintenanceModeWhenReachingMaxPartition extends ZkInte
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
     _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
index eaac6bc..f63769b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
@@ -26,8 +26,7 @@ import java.util.List;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -48,10 +47,9 @@ import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
 import static org.apache.helix.monitoring.mbeans.ClusterStatusMonitor.CLUSTER_DN_KEY;
-import static org.apache.helix.util.StatusUpdateUtil.ErrorType.RebalanceResourceFailure;
 
 public class TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit
-    extends ZkIntegrationTestBase {
+    extends ZkTestBase {
   static final int NUM_NODE = 10;
   static final int START_PORT = 12918;
   static final int _PARTITIONS = 5;
@@ -70,10 +68,6 @@ public class TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
     _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomIdealState.java
index f806f16..b971e38 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomIdealState.java
@@ -22,13 +22,13 @@ package org.apache.helix.integration.rebalancer;
 import java.util.Date;
 import org.apache.helix.TestHelper;
 import org.apache.helix.integration.TestDriver;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.tools.ClusterSetup;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.annotations.Test;
 
-public class TestCustomIdealState extends ZkIntegrationTestBase {
+public class TestCustomIdealState extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestCustomIdealState.class);
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomizedIdealStateRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomizedIdealStateRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomizedIdealStateRebalancer.java
index 9098e8e..ba6db12 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomizedIdealStateRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomizedIdealStateRebalancer.java
@@ -77,14 +77,14 @@ public class TestCustomizedIdealStateRebalancer extends ZkStandAloneCMTestBase {
 
   @Test
   public void testCustomizedIdealStateRebalancer() throws InterruptedException {
-    _setupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "MasterSlave");
-    _setupTool.addResourceProperty(CLUSTER_NAME, db2,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "MasterSlave");
+    _gSetupTool.addResourceProperty(CLUSTER_NAME, db2,
         IdealStateProperty.REBALANCER_CLASS_NAME.toString(),
         TestCustomizedIdealStateRebalancer.TestRebalancer.class.getName());
-    _setupTool.addResourceProperty(CLUSTER_NAME, db2, IdealStateProperty.REBALANCE_MODE.toString(),
+    _gSetupTool.addResourceProperty(CLUSTER_NAME, db2, IdealStateProperty.REBALANCE_MODE.toString(),
         RebalanceMode.USER_DEFINED.toString());
 
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 3);
 
     boolean result =
         ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestFullAutoNodeTagging.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestFullAutoNodeTagging.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestFullAutoNodeTagging.java
index e48b3fd..b966f86 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestFullAutoNodeTagging.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestFullAutoNodeTagging.java
@@ -160,6 +160,13 @@ public class TestFullAutoNodeTagging extends ZkUnitTestBase {
     boolean finalResult = TestHelper.verify(v, 10 * 1000);
     Assert.assertTrue(finalResult);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < NUM_PARTICIPANTS; i++) {
+      participants[i].syncStop();
+    }
+    TestHelper.dropCluster(clusterName, _gZkClient);
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -220,6 +227,7 @@ public class TestFullAutoNodeTagging extends ZkUnitTestBase {
       participants[i].syncStop();
     }
     controller.syncStop();
+    TestHelper.dropCluster(clusterName, _gZkClient);
   }
 
   /**
@@ -312,6 +320,7 @@ public class TestFullAutoNodeTagging extends ZkUnitTestBase {
       }
     }
     controller.syncStop();
+    TestHelper.dropCluster(clusterName, _gZkClient);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
index 156bad5..28e112e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
@@ -31,7 +31,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -53,7 +53,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
-public class TestMixedModeAutoRebalance extends ZkIntegrationTestBase {
+public class TestMixedModeAutoRebalance extends ZkTestBase {
   private final int NUM_NODE = 5;
   private static final int START_PORT = 12918;
   private static final int _PARTITIONS = 5;
@@ -72,10 +72,6 @@ public class TestMixedModeAutoRebalance extends ZkIntegrationTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
     _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestSemiAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestSemiAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestSemiAutoRebalance.java
index 3eb0e57..b979bbe 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestSemiAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestSemiAutoRebalance.java
@@ -26,7 +26,7 @@ import java.util.List;
 import java.util.Map;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -34,10 +34,11 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MasterSlaveSMD;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestSemiAutoRebalance extends ZkIntegrationTestBase {
+public class TestSemiAutoRebalance extends ZkTestBase {
   protected final String CLASS_NAME = getShortClassName();
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
@@ -49,7 +50,7 @@ public class TestSemiAutoRebalance extends ZkIntegrationTestBase {
   protected static final int REPLICA_NUMBER = 3;
   protected static final String STATE_MODEL = "MasterSlave";
 
-  protected List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  protected List<MockParticipantManager> _participants = new ArrayList<>();
   protected ClusterControllerManager _controller;
 
   protected HelixDataAccessor _accessor;
@@ -61,11 +62,6 @@ public class TestSemiAutoRebalance extends ZkIntegrationTestBase {
     System.out.println(
         "START " + getShortClassName() + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
     // setup storage cluster
     _gSetupTool.addCluster(CLUSTER_NAME, true);
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, DB_NAME, PARTITION_NUMBER, STATE_MODEL,
@@ -125,6 +121,18 @@ public class TestSemiAutoRebalance extends ZkIntegrationTestBase {
     }
   }
 
+  @AfterClass
+  public void afterClass() throws Exception {
+    _controller.syncStop();
+    for (MockParticipantManager p : _participants) {
+      if (p.isConnected()) {
+        p.syncStop();
+      }
+    }
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+  }
+
   @Test
   public void testAddParticipant()
       throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
index e1bfa82..267b88b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
@@ -29,7 +29,7 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
@@ -38,21 +38,19 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestZeroReplicaAvoidance extends ZkIntegrationTestBase implements
+public class TestZeroReplicaAvoidance extends ZkTestBase implements
     ExternalViewChangeListener, IdealStateChangeListener{
   final int NUM_NODE = 6;
   final int START_PORT = 12918;
   final String CLASS_NAME = getShortClassName();
   final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
-  ClusterSetup _setupTool = null;
   List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
   HelixClusterVerifier _clusterVerifier;
   boolean _testSuccess = true;
@@ -62,16 +60,11 @@ public class TestZeroReplicaAvoidance extends ZkIntegrationTestBase implements
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
       MockParticipantManager participant =
           new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName);
       participant.setTransition(new DelayedTransition());
@@ -185,7 +178,7 @@ public class TestZeroReplicaAvoidance extends ZkIntegrationTestBase implements
       return;
     }
     for (ExternalView view : externalViewList) {
-      IdealState is = _setupTool.getClusterManagementTool()
+      IdealState is = _gSetupTool.getClusterManagementTool()
           .getResourceIdealState(CLUSTER_NAME, view.getResourceName());
       validateNoZeroReplica(is, view);
     }
@@ -197,7 +190,7 @@ public class TestZeroReplicaAvoidance extends ZkIntegrationTestBase implements
       return;
     }
     for (IdealState is : idealStates) {
-      ExternalView view = _setupTool.getClusterManagementTool()
+      ExternalView view = _gSetupTool.getClusterManagementTool()
           .getResourceExternalView(CLUSTER_NAME, is.getResourceName());
       validateNoZeroReplica(is, view);
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
index e395519..0c1d0da 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProvider.java
@@ -12,7 +12,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.api.listeners.RoutingTableChangeListener;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
@@ -28,7 +28,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestRoutingTableProvider extends ZkIntegrationTestBase {
+public class TestRoutingTableProvider extends ZkTestBase {
 
   static final String STATE_MODEL = BuiltInStateModelDefinitions.MasterSlave.name();
   static final String TEST_DB = "TestDB";

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromCurrentStates.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromCurrentStates.java b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromCurrentStates.java
index 72a6dae..99e4ce6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromCurrentStates.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromCurrentStates.java
@@ -9,14 +9,13 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.spectator.RoutingTableProvider;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
@@ -24,9 +23,8 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestRoutingTableProviderFromCurrentStates extends ZkIntegrationTestBase {
+public class TestRoutingTableProviderFromCurrentStates extends ZkTestBase {
   private HelixManager _manager;
-  private ClusterSetup _setupTool;
   private final int NUM_NODES = 10;
   protected int NUM_PARTITIONS = 20;
   protected int NUM_REPLICAS = 3;
@@ -37,19 +35,11 @@ public class TestRoutingTableProviderFromCurrentStates extends ZkIntegrationTest
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
-    _participants =  new MockParticipantManager[NUM_NODES];
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
-
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     _participants = new MockParticipantManager[NUM_NODES];
     for (int i = 0; i < NUM_NODES; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     for (int i = 0; i < NUM_NODES; i++) {
@@ -74,12 +64,25 @@ public class TestRoutingTableProviderFromCurrentStates extends ZkIntegrationTest
 
   @AfterClass
   public void afterClass() throws Exception {
-    _manager.disconnect();
+    /**
+     * shutdown order: 1) disconnect the controller 2) disconnect participants
+     */
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
     for (int i = 0; i < NUM_NODES; i++) {
       if (_participants[i] != null && _participants[i].isConnected()) {
-        _participants[i].reset();
+        _participants[i].syncStop();
       }
     }
+    if (_manager != null && _manager.isConnected()) {
+      _manager.disconnect();
+    }
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gSetupTool.deleteCluster(CLUSTER_NAME);
+    }
   }
 
   @Test
@@ -90,8 +93,8 @@ public class TestRoutingTableProviderFromCurrentStates extends ZkIntegrationTest
 
     try {
       String db1 = "TestDB-1";
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db1, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db1, NUM_REPLICAS);
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db1, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db1, NUM_REPLICAS);
 
       Thread.sleep(200);
       HelixClusterVerifier clusterVerifier =
@@ -99,19 +102,19 @@ public class TestRoutingTableProviderFromCurrentStates extends ZkIntegrationTest
       Assert.assertTrue(clusterVerifier.verify());
 
       IdealState idealState1 =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db1);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db1);
       validate(idealState1, routingTableEV, routingTableCurrentStates);
 
       // add new DB
       String db2 = "TestDB-2";
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db2, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, NUM_REPLICAS);
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db2, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, NUM_REPLICAS);
 
       Thread.sleep(200);
       Assert.assertTrue(clusterVerifier.verify());
 
       IdealState idealState2 =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db2);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db2);
       validate(idealState2, routingTableEV, routingTableCurrentStates);
 
       // shutdown an instance

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromTargetEV.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromTargetEV.java b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromTargetEV.java
index 16a759e..3d23e30 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromTargetEV.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderFromTargetEV.java
@@ -9,7 +9,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.integration.task.WorkflowGenerator;
@@ -19,15 +19,13 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.spectator.RoutingTableProvider;
-import org.apache.helix.tools.ClusterSetup;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestRoutingTableProviderFromTargetEV extends ZkIntegrationTestBase {
+public class TestRoutingTableProviderFromTargetEV extends ZkTestBase {
   private HelixManager _manager;
-  private ClusterSetup _setupTool;
   private final String MASTER_SLAVE_STATE_MODEL = "MasterSlave";
   private final int NUM_NODES = 10;
   protected int NUM_PARTITIONS = 20;
@@ -40,25 +38,19 @@ public class TestRoutingTableProviderFromTargetEV extends ZkIntegrationTestBase
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
     _participants =  new MockParticipantManager[NUM_NODES];
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     _participants = new MockParticipantManager[NUM_NODES];
     for (int i = 0; i < NUM_NODES; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_PARTITIONS,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_PARTITIONS,
         MASTER_SLAVE_STATE_MODEL, IdealState.RebalanceMode.FULL_AUTO.name());
 
-    _setupTool
+    _gSetupTool
         .rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, NUM_REPLICAS);
 
     for (int i = 0; i < NUM_NODES; i++) {
@@ -85,12 +77,22 @@ public class TestRoutingTableProviderFromTargetEV extends ZkIntegrationTestBase
 
   @AfterClass
   public void afterClass() throws Exception {
-    _manager.disconnect();
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
     for (int i = 0; i < NUM_NODES; i++) {
       if (_participants[i] != null && _participants[i].isConnected()) {
-        _participants[i].reset();
+        _participants[i].syncStop();
       }
     }
+    if (_manager != null && _manager.isConnected()) {
+      _manager.disconnect();
+    }
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gSetupTool.deleteCluster(CLUSTER_NAME);
+    }
   }
 
   @Test (expectedExceptions = HelixException.class)
@@ -135,7 +137,7 @@ public class TestRoutingTableProviderFromTargetEV extends ZkIntegrationTestBase
       Assert.assertEquals(targetExternalViewMasters.size(), NUM_NODES);
 
       // TargetExternalView MASTERS mapping should exactly match IdealState MASTERS mapping
-      Map<String, Map<String, String>> stateMap = _setupTool.getClusterManagementTool()
+      Map<String, Map<String, String>> stateMap = _gSetupTool.getClusterManagementTool()
           .getResourceIdealState(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB).getRecord().getMapFields();
 
       Set<String> idealMasters = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderPeriodicRefresh.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderPeriodicRefresh.java b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderPeriodicRefresh.java
index dac7617..0a922cf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderPeriodicRefresh.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableProviderPeriodicRefresh.java
@@ -10,7 +10,7 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
@@ -27,7 +27,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestRoutingTableProviderPeriodicRefresh extends ZkIntegrationTestBase {
+public class TestRoutingTableProviderPeriodicRefresh extends ZkTestBase {
   private static final org.slf4j.Logger logger = LoggerFactory.getLogger(TestRoutingTableProviderPeriodicRefresh.class);
 
   private static final String STATE_MODEL = BuiltInStateModelDefinitions.MasterSlave.name();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableSnapshot.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableSnapshot.java b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableSnapshot.java
index 118387c..b451e4a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableSnapshot.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/spectator/TestRoutingTableSnapshot.java
@@ -6,14 +6,13 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyType;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.spectator.RoutingTableProvider;
 import org.apache.helix.spectator.RoutingTableSnapshot;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
@@ -22,9 +21,8 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 
-public class TestRoutingTableSnapshot extends ZkIntegrationTestBase {
+public class TestRoutingTableSnapshot extends ZkTestBase {
   private HelixManager _manager;
-  private ClusterSetup _setupTool;
   private final int NUM_NODES = 10;
   protected int NUM_PARTITIONS = 20;
   protected int NUM_REPLICAS = 3;
@@ -35,19 +33,13 @@ public class TestRoutingTableSnapshot extends ZkIntegrationTestBase {
 
   @BeforeClass
   public void beforeClass() throws Exception {
-    String namespace = "/" + CLUSTER_NAME;
     _participants =  new MockParticipantManager[NUM_NODES];
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     _participants = new MockParticipantManager[NUM_NODES];
     for (int i = 0; i < NUM_NODES; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     for (int i = 0; i < NUM_NODES; i++) {
@@ -82,8 +74,8 @@ public class TestRoutingTableSnapshot extends ZkIntegrationTestBase {
 
     try {
       String db1 = "TestDB-1";
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db1, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db1, NUM_REPLICAS);
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db1, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db1, NUM_REPLICAS);
 
       Thread.sleep(200);
       HelixClusterVerifier clusterVerifier =
@@ -91,7 +83,7 @@ public class TestRoutingTableSnapshot extends ZkIntegrationTestBase {
       Assert.assertTrue(clusterVerifier.verify());
 
       IdealState idealState1 =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db1);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db1);
 
       RoutingTableSnapshot routingTableSnapshot = routingTableProvider.getRoutingTableSnapshot();
       validateMapping(idealState1, routingTableSnapshot);
@@ -102,8 +94,8 @@ public class TestRoutingTableSnapshot extends ZkIntegrationTestBase {
 
       // add new DB and shutdown an instance
       String db2 = "TestDB-2";
-      _setupTool.addResourceToCluster(CLUSTER_NAME, db2, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, NUM_REPLICAS);
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, db2, NUM_PARTITIONS, "MasterSlave", IdealState.RebalanceMode.FULL_AUTO.name());
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, NUM_REPLICAS);
 
       // shutdown an instance
       _participants[0].syncStop();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
index d1f0891..ab29607 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TaskTestBase.java
@@ -28,7 +28,6 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 
 public class TaskTestBase extends TaskSynchronizedTestBase {
-  protected ClusterControllerManager _controller;
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -39,14 +38,6 @@ public class TaskTestBase extends TaskSynchronizedTestBase {
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    boolean result = ClusterStateVerifier.verifyByZkCallback(
-        new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME));
-    Assert.assertTrue(result);
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    super.afterClass();
-    _controller.syncStop();
+    Assert.assertTrue(_clusterVerifier.verify());
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestBatchAddJobs.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestBatchAddJobs.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestBatchAddJobs.java
index d50845e..36bbfa7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestBatchAddJobs.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestBatchAddJobs.java
@@ -2,15 +2,12 @@ package org.apache.helix.integration.task;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
-import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobDag;
 import org.apache.helix.task.JobQueue;
@@ -23,7 +20,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestBatchAddJobs extends ZkIntegrationTestBase {
+public class TestBatchAddJobs extends ZkTestBase {
   private static final String CLUSTER_NAME = CLUSTER_PREFIX + "_TestBatchAddJobs";
   private static final String QUEUE_NAME = "TestBatchAddJobQueue";
   private ClusterSetup _setupTool;
@@ -31,15 +28,9 @@ public class TestBatchAddJobs extends ZkIntegrationTestBase {
 
   @BeforeClass
   public void beforeClass() {
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
     _setupTool = new ClusterSetup(ZK_ADDR);
     _setupTool.addCluster(CLUSTER_NAME, true);
     _submitJobTasks = new ArrayList<>();
-
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
index 28d182b..5540bbf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestIndependentTaskRebalancer.java
@@ -64,17 +64,11 @@ public class TestIndependentTaskRebalancer extends TaskTestBase {
   @BeforeClass
   public void beforeClass() throws Exception {
     _participants = new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
 
-    // Setup cluster and instances
-    ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
-    setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     for (int i = 0; i < _numNodes; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
-      setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // start dummy participants

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailure.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailure.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailure.java
index 5eb462b..a2608c5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailure.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestJobFailure.java
@@ -46,8 +46,6 @@ import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
 public final class TestJobFailure extends TaskSynchronizedTestBase {
-
-  private ClusterControllerManager _controller;
   private final String DB_NAME = WorkflowGenerator.DEFAULT_TGT_DB;
 
   @BeforeClass
@@ -58,13 +56,7 @@ public final class TestJobFailure extends TaskSynchronizedTestBase {
     _numReplicas = 1; // only Master, no Slave
     _numDbs = 1;
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
     startParticipants();


[4/5] helix git commit: Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java b/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
index 2d8749e..a5f7d50 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
@@ -39,7 +39,9 @@ import org.apache.helix.spectator.RoutingTableProvider;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
@@ -73,6 +75,11 @@ public class TestCorrectnessOnConnectivityLoss {
     _controller.connect();
   }
 
+  @AfterMethod
+  public void afterMethod() throws Exception {
+    TestHelper.stopZkServer(_zkServer);
+  }
+
   @Test
   public void testParticipant() throws Exception {
     Map<String, Integer> stateReachedCounts = Maps.newHashMap();
@@ -138,11 +145,6 @@ public class TestCorrectnessOnConnectivityLoss {
     }
   }
 
-  @AfterMethod
-  public void afterMethod() throws Exception {
-    TestHelper.stopZkServer(_zkServer);
-  }
-
   @StateModelInfo(initialState = "OFFLINE", states = {
       "MASTER", "SLAVE", "OFFLINE", "ERROR"
   })

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
index d512e9e..faf212b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
@@ -27,7 +27,7 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -42,7 +42,7 @@ import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestDisable extends ZkIntegrationTestBase {
+public class TestDisable extends ZkTestBase {
 
   @Test
   public void testDisableNodeCustomIS() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
index f4deef9..653dd82 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
@@ -23,7 +23,7 @@ import java.util.Date;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -42,7 +42,7 @@ import org.testng.annotations.Test;
  * if DISABLE_EXTERNAL_VIEW is set to true in a resource's idealstate,
  * there should be no external view for this resource.
  */
-public class TestDisableExternalView extends ZkIntegrationTestBase {
+public class TestDisableExternalView extends ZkTestBase {
   private static final String TEST_DB1 = "test_db1";
   private static final String TEST_DB2 = "test_db2";
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
index 116aed5..c9e5646 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
@@ -153,6 +153,7 @@ public class TestDisableResource extends ZkUnitTestBase {
     for (int i = 0; i < N; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -219,6 +220,7 @@ public class TestDisableResource extends ZkUnitTestBase {
     for (int i = 0; i < N; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
index 515c640..82f38d9 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -36,7 +36,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestDistributedCMMain extends ZkIntegrationTestBase {
+public class TestDistributedCMMain extends ZkTestBase {
 
   @Test
   public void testDistributedCMMain() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
index 3095a2b..777ba67 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -35,7 +35,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestDistributedClusterController extends ZkIntegrationTestBase {
+public class TestDistributedClusterController extends ZkTestBase {
 
   @Test
   public void testDistributedClusterController() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java b/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java
index 73af2bb..cf20def 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDriver.java
@@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
@@ -39,7 +39,6 @@ import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.store.PropertyJsonSerializer;
 import org.apache.helix.store.PropertyStoreException;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.apache.helix.tools.DefaultIdealStateCalculator;
@@ -55,7 +54,7 @@ import org.testng.Assert;
 
 public class TestDriver {
   private static Logger LOG = LoggerFactory.getLogger(TestDriver.class);
-  private static final String ZK_ADDR = ZkIntegrationTestBase.ZK_ADDR;
+  private static final String ZK_ADDR = ZkTestBase.ZK_ADDR;
 
   // private static final String CLUSTER_PREFIX = "TestDriver";
   private static final String STATE_MODEL = "MasterSlave";

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
index 3142b80..511888c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
@@ -31,7 +31,7 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -48,7 +48,7 @@ import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestDrop extends ZkIntegrationTestBase {
+public class TestDrop extends ZkTestBase {
 
   /**
    * Assert externalView and currentState for each participant are empty

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java
index 7f0b77c..1fb6785 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDropResource.java
@@ -31,8 +31,8 @@ public class TestDropResource extends ZkStandAloneCMTestBase {
   @Test()
   public void testDropResource() throws Exception {
     // add a resource to be dropped
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 6, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 6, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 3);
 
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -50,8 +50,8 @@ public class TestDropResource extends ZkStandAloneCMTestBase {
   @Test()
   public void testDropResourceWhileNodeDead() throws Exception {
     // add a resource to be dropped
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB2", 16, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "MyDB2", 16, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 3);
 
     boolean verifyResult =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java b/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
index 6bd950c..e6e02fa 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
@@ -8,7 +8,7 @@ import java.util.concurrent.TimeUnit;
 import org.I0Itec.zkclient.serialize.BytesPushThroughSerializer;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZkClient;
@@ -43,7 +43,7 @@ import org.testng.annotations.Test;
  * Compression can be enabled for a specific resource by setting enableCompression=true in the
  * idealstate of the resource. Generally this is used when the number of partitions is large
  */
-public class TestEnableCompression extends ZkIntegrationTestBase {
+public class TestEnableCompression extends ZkTestBase {
   @Test()
   public void testEnableCompressionResource() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
index 7c8e9e8..21f2035 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
+public class TestEnablePartitionDuringDisable extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestEnablePartitionDuringDisable.class);
 
   static {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java b/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
index dc38369..a879320 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
@@ -128,6 +128,7 @@ public class TestEntropyFreeNodeBounce extends ZkUnitTestBase {
       for (HelixManager participant : participants) {
         participant.disconnect();
       }
+      TestHelper.dropCluster(clusterName, _gZkClient);
       System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
index deef1bc..9ded9b1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
@@ -26,7 +26,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -35,7 +35,7 @@ import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestErrorPartition extends ZkIntegrationTestBase {
+public class TestErrorPartition extends ZkTestBase {
   @Test()
   public void testErrorPartition() throws Exception {
     String clusterName = getShortClassName();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java b/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java
index 2bf2b85..e3e018d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestExpandCluster.java
@@ -33,35 +33,35 @@ public class TestExpandCluster extends ZkStandAloneCMTestBase {
   @Test
   public void testExpandCluster() throws Exception {
     String DB2 = "TestDB2";
-    int partitions = 100;
+    int partitions = 30;
     int replica = 3;
-    _setupTool.addResourceToCluster(CLUSTER_NAME, DB2, partitions, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, DB2, replica, "keyX");
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, DB2, partitions, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, DB2, replica, "keyX");
 
     String DB3 = "TestDB3";
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, DB3, partitions, STATE_MODEL);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, DB3, partitions, STATE_MODEL);
 
     IdealState testDB0 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     IdealState testDB2 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB2);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB2);
     IdealState testDB3 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB3);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB3);
 
     for (int i = 0; i < 5; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (27960 + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
     String command = "-zkSvr localhost:2183 -expandCluster " + CLUSTER_NAME;
     ClusterSetup.processCommandLineArgs(command.split(" "));
 
     IdealState testDB0_1 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     IdealState testDB2_1 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB2);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB2);
     IdealState testDB3_1 =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB3);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB3);
 
     Map<String, Object> resultOld2 = RebalanceUtil.buildInternalIdealState(testDB2);
     Map<String, Object> result2 = RebalanceUtil.buildInternalIdealState(testDB2_1);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
index beff46d..de90677 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
@@ -26,7 +26,7 @@ import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -37,7 +37,7 @@ import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestExternalViewUpdates extends ZkIntegrationTestBase {
+public class TestExternalViewUpdates extends ZkTestBase {
   @Test
   public void testExternalViewUpdates() throws Exception {
     System.out.println("START testExternalViewUpdates at " + new Date(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
index 5b331c5..3ff0086 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
@@ -29,7 +29,7 @@ import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
+public class TestHelixCustomCodeRunner extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestHelixCustomCodeRunner.class);
 
   private final String _clusterName = "CLUSTER_" + getShortClassName();
@@ -133,6 +133,8 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
     for (int i = 0; i < _nodeNb; i++) {
       participants[i].syncStop();
     }
+    accessor.removeProperty(keyBuilder.liveInstance("newLiveInstance"));
+    TestHelper.dropCluster(_clusterName, _gZkClient);
 
     System.out.println("END " + _clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
index 211cca1..3c8d197 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
@@ -43,10 +43,10 @@ public class TestHelixInstanceTag extends ZkStandAloneCMTestBase {
     int replica = 2;
     for (int i = 0; i < 2; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, instanceName, DB2tag);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, instanceName, DB2tag);
     }
-    _setupTool.addResourceToCluster(CLUSTER_NAME, DB2, partitions, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, DB2, DB2tag, replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, DB2, partitions, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, DB2, DB2tag, replica);
 
     boolean result =
         ClusterStateVerifier
@@ -72,10 +72,10 @@ public class TestHelixInstanceTag extends ZkStandAloneCMTestBase {
     replica = 3;
     for (int i = 1; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, instanceName, DB3Tag);
+      _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, instanceName, DB3Tag);
     }
-    _setupTool.addResourceToCluster(CLUSTER_NAME, DB3, partitions, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, DB3, DB3Tag, replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, DB3, partitions, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, DB3, DB3Tag, replica);
 
     result =
         ClusterStateVerifier

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestHelixUsingDifferentParams.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixUsingDifferentParams.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixUsingDifferentParams.java
index 6f3d832..4350faf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixUsingDifferentParams.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixUsingDifferentParams.java
@@ -21,12 +21,12 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.annotations.Test;
 
-public class TestHelixUsingDifferentParams extends ZkIntegrationTestBase {
+public class TestHelixUsingDifferentParams extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestHelixUsingDifferentParams.class);
 
   @Test()

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
index 7b56f5d..cb2c1c2 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
@@ -97,6 +97,7 @@ public class TestInvalidResourceRebalance extends ZkUnitTestBase {
       participants[i].syncStop();
     }
     controller.syncStop();
+    TestHelper.dropCluster(clusterName, _gZkClient);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java b/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java
index bef21a7..37b8b82 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestNullReplica.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
@@ -33,7 +33,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestNullReplica extends ZkIntegrationTestBase {
+public class TestNullReplica extends ZkTestBase {
 
   @Test
   public void testNullReplica() throws Exception {
@@ -82,6 +82,7 @@ public class TestNullReplica extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
index 0f2fe6d..3ec3279 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
@@ -30,7 +30,7 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -50,7 +50,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestPartitionLevelTransitionConstraint extends ZkIntegrationTestBase {
+public class TestPartitionLevelTransitionConstraint extends ZkTestBase {
 
   private static Logger LOG = LoggerFactory.getLogger(TestPartitionLevelTransitionConstraint.class);
 
@@ -197,6 +197,7 @@ public class TestPartitionLevelTransitionConstraint extends ZkIntegrationTestBas
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionMovementThrottle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionMovementThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionMovementThrottle.java
index 6c9c2e2..ca2d7dd 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionMovementThrottle.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionMovementThrottle.java
@@ -43,7 +43,6 @@ import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
@@ -60,18 +59,12 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(_gZkClient);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // add dummy participants
@@ -117,8 +110,9 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
         StateTransitionThrottleConfig.ThrottleScope.CLUSTER, 100);
 
     clusterConfig
-        .setStateTransitionThrottleConfigs(Arrays.asList(resourceLoadThrottle, instanceLoadThrottle,
-            clusterLoadThrottle, resourceRecoveryThrottle, clusterRecoveryThrottle));
+        .setStateTransitionThrottleConfigs(Arrays
+            .asList(resourceLoadThrottle, instanceLoadThrottle, clusterLoadThrottle,
+                resourceRecoveryThrottle, clusterRecoveryThrottle));
 
     clusterConfig.setPersistIntermediateAssignment(true);
     _configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
@@ -133,9 +127,9 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
 
     for (int i = 0; i < 5; i++) {
       String dbName = "TestDB-" + i;
-      _setupTool.addResourceToCluster(CLUSTER_NAME, dbName, 10, STATE_MODEL,
+      _gSetupTool.addResourceToCluster(CLUSTER_NAME, dbName, 10, STATE_MODEL,
           RebalanceMode.FULL_AUTO + "");
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
       _dbs.add(dbName);
     }
 
@@ -168,9 +162,9 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
     for (int i = 0; i < NODE_NR - 2; i++) {
       _participants[i].syncStart();
     }
-    _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, 10, STATE_MODEL,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, 10, STATE_MODEL,
         RebalanceMode.FULL_AUTO.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _replica);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, _replica);
 
     HelixClusterVerifier _clusterVerifier =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
@@ -202,9 +196,9 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
     for (int i = 0; i < NODE_NR - 3; i++) {
       _participants[i].syncStart();
     }
-    _setupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "_ANY", 20,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "_ANY", 20,
         STATE_MODEL, RebalanceMode.FULL_AUTO.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "_ANY",
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + "_ANY",
         _replica);
 
     HelixClusterVerifier _clusterVerifier =
@@ -234,7 +228,7 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
   @AfterMethod
   public void cleanupTest() throws InterruptedException {
     for (String db : _dbs) {
-      _setupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
       Thread.sleep(20);
     }
     _dbs.clear();
@@ -267,7 +261,7 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
     for (int i = 0; i < 5; i++) {
       String dbName = "TestDB-" + i;
       IdealState is =
-          _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
       if (is != null) {
         System.err.println(dbName + "exists!");
         is.setReplicas(String.valueOf(replica));
@@ -276,12 +270,12 @@ public class TestPartitionMovementThrottle extends ZkStandAloneCMTestBase {
           is.setRebalanceDelay(delay);
         }
         is.setRebalancerClassName(DelayedAutoRebalancer.class.getName());
-        _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
+        _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
       } else {
         createResourceWithDelayedRebalance(CLUSTER_NAME, dbName, STATE_MODEL, partition, replica,
             minActiveReplica, delay);
       }
-      _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
       _dbs.add(dbName);
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java b/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java
index badf572..3a0e784 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPauseSignal.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -38,7 +38,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestPauseSignal extends ZkIntegrationTestBase {
+public class TestPauseSignal extends ZkTestBase {
   @Test()
   public void testPauseSignal() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java b/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
index debd923..c39ce0e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
@@ -50,17 +50,12 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // start controller
@@ -87,9 +82,9 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
       throws Exception {
     String testDb = "TestDB2-" + rebalanceMode.name();
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.LeaderStandby.name(), rebalanceMode.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
 
     BestPossibleExternalViewVerifier.Builder verifierBuilder =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
@@ -106,14 +101,14 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     Assert.assertTrue(verifierBuilder.build().verify());
 
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
 
     Set<String> excludedInstances = new HashSet<String>();
     excludedInstances.add(_participants[0].getInstanceName());
     verifyAssignmentInIdealStateWithPersistDisabled(idealState, excludedInstances);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
     _participants[0] =
         new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _participants[0].getInstanceName());
     _participants[0].syncStart();
@@ -125,9 +120,9 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     String testDb = "TestDB1-" + rebalanceMode.name();
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.LeaderStandby.name(), rebalanceMode.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
 
     BestPossibleExternalViewVerifier.Builder verifierBuilder =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
@@ -136,7 +131,7 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     Assert.assertTrue(verifierBuilder.build().verify());
 
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     verifyAssignmentInIdealStateWithPersistEnabled(idealState, new HashSet<String>());
 
     // kill 1 node
@@ -147,7 +142,7 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     verifierBuilder.setExpectLiveInstances(liveInstances);
     Assert.assertTrue(verifierBuilder.build().verify());
 
-    idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+    idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     // verify that IdealState contains updated assignment in it map fields.
 
     Set<String> excludedInstances = new HashSet<String>();
@@ -155,7 +150,7 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     verifyAssignmentInIdealStateWithPersistEnabled(idealState, excludedInstances);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
     _participants[0] =
         new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _participants[0].getInstanceName());
     _participants[0].syncStart();
@@ -171,9 +166,9 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     String testDb = "TestDB1-MasterSlave";
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.SEMI_AUTO.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
 
     BestPossibleExternalViewVerifier.Builder verifierBuilder =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
@@ -182,7 +177,7 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     Assert.assertTrue(verifierBuilder.build().verify());
 
     IdealState idealState =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     verifySemiAutoMasterSlaveAssignment(idealState);
 
     // kill 1 node
@@ -193,18 +188,18 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     verifierBuilder.setExpectLiveInstances(liveInstances);
     Assert.assertTrue(verifierBuilder.build().verify());
 
-    idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+    idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     verifySemiAutoMasterSlaveAssignment(idealState);
 
     // disable an instance
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, _participants[1].getInstanceName(), false);
-    idealState = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+    idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     verifySemiAutoMasterSlaveAssignment(idealState);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, _participants[1].getInstanceName(), true);
     _participants[0].reset();
     _participants[0].syncStart();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java b/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
index a1f1868..d81344c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
@@ -157,6 +157,7 @@ public class TestReelectedPipelineCorrectness extends ZkUnitTestBase {
     for (int i = 2; i < NUM_PARTICIPANTS; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("STOP " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
index 5177564..0f74565 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
@@ -28,7 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -40,7 +40,7 @@ import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestRenamePartition extends ZkIntegrationTestBase {
+public class TestRenamePartition extends ZkTestBase {
   // map from clusterName to participants
   final Map<String, MockParticipantManager[]> _participantMap =
       new ConcurrentHashMap<String, MockParticipantManager[]>();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
index a24e488..b307799 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.ErrTransition;
@@ -34,7 +34,7 @@ import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestResetInstance extends ZkIntegrationTestBase {
+public class TestResetInstance extends ZkTestBase {
 
   @Test
   public void testResetInstance() throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
index 295c1c0..c1e92a6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
@@ -28,7 +28,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -41,7 +41,7 @@ import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestResetPartitionState extends ZkIntegrationTestBase {
+public class TestResetPartitionState extends ZkTestBase {
   int _errToOfflineInvoked = 0;
 
   class ErrTransitionWithResetCnt extends ErrTransition {
@@ -175,6 +175,7 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
index 2246c6e..e038ebb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.ErrTransition;
@@ -34,7 +34,7 @@ import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestResetResource extends ZkIntegrationTestBase {
+public class TestResetResource extends ZkTestBase {
   @Test
   public void testResetNode() throws Exception {
     String className = TestHelper.getTestClassName();
@@ -106,6 +106,7 @@ public class TestResetResource extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java b/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
index f2d876d..ffd3000 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import org.apache.helix.HelixAdmin;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.ZkTestManager;
 import org.apache.helix.manager.zk.CallbackHandler;
@@ -36,7 +36,6 @@ import org.apache.helix.mock.participant.DummyProcess;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.OnlineOfflineSMD;
 import org.apache.helix.spectator.RoutingTableProvider;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -54,7 +53,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestResourceGroupEndtoEnd extends ZkIntegrationTestBase {
+public class TestResourceGroupEndtoEnd extends ZkTestBase {
 
   protected static final int GROUP_NODE_NR = 5;
   protected static final int START_PORT = 12918;
@@ -140,6 +139,7 @@ public class TestResourceGroupEndtoEnd extends ZkIntegrationTestBase {
     _controller.syncStop();
     _spectator.disconnect();
     _routingTableProvider.shutdown();
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
   }
 
   private void addInstanceGroup(String clusterName, String instanceTag, int numInstance) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestResourceWithSamePartitionKey.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResourceWithSamePartitionKey.java b/helix-core/src/test/java/org/apache/helix/integration/TestResourceWithSamePartitionKey.java
index bbb46eb..23bc801 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResourceWithSamePartitionKey.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResourceWithSamePartitionKey.java
@@ -119,6 +119,7 @@ public class TestResourceWithSamePartitionKey extends ZkUnitTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java
index 36ff05d..7cc367c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchemataSM.java
@@ -26,7 +26,7 @@ import java.util.Map;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -38,7 +38,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestSchemataSM extends ZkIntegrationTestBase {
+public class TestSchemataSM extends ZkTestBase {
   @Test
   public void testSchemataSM() throws Exception {
     String className = TestHelper.getTestClassName();

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
index fc2b3b5..698b099 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
@@ -25,7 +25,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
+public class TestSessionExpiryInTransition extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestSessionExpiryInTransition.class);
 
   public class SessionExpiryTransition extends MockTransition {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
index 7c2c930..4b56c09 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
@@ -23,7 +23,7 @@ import java.util.Date;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase {
+public class TestStandAloneCMSessionExpiry extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestStandAloneCMSessionExpiry.class);
 
   @Test()

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
index 348cfe2..30e37af 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
@@ -24,13 +24,13 @@ import java.util.Date;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestStartMultipleControllersWithSameName extends ZkIntegrationTestBase {
+public class TestStartMultipleControllersWithSameName extends ZkTestBase {
   @Test
   public void test() throws Exception {
     String className = TestHelper.getTestClassName();
@@ -67,6 +67,7 @@ public class TestStartMultipleControllersWithSameName extends ZkIntegrationTestB
     for (int i = 0; i < 4; i++) {
       controllers[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionCancellation.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionCancellation.java b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionCancellation.java
index e080e84..e2fa6f5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionCancellation.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionCancellation.java
@@ -66,13 +66,8 @@ public class TestStateTransitionCancellation extends TaskTestBase {
     _numReplicas = 2;
     _verifier =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
     setupParticipants();
     setupDBs();
 
@@ -98,13 +93,13 @@ public class TestStateTransitionCancellation extends TaskTestBase {
     Thread.sleep(2000);
 
     // Disable the resource
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, false);
 
 
     // Wait for pipeline reaching final stage
     Assert.assertTrue(_verifier.verify());
-    ExternalView externalView = _setupTool.getClusterManagementTool()
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
         .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
     for (String partition : externalView.getPartitionSet()) {
       for (String currentState : externalView.getStateMap(partition).values()) {
@@ -122,19 +117,19 @@ public class TestStateTransitionCancellation extends TaskTestBase {
 
     // Reenable resource
     stateCleanUp();
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, true);
 
     // Wait for assignment done
     Thread.sleep(2000);
 
     // Disable the resource
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, false);
 
     // Wait for pipeline reaching final stage
     Thread.sleep(2000L);
-    ExternalView externalView = _setupTool.getClusterManagementTool()
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
         .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
     for (String partition : externalView.getPartitionSet()) {
       Assert.assertTrue(externalView.getStateMap(partition).values().contains("SLAVE"));
@@ -151,7 +146,7 @@ public class TestStateTransitionCancellation extends TaskTestBase {
 
     // Reenable resource
     stateCleanUp();
-    _setupTool.getClusterManagementTool()
+    _gSetupTool.getClusterManagementTool()
         .enableResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, true);
 
     // Wait for assignment done
@@ -159,7 +154,7 @@ public class TestStateTransitionCancellation extends TaskTestBase {
     int numNodesToStart = 10;
     for (int i = 0; i < numNodesToStart; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + _numNodes + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
     MockParticipantManager[] newParticipants = new MockParticipantManager[numNodesToStart];
     registerParticipants(newParticipants, numNodesToStart, _startPort + _numNodes, 1000, -3000000L);
@@ -167,7 +162,7 @@ public class TestStateTransitionCancellation extends TaskTestBase {
     // Wait for pipeline reaching final stage
     Thread.sleep(2000L);
     int numOfMasters = 0;
-    ExternalView externalView = _setupTool.getClusterManagementTool()
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
         .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
     for (String partition : externalView.getPartitionSet()) {
       if (externalView.getStateMap(partition).values().contains("MASTER")) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionThrottle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionThrottle.java
index 5e8db94..124160e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionThrottle.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionThrottle.java
@@ -21,7 +21,7 @@ package org.apache.helix.integration;
 
 import org.apache.helix.*;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -38,7 +38,7 @@ import org.testng.annotations.Test;
 
 import java.util.*;
 
-public class TestStateTransitionThrottle extends ZkIntegrationTestBase {
+public class TestStateTransitionThrottle extends ZkTestBase {
   int participantCount = 4;
   String resourceName = "TestDB0";
   String resourceNamePrefix = "TestDB";
@@ -103,6 +103,7 @@ public class TestStateTransitionThrottle extends ZkIntegrationTestBase {
     for (int i = 0; i < participantCount; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -178,6 +179,7 @@ public class TestStateTransitionThrottle extends ZkIntegrationTestBase {
     for (int i = 0; i < participantCount; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -205,6 +207,8 @@ public class TestStateTransitionThrottle extends ZkIntegrationTestBase {
         StateTransitionThrottleConfig.ThrottleScope.CLUSTER, 100));
     clusterConfig.setStateTransitionThrottleConfigs(throttleConfigs);
     accessor.setProperty(keyBuilder.clusterConfig(), clusterConfig);
+
+    _gSetupTool.deleteCluster(clusterName);
   }
 
   private static boolean pollForPartitionAssignment(final HelixDataAccessor accessor,

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
index ab6b2a3..01460aa 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
@@ -24,7 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.common.ZkStandAloneCMTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
@@ -40,7 +40,7 @@ public class TestStatusUpdate extends ZkStandAloneCMTestBase {
   // this test
   // @Test
   public void testParticipantStatusUpdates() throws Exception {
-    ZkClient zkClient = new ZkClient(ZkIntegrationTestBase.ZK_ADDR);
+    ZkClient zkClient = new ZkClient(ZkTestBase.ZK_ADDR);
     zkClient.setZkSerializer(new ZNRecordSerializer());
     ZKHelixDataAccessor accessor =
         new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(zkClient));

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java
index e086671..acb2f65 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSwapInstance.java
@@ -35,8 +35,8 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
   public void TestSwap() throws Exception {
     HelixManager manager = _controller;
     HelixDataAccessor helixAccessor = manager.getHelixDataAccessor();
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 64, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", _replica);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 64, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", _replica);
 
     ZNRecord idealStateOld1 = new ZNRecord("TestDB");
     ZNRecord idealStateOld2 = new ZNRecord("MyDB");
@@ -51,8 +51,7 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
         .verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR, CLUSTER_NAME)));
 
     String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
-    _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instanceName, false);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instanceName, false);
 
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -60,11 +59,11 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
     Assert.assertTrue(result);
 
     String instanceName2 = PARTICIPANT_PREFIX + "_" + (START_PORT + 444);
-    _setupTool.addInstanceToCluster(CLUSTER_NAME, instanceName2);
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instanceName2);
 
     boolean exception = false;
     try {
-      _setupTool.swapInstance(CLUSTER_NAME, instanceName, instanceName2);
+      _gSetupTool.swapInstance(CLUSTER_NAME, instanceName, instanceName2);
     } catch (Exception e) {
       exception = true;
     }
@@ -75,7 +74,7 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
 
     exception = false;
     try {
-      _setupTool.swapInstance(CLUSTER_NAME, instanceName, instanceName2);
+      _gSetupTool.swapInstance(CLUSTER_NAME, instanceName, instanceName2);
     } catch (Exception e) {
       e.printStackTrace();
       exception = true;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestSyncSessionToController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSyncSessionToController.java b/helix-core/src/test/java/org/apache/helix/integration/TestSyncSessionToController.java
index 116b469..4579dfb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSyncSessionToController.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSyncSessionToController.java
@@ -9,7 +9,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixManager;
@@ -20,7 +20,7 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
-public class TestSyncSessionToController extends ZkIntegrationTestBase {
+public class TestSyncSessionToController extends ZkTestBase {
   @Test
   public void testSyncSessionToController() throws Exception {
     System.out.println("START testSyncSessionToController at " + new Date(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestWeightBasedRebalanceUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestWeightBasedRebalanceUtil.java b/helix-core/src/test/java/org/apache/helix/integration/TestWeightBasedRebalanceUtil.java
index 488fcab..243ce9e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestWeightBasedRebalanceUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestWeightBasedRebalanceUtil.java
@@ -14,9 +14,8 @@ import org.apache.helix.controller.rebalancer.constraint.dataprovider.MockCapaci
 import org.apache.helix.controller.rebalancer.constraint.dataprovider.MockPartitionWeightProvider;
 import org.apache.helix.controller.rebalancer.constraint.dataprovider.ZkBasedCapacityProvider;
 import org.apache.helix.controller.rebalancer.constraint.dataprovider.ZkBasedPartitionWeightProvider;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.model.*;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.util.WeightAwareRebalanceUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,10 +28,9 @@ import java.util.*;
 
 import static org.apache.helix.controller.rebalancer.constraint.dataprovider.ZkBasedPartitionWeightProvider.DEFAULT_WEIGHT_VALUE;
 
-public class TestWeightBasedRebalanceUtil extends ZkIntegrationTestBase {
+public class TestWeightBasedRebalanceUtil extends ZkTestBase {
   private static Logger _logger = LoggerFactory.getLogger(TestWeightBasedRebalanceUtil.class);
   private static String CLUSTER_NAME;
-  private static ClusterSetup _setupTool;
 
   final String resourceNamePrefix = "resource";
   final int nParticipants = 40;
@@ -80,19 +78,13 @@ public class TestWeightBasedRebalanceUtil extends ZkIntegrationTestBase {
 
     setupMockCluster();
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
   }
 
   @AfterClass
   public void afterClass() {
-    _setupTool.deleteCluster(CLUSTER_NAME);
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
   }
 
   private void setupMockCluster() {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestZkConnectionLost.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkConnectionLost.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkConnectionLost.java
index 319f540..c6a795e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestZkConnectionLost.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkConnectionLost.java
@@ -12,6 +12,8 @@ import org.apache.helix.integration.task.MockTask;
 import org.apache.helix.integration.task.TaskTestBase;
 import org.apache.helix.integration.task.TaskTestUtil;
 import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.manager.zk.ZNRecordSerializer;
+import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.TaskState;
@@ -22,6 +24,7 @@ import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
@@ -37,22 +40,21 @@ public class TestZkConnectionLost extends TaskTestBase {
   private final AtomicReference<ZkServer> _zkServerRef = new AtomicReference<>();
 
   private String _zkAddr = "localhost:2189";
+  ClusterSetup _setupTool;
+  ZkClient _zkClient;
+
 
   @BeforeClass
   public void beforeClass() throws Exception {
     ZkServer zkServer = TestHelper.startZkServer(_zkAddr);
     _zkServerRef.set(zkServer);
-
+    _zkClient = new ZkClient(_zkAddr);
+    _zkClient.setZkSerializer(new ZNRecordSerializer());
+    _setupTool = new ClusterSetup(_zkClient);
     _participants =  new MockParticipantManager[_numNodes];
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    _setupTool = new ClusterSetup(_zkAddr);
     _setupTool.addCluster(CLUSTER_NAME, true);
-    setupParticipants();
-    setupDBs();
+    setupParticipants(_setupTool);
+    setupDBs(_setupTool);
     createManagers(_zkAddr, CLUSTER_NAME);
 
     // start controller
@@ -65,6 +67,29 @@ public class TestZkConnectionLost extends TaskTestBase {
     Assert.assertTrue(clusterVerifier.verify());
   }
 
+  @AfterClass
+  public void afterClass() throws Exception {
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
+    if (_manager != null && _manager.isConnected()) {
+      _manager.disconnect();
+    }
+    stopParticipants();
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_zkClient.exists(namespace)) {
+      try {
+        _setupTool.deleteCluster(CLUSTER_NAME);
+      } catch (Exception ex) {
+        System.err.println(
+            "Failed to delete cluster " + CLUSTER_NAME + ", error: " + ex.getLocalizedMessage());
+      }
+    }
+    _zkClient.close();
+    TestHelper.stopZkServer(_zkServerRef.get());
+  }
+
   @Test
   public void testLostZkConnection() throws Exception {
     System.setProperty(SystemPropertyKeys.ZK_WAIT_CONNECTED_TIMEOUT, "1000");

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/common/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/common/IntegrationTest.java b/helix-core/src/test/java/org/apache/helix/integration/common/IntegrationTest.java
deleted file mode 100644
index c2882e4..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/common/IntegrationTest.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package org.apache.helix.integration.common;
-
-/*
- * 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 dist_hdlrFtyRegistryributed 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.
- */
-
-import java.util.Date;
-
-import org.testng.annotations.Test;
-
-/**
- * This is a simple integration test. We will use this until we have framework
- * which helps us write integration tests easily
- */
-
-public class IntegrationTest extends ZkStandAloneCMTestBase {
-  @Test
-  public void integrationTest() throws Exception {
-    System.out.println("START IntegrationTest at " + new Date(System.currentTimeMillis()));
-    // Thread.currentThread().join();
-    System.out.println("END IntegrationTest at " + new Date(System.currentTimeMillis()));
-  }
-}


[5/5] helix git commit: Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.

Posted by jx...@apache.org.
Properly remove clusters after each test, and clean up duplicated codes in tests and move them into base test classes.


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

Branch: refs/heads/master
Commit: c0d5792b745c67b6fee56ba79df02be89d1f049e
Parents: fe97055
Author: Lei Xia <lx...@linkedin.com>
Authored: Thu Jun 7 17:15:54 2018 -0700
Committer: Lei Xia <lx...@linkedin.com>
Committed: Fri Jul 13 11:20:49 2018 -0700

----------------------------------------------------------------------
 .../BestPossibleExternalViewVerifier.java       |  13 +-
 .../java/org/apache/helix/ZkUnitTestBase.java   | 442 +-----------
 .../org/apache/helix/common/ZkTestBase.java     | 691 +++++++++++++++++++
 .../stages/TestMessageThrottleStage.java        |   4 +-
 .../stages/TestRebalancePipeline.java           |   8 +-
 .../SinglePartitionLeaderStandByTest.java       |   5 +-
 .../helix/integration/TestAddClusterV2.java     |  23 +-
 .../TestAddNodeAfterControllerStart.java        |   6 +-
 .../TestAddStateModelFactoryAfterConnect.java   |   5 +-
 .../TestAlertingRebalancerFailure.java          |  39 +-
 .../helix/integration/TestBasicSpectator.java   |   4 +-
 .../integration/TestBatchMessageHandling.java   |   6 +-
 .../integration/TestBucketizedResource.java     |   4 +-
 .../integration/TestCMWithFailParticipant.java  |   4 +-
 .../integration/TestCarryOverBadCurState.java   |   6 +-
 .../integration/TestCleanupExternalView.java    |   5 +-
 .../helix/integration/TestClusterStartsup.java  |  22 +-
 .../TestCorrectnessOnConnectivityLoss.java      |  12 +-
 .../apache/helix/integration/TestDisable.java   |   4 +-
 .../integration/TestDisableExternalView.java    |   4 +-
 .../helix/integration/TestDisableResource.java  |   2 +
 .../integration/TestDistributedCMMain.java      |   4 +-
 .../TestDistributedClusterController.java       |   4 +-
 .../apache/helix/integration/TestDriver.java    |   5 +-
 .../org/apache/helix/integration/TestDrop.java  |   4 +-
 .../helix/integration/TestDropResource.java     |   8 +-
 .../integration/TestEnableCompression.java      |   4 +-
 .../TestEnablePartitionDuringDisable.java       |   4 +-
 .../integration/TestEntropyFreeNodeBounce.java  |   1 +
 .../helix/integration/TestErrorPartition.java   |   4 +-
 .../helix/integration/TestExpandCluster.java    |  22 +-
 .../integration/TestExternalViewUpdates.java    |   4 +-
 .../integration/TestHelixCustomCodeRunner.java  |   6 +-
 .../helix/integration/TestHelixInstanceTag.java |  12 +-
 .../TestHelixUsingDifferentParams.java          |   4 +-
 .../TestInvalidResourceRebalance.java           |   1 +
 .../helix/integration/TestNullReplica.java      |   5 +-
 .../TestPartitionLevelTransitionConstraint.java |   5 +-
 .../TestPartitionMovementThrottle.java          |  36 +-
 .../helix/integration/TestPauseSignal.java      |   4 +-
 .../TestRebalancerPersistAssignments.java       |  43 +-
 .../TestReelectedPipelineCorrectness.java       |   1 +
 .../helix/integration/TestRenamePartition.java  |   4 +-
 .../helix/integration/TestResetInstance.java    |   4 +-
 .../integration/TestResetPartitionState.java    |   5 +-
 .../helix/integration/TestResetResource.java    |   5 +-
 .../integration/TestResourceGroupEndtoEnd.java  |   6 +-
 .../TestResourceWithSamePartitionKey.java       |   1 +
 .../helix/integration/TestSchemataSM.java       |   4 +-
 .../TestSessionExpiryInTransition.java          |   4 +-
 .../TestStandAloneCMSessionExpiry.java          |   4 +-
 ...estStartMultipleControllersWithSameName.java |   5 +-
 .../TestStateTransitionCancellation.java        |  23 +-
 .../TestStateTransitionThrottle.java            |   8 +-
 .../helix/integration/TestStatusUpdate.java     |   4 +-
 .../helix/integration/TestSwapInstance.java     |  13 +-
 .../TestSyncSessionToController.java            |   4 +-
 .../TestWeightBasedRebalanceUtil.java           |  16 +-
 .../helix/integration/TestZkConnectionLost.java |  43 +-
 .../integration/common/IntegrationTest.java     |  38 -
 .../common/ZkIntegrationTestBase.java           | 322 ---------
 .../common/ZkStandAloneCMTestBase.java          |  36 +-
 .../TestClusterDataCacheSelectiveUpdate.java    |  12 +-
 .../controller/TestClusterMaintenanceMode.java  |  17 +-
 .../TestControllerLeadershipChange.java         |   4 +-
 .../controller/TestControllerLiveLock.java      |   1 +
 .../TestSkipBestPossibleCalculation.java        |   4 +-
 .../TestDistributedControllerManager.java       |   4 +-
 .../manager/TestHelixDataAccessor.java          |   7 +-
 .../manager/TestParticipantManager.java         |   4 +-
 .../integration/messaging/TestBatchMessage.java |  15 +-
 .../messaging/TestBatchMessageWrapper.java      |   1 +
 .../messaging/TestMessageThrottle.java          |   4 +-
 .../messaging/TestMessageThrottle2.java         |   4 +-
 .../messaging/TestP2PMessageSemiAuto.java       |  19 +-
 .../paticipant/TestInstanceAutoJoin.java        |   4 +-
 .../paticipant/TestNonOfflineInitState.java     |  21 +-
 .../paticipant/TestRestartParticipant.java      |   5 +-
 .../paticipant/TestStateTransitionTimeout.java  |  18 +-
 .../TestStateTransitionTimeoutWithResource.java |  30 +-
 .../TestCrushAutoRebalance.java                 |  81 +--
 .../TestCrushAutoRebalanceNonRack.java          |  73 +-
 ...rushAutoRebalanceTopoplogyAwareDisabled.java |  20 +-
 .../TestDelayedAutoRebalance.java               |  48 +-
 ...elayedAutoRebalanceWithDisabledInstance.java |  54 +-
 .../TestDelayedAutoRebalanceWithRackaware.java  |  11 +-
 .../PartitionMigration/TestExpandCluster.java   |   6 +-
 .../TestFullAutoMigration.java                  |   4 +-
 .../TestPartitionMigrationBase.java             |  43 +-
 .../rebalancer/TestAutoIsWithEmptyMap.java      |   5 +-
 .../rebalancer/TestAutoRebalance.java           |  30 +-
 .../TestAutoRebalancePartitionLimit.java        |  16 +-
 .../TestAutoRebalanceWithDisabledInstance.java  |  18 +-
 ...MaintenanceModeWhenReachingMaxPartition.java |   9 +-
 ...ceModeWhenReachingOfflineInstancesLimit.java |  10 +-
 .../rebalancer/TestCustomIdealState.java        |   4 +-
 .../TestCustomizedIdealStateRebalancer.java     |   8 +-
 .../rebalancer/TestFullAutoNodeTagging.java     |   9 +
 .../rebalancer/TestMixedModeAutoRebalance.java  |   8 +-
 .../rebalancer/TestSemiAutoRebalance.java       |  24 +-
 .../rebalancer/TestZeroReplicaAvoidance.java    |  19 +-
 .../spectator/TestRoutingTableProvider.java     |   4 +-
 ...stRoutingTableProviderFromCurrentStates.java |  47 +-
 .../TestRoutingTableProviderFromTargetEV.java   |  36 +-
 ...TestRoutingTableProviderPeriodicRefresh.java |   4 +-
 .../spectator/TestRoutingTableSnapshot.java     |  26 +-
 .../helix/integration/task/TaskTestBase.java    |  11 +-
 .../integration/task/TestBatchAddJobs.java      |  13 +-
 .../task/TestIndependentTaskRebalancer.java     |  10 +-
 .../helix/integration/task/TestJobFailure.java  |  10 +-
 .../task/TestJobFailureDependence.java          |   8 +-
 .../task/TestJobFailureHighThreshold.java       |   9 +-
 .../task/TestJobFailureTaskNotStarted.java      |  20 +-
 .../helix/integration/task/TestJobTimeout.java  |   9 +-
 .../task/TestJobTimeoutTaskNotStarted.java      |  10 +-
 .../task/TestRebalanceRunningTask.java          |  10 +-
 .../task/TestRunJobsWithMissingTarget.java      |   4 +-
 .../integration/task/TestTaskAssignment.java    |   2 +-
 .../task/TestTaskRebalancerParallel.java        |   4 -
 .../integration/task/TestTaskThreadLeak.java    |   4 +-
 .../integration/task/TestTaskThrottling.java    |   6 +-
 .../task/TestTaskWithInstanceDisabled.java      |   2 +-
 .../integration/task/TestUserContentStore.java  |   9 +-
 .../helix/manager/zk/TestHandleNewSession.java  |   5 +-
 .../manager/zk/TestZkBaseDataAccessor.java      |  91 +--
 .../zk/TestZkManagerFlappingDetection.java      |   4 +-
 .../handling/TestBatchMessageModeConfigs.java   |  14 +-
 .../handling/TestResourceThreadpoolSize.java    |  24 +-
 .../TestClusterStatusMonitorLifecycle.java      |  23 +-
 .../mbeans/TestClusterAggregateMetrics.java     |   4 +-
 .../mbeans/TestDisableResourceMbean.java        |   1 +
 .../mbeans/TestDropResourceMetricsReset.java    |   1 +
 .../mbeans/TestResetClusterMetrics.java         |   1 +
 .../participant/TestDistControllerElection.java |  16 +-
 .../TestDistControllerStateModel.java           |   8 +
 .../helix/spectator/TestRoutingDataCache.java   |  14 +-
 .../helix/task/TaskSynchronizedTestBase.java    |  66 +-
 ...signableInstanceManagerControllerSwitch.java |   2 +-
 .../helix/task/TestJobStateOnCreation.java      |   8 +-
 .../helix/task/TestSemiAutoStateTransition.java |  12 +-
 .../apache/helix/tools/TestClusterSetup.java    |  11 +-
 .../apache/helix/tools/TestHelixAdminCli.java   |   4 +-
 142 files changed, 1561 insertions(+), 1719 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/BestPossibleExternalViewVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/BestPossibleExternalViewVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/BestPossibleExternalViewVerifier.java
index 4abfd07..fa5d29f 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/BestPossibleExternalViewVerifier.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/BestPossibleExternalViewVerifier.java
@@ -82,18 +82,7 @@ public class BestPossibleExternalViewVerifier extends ZkHelixClusterVerifier {
     _expectLiveInstances = expectLiveInstances;
     _clusterDataCache = new ClusterDataCache();
   }
-
-  public static void main (String [] args) {
-    Set<String> resources = Collections.singleton("SyncColoTestDB");
-    BestPossibleExternalViewVerifier verifier =
-        new BestPossibleExternalViewVerifier.Builder("ESPRESSO_MT1")
-            .setZkAddr("zk-ltx1-espresso.stg.linkedin.com:12913")
-            .setResources(resources)
-            .build();
-
-    verifier.verify();
-  }
-
+  
   public static class Builder {
     private String _clusterName;
     private Map<String, Map<String, String>> _errStates;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
index 483f0af..5757b24 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
@@ -19,445 +19,15 @@ package org.apache.helix;
  * under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.I0Itec.zkclient.IZkStateListener;
-import org.I0Itec.zkclient.ZkConnection;
-import org.I0Itec.zkclient.ZkServer;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.controller.pipeline.AbstractAsyncBaseStage;
-import org.apache.helix.controller.pipeline.Pipeline;
-import org.apache.helix.controller.pipeline.Stage;
-import org.apache.helix.controller.pipeline.StageContext;
-import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.manager.zk.ZKHelixAdmin;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.Attributes;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.tools.ClusterStateVerifier.ZkVerifier;
-import org.apache.helix.tools.StateModelConfigGenerator;
-import org.apache.helix.util.ZKClientPool;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.Assert;
-import org.testng.AssertJUnit;
-import org.testng.annotations.AfterSuite;
-import org.testng.annotations.BeforeSuite;
+import org.apache.helix.common.ZkTestBase;
 
 // TODO merge code with ZkIntegrationTestBase
-public class ZkUnitTestBase {
-  private static Logger LOG = LoggerFactory.getLogger(ZkUnitTestBase.class);
-  protected static ZkServer _zkServer = null;
-  protected static ZkClient _gZkClient;
-
-  public static final String ZK_ADDR = "localhost:2185";
-  protected static final String CLUSTER_PREFIX = "CLUSTER";
-  protected static final String CONTROLLER_CLUSTER_PREFIX = "CONTROLLER_CLUSTER";
-
-  @BeforeSuite(alwaysRun = true)
-  public void beforeSuite() throws Exception {
-    // Due to ZOOKEEPER-2693 fix, we need to specify whitelist for execute zk commends
-    System.setProperty("zookeeper.4lw.commands.whitelist", "*");
-    System.setProperty(SystemPropertyKeys.CONTROLLER_MESSAGE_PURGE_DELAY, "3000");
-
-    _zkServer = TestHelper.startZkServer(ZK_ADDR);
-    AssertJUnit.assertTrue(_zkServer != null);
-    ZKClientPool.reset();
-
-    // System.out.println("Number of open zkClient before ZkUnitTests: "
-    // + ZkClient.getNumberOfConnections());
-
-    _gZkClient = new ZkClient(ZK_ADDR);
-    _gZkClient.setZkSerializer(new ZNRecordSerializer());
-  }
-
-  @AfterSuite(alwaysRun = true)
-  public void afterSuite() {
-    _gZkClient.close();
-    TestHelper.stopZkServer(_zkServer);
-    _zkServer = null;
-
-    // System.out.println("Number of open zkClient after ZkUnitTests: "
-    // + ZkClient.getNumberOfConnections());
-
-  }
-
-  protected String getShortClassName() {
-    String className = this.getClass().getName();
-    return className.substring(className.lastIndexOf('.') + 1);
-  }
-
-  protected String getCurrentLeader(ZkClient zkClient, String clusterName) {
-    String leaderPath = PropertyPathBuilder.controllerLeader(clusterName);
-    ZNRecord leaderRecord = zkClient.<ZNRecord> readData(leaderPath);
-    if (leaderRecord == null) {
-      return null;
-    }
-
-    String leader = leaderRecord.getSimpleField(PropertyType.LEADER.toString());
-    return leader;
-  }
-
-  protected void stopCurrentLeader(ZkClient zkClient, String clusterName,
-      Map<String, Thread> threadMap, Map<String, HelixManager> managerMap) {
-    String leader = getCurrentLeader(zkClient, clusterName);
-    Assert.assertTrue(leader != null);
-    System.out.println("stop leader:" + leader + " in " + clusterName);
-    Assert.assertTrue(leader != null);
-
-    HelixManager manager = managerMap.remove(leader);
-    Assert.assertTrue(manager != null);
-    manager.disconnect();
-
-    Thread thread = threadMap.remove(leader);
-    Assert.assertTrue(thread != null);
-    thread.interrupt();
-
-    boolean isNewLeaderElected = false;
-    try {
-      // Thread.sleep(2000);
-      for (int i = 0; i < 5; i++) {
-        Thread.sleep(1000);
-        String newLeader = getCurrentLeader(zkClient, clusterName);
-        if (!newLeader.equals(leader)) {
-          isNewLeaderElected = true;
-          System.out.println("new leader elected: " + newLeader + " in " + clusterName);
-          break;
-        }
-      }
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-    if (isNewLeaderElected == false) {
-      System.out.println("fail to elect a new leader elected in " + clusterName);
-    }
-    AssertJUnit.assertTrue(isNewLeaderElected);
-  }
-
-  public void verifyInstance(ZkClient zkClient, String clusterName, String instance,
-      boolean wantExists) {
-    // String instanceConfigsPath = HelixUtil.getConfigPath(clusterName);
-    String instanceConfigsPath = PropertyPathBuilder.instanceConfig(clusterName);
-    String instanceConfigPath = instanceConfigsPath + "/" + instance;
-    String instancePath = PropertyPathBuilder.instance(clusterName, instance);
-    AssertJUnit.assertEquals(wantExists, zkClient.exists(instanceConfigPath));
-    AssertJUnit.assertEquals(wantExists, zkClient.exists(instancePath));
-  }
-
-  public void verifyResource(ZkClient zkClient, String clusterName, String resource,
-      boolean wantExists) {
-    String resourcePath = PropertyPathBuilder.idealState(clusterName, resource);
-    AssertJUnit.assertEquals(wantExists, zkClient.exists(resourcePath));
-  }
-
-  public void verifyEnabled(ZkClient zkClient, String clusterName, String instance,
-      boolean wantEnabled) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instance));
-    AssertJUnit.assertEquals(wantEnabled, config.getInstanceEnabled());
-  }
-
-  public void verifyReplication(ZkClient zkClient, String clusterName, String resource, int repl) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resource));
-    for (String partitionName : idealState.getPartitionSet()) {
-      if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
-        AssertJUnit.assertEquals(repl, idealState.getPreferenceList(partitionName).size());
-      } else if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
-        AssertJUnit.assertEquals(repl, idealState.getInstanceStateMap(partitionName).size());
-      }
-    }
-  }
-
-  protected void simulateSessionExpiry(ZkConnection zkConnection) throws IOException,
-      InterruptedException {
-    ZooKeeper oldZookeeper = zkConnection.getZookeeper();
-    LOG.info("Old sessionId = " + oldZookeeper.getSessionId());
-
-    Watcher watcher = new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        LOG.info("In New connection, process event:" + event);
-      }
-    };
-
-    ZooKeeper newZookeeper =
-        new ZooKeeper(zkConnection.getServers(), oldZookeeper.getSessionTimeout(), watcher,
-            oldZookeeper.getSessionId(), oldZookeeper.getSessionPasswd());
-    LOG.info("New sessionId = " + newZookeeper.getSessionId());
-    // Thread.sleep(3000);
-    newZookeeper.close();
-    Thread.sleep(10000);
-    oldZookeeper = zkConnection.getZookeeper();
-    LOG.info("After session expiry sessionId = " + oldZookeeper.getSessionId());
-  }
-
-  protected void simulateSessionExpiry(ZkClient zkClient) throws IOException, InterruptedException {
-    IZkStateListener listener = new IZkStateListener() {
-      @Override
-      public void handleStateChanged(KeeperState state) throws Exception {
-        LOG.info("In Old connection, state changed:" + state);
-      }
-
-      @Override
-      public void handleNewSession() throws Exception {
-        LOG.info("In Old connection, new session");
-      }
-
-      @Override
-      public void handleSessionEstablishmentError(Throwable var1) throws Exception {
-      }
-    };
-    zkClient.subscribeStateChanges(listener);
-    ZkConnection connection = ((ZkConnection) zkClient.getConnection());
-    ZooKeeper oldZookeeper = connection.getZookeeper();
-    LOG.info("Old sessionId = " + oldZookeeper.getSessionId());
-
-    Watcher watcher = new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        LOG.info("In New connection, process event:" + event);
-      }
-    };
-
-    ZooKeeper newZookeeper =
-        new ZooKeeper(connection.getServers(), oldZookeeper.getSessionTimeout(), watcher,
-            oldZookeeper.getSessionId(), oldZookeeper.getSessionPasswd());
-    LOG.info("New sessionId = " + newZookeeper.getSessionId());
-    // Thread.sleep(3000);
-    newZookeeper.close();
-    Thread.sleep(10000);
-    connection = (ZkConnection) zkClient.getConnection();
-    oldZookeeper = connection.getZookeeper();
-    LOG.info("After session expiry sessionId = " + oldZookeeper.getSessionId());
-  }
-
-  protected void setupStateModel(String clusterName) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    StateModelDefinition masterSlave =
-        new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave());
-    accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), masterSlave);
-
-    StateModelDefinition leaderStandby =
-        new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby());
-    accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), leaderStandby);
-
-    StateModelDefinition onlineOffline =
-        new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline());
-    accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), onlineOffline);
-
-  }
-
-  protected List<IdealState> setupIdealState(String clusterName, int[] nodes, String[] resources,
-      int partitions, int replicas) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    List<IdealState> idealStates = new ArrayList<IdealState>();
-    List<String> instances = new ArrayList<String>();
-    for (int i : nodes) {
-      instances.add("localhost_" + i);
-    }
-
-    for (String resourceName : resources) {
-      IdealState idealState = new IdealState(resourceName);
-      for (int p = 0; p < partitions; p++) {
-        List<String> value = new ArrayList<String>();
-        for (int r = 0; r < replicas; r++) {
-          int n = nodes[(p + r) % nodes.length];
-          value.add("localhost_" + n);
-        }
-        idealState.getRecord().setListField(resourceName + "_" + p, value);
-      }
-
-      idealState.setReplicas(Integer.toString(replicas));
-      idealState.setStateModelDefRef("MasterSlave");
-      idealState.setRebalanceMode(RebalanceMode.SEMI_AUTO);
-      idealState.setNumPartitions(partitions);
-      idealStates.add(idealState);
-
-      // System.out.println(idealState);
-      accessor.setProperty(keyBuilder.idealStates(resourceName), idealState);
-    }
-    return idealStates;
-  }
-
-  protected void setupLiveInstances(String clusterName, int[] liveInstances) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    for (int i = 0; i < liveInstances.length; i++) {
-      String instance = "localhost_" + liveInstances[i];
-      LiveInstance liveInstance = new LiveInstance(instance);
-      liveInstance.setSessionId("session_" + liveInstances[i]);
-      liveInstance.setHelixVersion("0.0.0");
-      accessor.setProperty(keyBuilder.liveInstance(instance), liveInstance);
-    }
-  }
-
-  protected void setupInstances(String clusterName, int[] instances) {
-    HelixAdmin admin = new ZKHelixAdmin(_gZkClient);
-    for (int i = 0; i < instances.length; i++) {
-      String instance = "localhost_" + instances[i];
-      InstanceConfig instanceConfig = new InstanceConfig(instance);
-      instanceConfig.setHostName("localhost");
-      instanceConfig.setPort("" + instances[i]);
-      instanceConfig.setInstanceEnabled(true);
-      admin.addInstance(clusterName, instanceConfig);
-    }
-  }
-
-  protected void runPipeline(ClusterEvent event, Pipeline pipeline) {
-    try {
-      pipeline.handle(event);
-      pipeline.finish();
-    } catch (Exception e) {
-      LOG.error("Exception while executing pipeline:" + pipeline
-          + ". Will not continue to next pipeline", e);
-    }
-  }
-
-  protected void runStage(ClusterEvent event, Stage stage) throws Exception {
-    StageContext context = new StageContext();
-    stage.init(context);
-    stage.preProcess();
-
-    // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in
-    // execute() function call
-    // TODO (harry): duplicated code in ZkIntegrationTestBase, consider moving runStage()
-    // to a shared library
-    if (stage instanceof AbstractAsyncBaseStage) {
-      ((AbstractAsyncBaseStage) stage).execute(event);
-    } else {
-      stage.process(event);
-    }
-    stage.postProcess();
-  }
-
-  protected Message createMessage(MessageType type, String msgId, String fromState, String toState,
-      String resourceName, String tgtName) {
-    Message msg = new Message(type.toString(), msgId);
-    msg.setFromState(fromState);
-    msg.setToState(toState);
-    msg.getRecord().setSimpleField(Attributes.RESOURCE_NAME.toString(), resourceName);
-    msg.setTgtName(tgtName);
-    return msg;
-  }
-
-  /**
-   * Poll for the existence (or lack thereof) of a specific Helix property
-   * @param clazz the HelixProeprty subclass
-   * @param accessor connected HelixDataAccessor
-   * @param key the property key to look up
-   * @param shouldExist true if the property should exist, false otherwise
-   * @return the property if found, or null if it does not exist
-   */
-  protected <T extends HelixProperty> T pollForProperty(Class<T> clazz, HelixDataAccessor accessor,
-      PropertyKey key, boolean shouldExist) throws InterruptedException {
-    final int POLL_TIMEOUT = 5000;
-    final int POLL_INTERVAL = 50;
-    T property = accessor.getProperty(key);
-    int timeWaited = 0;
-    while (((shouldExist && property == null) || (!shouldExist && property != null))
-        && timeWaited < POLL_TIMEOUT) {
-      Thread.sleep(POLL_INTERVAL);
-      timeWaited += POLL_INTERVAL;
-      property = accessor.getProperty(key);
-    }
-    return property;
-  }
-
-  /**
-   * Ensures that external view and current state are empty
-   */
-  protected static class EmptyZkVerifier implements ZkVerifier {
-    private final String _clusterName;
-    private final String _resourceName;
-    private final ZkClient _zkClient;
-
-    /**
-     * Instantiate the verifier
-     * @param clusterName the cluster to verify
-     * @param resourceName the resource to verify
-     */
-    public EmptyZkVerifier(String clusterName, String resourceName) {
-      _clusterName = clusterName;
-      _resourceName = resourceName;
-      _zkClient = ZKClientPool.getZkClient(ZK_ADDR);
-    }
-
-    @Override
-    public boolean verify() {
-      BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-      HelixDataAccessor accessor = new ZKHelixDataAccessor(_clusterName, baseAccessor);
-      PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-      ExternalView externalView = accessor.getProperty(keyBuilder.externalView(_resourceName));
-
-      // verify external view empty
-      if (externalView != null) {
-        for (String partition : externalView.getPartitionSet()) {
-          Map<String, String> stateMap = externalView.getStateMap(partition);
-          if (stateMap != null && !stateMap.isEmpty()) {
-            LOG.error("External view not empty for " + partition);
-            return false;
-          }
-        }
-      }
-
-      // verify current state empty
-      List<String> liveParticipants = accessor.getChildNames(keyBuilder.liveInstances());
-      for (String participant : liveParticipants) {
-        List<String> sessionIds = accessor.getChildNames(keyBuilder.sessions(participant));
-        for (String sessionId : sessionIds) {
-          CurrentState currentState =
-              accessor.getProperty(keyBuilder.currentState(participant, sessionId, _resourceName));
-          Map<String, String> partitionStateMap = currentState.getPartitionStateMap();
-          if (partitionStateMap != null && !partitionStateMap.isEmpty()) {
-            LOG.error("Current state not empty for " + participant);
-            return false;
-          }
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public ZkClient getZkClient() {
-      return _zkClient;
-    }
+public class ZkUnitTestBase extends ZkTestBase {
 
-    @Override
-    public String getClusterName() {
-      return _clusterName;
+  protected void deleteCluster(String clusterName) {
+    String namespace = "/" + clusterName;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursively(namespace);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
new file mode 100644
index 0000000..7d98119
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
@@ -0,0 +1,691 @@
+package org.apache.helix.common;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import org.I0Itec.zkclient.IZkStateListener;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.ZkServer;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.PropertyPathBuilder;
+import org.apache.helix.SystemPropertyKeys;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.pipeline.AbstractAsyncBaseStage;
+import org.apache.helix.controller.pipeline.Pipeline;
+import org.apache.helix.controller.pipeline.Stage;
+import org.apache.helix.controller.pipeline.StageContext;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy;
+import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.ClusterEvent;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZNRecordSerializer;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.OnlineOfflineSMD;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.tools.ClusterSetup;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.helix.tools.StateModelConfigGenerator;
+import org.apache.helix.util.ZKClientPool;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.AssertJUnit;
+import org.testng.ITestContext;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.BeforeSuite;
+
+public class ZkTestBase {
+  private static Logger LOG = LoggerFactory.getLogger(ZkTestBase.class);
+
+  protected static ZkServer _zkServer;
+  protected static ZkClient _gZkClient;
+  protected static ClusterSetup _gSetupTool;
+  protected static BaseDataAccessor<ZNRecord> _baseAccessor;
+
+  public static final String ZK_ADDR = "localhost:2183";
+  protected static final String CLUSTER_PREFIX = "CLUSTER";
+  protected static final String CONTROLLER_CLUSTER_PREFIX = "CONTROLLER_CLUSTER";
+  protected final String CONTROLLER_PREFIX = "controller";
+  protected final String PARTICIPANT_PREFIX = "localhost";
+
+
+  @BeforeSuite
+  public void beforeSuite() throws Exception {
+    // TODO: use logging.properties file to config java.util.logging.Logger levels
+    java.util.logging.Logger topJavaLogger = java.util.logging.Logger.getLogger("");
+    topJavaLogger.setLevel(Level.WARNING);
+
+    // Due to ZOOKEEPER-2693 fix, we need to specify whitelist for execute zk commends
+    System.setProperty("zookeeper.4lw.commands.whitelist", "*");
+    System.setProperty(SystemPropertyKeys.CONTROLLER_MESSAGE_PURGE_DELAY, "3000");
+
+    _zkServer = TestHelper.startZkServer(ZK_ADDR);
+    AssertJUnit.assertTrue(_zkServer != null);
+    ZKClientPool.reset();
+
+    _gZkClient = new ZkClient(ZK_ADDR);
+    _gZkClient.setZkSerializer(new ZNRecordSerializer());
+    _gSetupTool = new ClusterSetup(_gZkClient);
+    _baseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+  }
+
+  @AfterSuite
+  public void afterSuite() {
+    ZKClientPool.reset();
+    _gZkClient.close();
+    TestHelper.stopZkServer(_zkServer);
+  }
+
+  @BeforeMethod
+  public void beforeTest(Method testMethod, ITestContext testContext){
+    long startTime = System.currentTimeMillis();
+    System.out.println("START " + testMethod.getName() + " at " + new Date(startTime));
+    testContext.setAttribute("StartTime", System.currentTimeMillis());
+  }
+
+  @AfterMethod
+  public void endTest(Method testMethod, ITestContext testContext) {
+    Long startTime = (Long) testContext.getAttribute("StartTime");
+    long endTime = System.currentTimeMillis();
+    System.out.println(
+        "END " + testMethod.getName() + " at " + new Date(endTime) + ", took: " + (endTime
+            - startTime) + "ms.");
+  }
+
+
+  protected String getShortClassName() {
+    return this.getClass().getSimpleName();
+  }
+
+  protected String getCurrentLeader(ZkClient zkClient, String clusterName) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader());
+    if (leader == null) {
+      return null;
+    }
+    return leader.getInstanceName();
+  }
+
+  protected void stopCurrentLeader(ZkClient zkClient, String clusterName,
+      Map<String, Thread> threadMap, Map<String, HelixManager> managerMap) {
+    String leader = getCurrentLeader(zkClient, clusterName);
+    Assert.assertTrue(leader != null);
+    System.out.println("stop leader:" + leader + " in " + clusterName);
+    Assert.assertTrue(leader != null);
+
+    HelixManager manager = managerMap.remove(leader);
+    Assert.assertTrue(manager != null);
+    manager.disconnect();
+
+    Thread thread = threadMap.remove(leader);
+    Assert.assertTrue(thread != null);
+    thread.interrupt();
+
+    boolean isNewLeaderElected = false;
+    try {
+      // Thread.sleep(2000);
+      for (int i = 0; i < 5; i++) {
+        Thread.sleep(1000);
+        String newLeader = getCurrentLeader(zkClient, clusterName);
+        if (!newLeader.equals(leader)) {
+          isNewLeaderElected = true;
+          System.out.println("new leader elected: " + newLeader + " in " + clusterName);
+          break;
+        }
+      }
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+    if (isNewLeaderElected == false) {
+      System.out.println("fail to elect a new leader elected in " + clusterName);
+    }
+    AssertJUnit.assertTrue(isNewLeaderElected);
+  }
+
+  protected void enableHealthCheck(String clusterName) {
+    ConfigScope scope = new ConfigScopeBuilder().forCluster(clusterName).build();
+    new ConfigAccessor(_gZkClient).set(scope, "healthChange" + ".enabled", "" + true);
+  }
+
+  protected void enablePersistBestPossibleAssignment(ZkClient zkClient, String clusterName,
+      Boolean enabled) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
+    clusterConfig.setPersistBestPossibleAssignment(enabled);
+    configAccessor.setClusterConfig(clusterName, clusterConfig);
+  }
+
+  protected void enablePersistIntermediateAssignment(ZkClient zkClient, String clusterName,
+      Boolean enabled) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
+    clusterConfig.setPersistIntermediateAssignment(enabled);
+    configAccessor.setClusterConfig(clusterName, clusterConfig);
+  }
+
+  protected void enableTopologyAwareRebalance(ZkClient zkClient, String clusterName,
+      Boolean enabled) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
+    clusterConfig.setTopologyAwareEnabled(enabled);
+    configAccessor.setClusterConfig(clusterName, clusterConfig);
+  }
+
+  protected void enableDelayRebalanceInCluster(ZkClient zkClient, String clusterName,
+      boolean enabled) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
+    clusterConfig.setDelayRebalaceEnabled(enabled);
+    configAccessor.setClusterConfig(clusterName, clusterConfig);
+  }
+
+  protected void enableDelayRebalanceInInstance(ZkClient zkClient, String clusterName,
+      String instanceName, boolean enabled) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    InstanceConfig instanceConfig = configAccessor.getInstanceConfig(clusterName, instanceName);
+    instanceConfig.setDelayRebalanceEnabled(enabled);
+    configAccessor.setInstanceConfig(clusterName, instanceName, instanceConfig);
+  }
+
+  protected void setDelayTimeInCluster(ZkClient zkClient, String clusterName, long delay) {
+    ConfigAccessor configAccessor = new ConfigAccessor(zkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(clusterName);
+    clusterConfig.setRebalanceDelayTime(delay);
+    configAccessor.setClusterConfig(clusterName, clusterConfig);
+  }
+
+  protected IdealState createResourceWithDelayedRebalance(String clusterName, String db,
+      String stateModel, int numPartition, int replica, int minActiveReplica, long delay) {
+    return createResourceWithDelayedRebalance(clusterName, db, stateModel, numPartition, replica,
+        minActiveReplica, delay, AutoRebalanceStrategy.class.getName());
+  }
+
+  protected IdealState createResourceWithDelayedRebalance(String clusterName, String db,
+      String stateModel, int numPartition, int replica, int minActiveReplica, long delay,
+      String rebalanceStrategy) {
+    IdealState idealState =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
+    if (idealState == null) {
+      _gSetupTool.addResourceToCluster(clusterName, db, numPartition, stateModel,
+          IdealState.RebalanceMode.FULL_AUTO + "", rebalanceStrategy);
+    }
+
+    idealState =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
+    idealState.setMinActiveReplicas(minActiveReplica);
+    if (!idealState.isDelayRebalanceEnabled()) {
+      idealState.setDelayRebalanceEnabled(true);
+    }
+    if (delay > 0) {
+      idealState.setRebalanceDelay(delay);
+    }
+    idealState.setRebalancerClassName(DelayedAutoRebalancer.class.getName());
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(clusterName, db, idealState);
+    _gSetupTool.rebalanceStorageCluster(clusterName, db, replica);
+    idealState =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
+
+    return idealState;
+  }
+
+  protected IdealState createIdealState(String resourceGroupName, String instanceGroupTag,
+      List<String> instanceNames, int numPartition, int replica, String rebalanceMode,
+      String stateModelDef) {
+    IdealState is = _gSetupTool
+        .createIdealStateForResourceGroup(resourceGroupName, instanceGroupTag, numPartition,
+            replica, rebalanceMode, stateModelDef);
+
+    // setup initial partition->instance mapping.
+    int nodeIdx = 0;
+    int numNode = instanceNames.size();
+    assert (numNode >= replica);
+    for (int i = 0; i < numPartition; i++) {
+      String partitionName = resourceGroupName + "_" + i;
+      for (int j = 0; j < replica; j++) {
+        is.setPartitionState(partitionName, instanceNames.get((nodeIdx + j) % numNode),
+            OnlineOfflineSMD.States.ONLINE.toString());
+      }
+      nodeIdx++;
+    }
+
+    return is;
+  }
+
+  protected void createDBInSemiAuto(ClusterSetup clusterSetup, String clusterName, String dbName,
+      List<String> preferenceList, String stateModelDef, int numPartition, int replica) {
+    clusterSetup.addResourceToCluster(clusterName, dbName, numPartition, stateModelDef,
+        IdealState.RebalanceMode.SEMI_AUTO.toString());
+    clusterSetup.rebalanceStorageCluster(clusterName, dbName, replica);
+
+    IdealState is =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, dbName);
+    for (String p : is.getPartitionSet()) {
+      is.setPreferenceList(p, preferenceList);
+    }
+    clusterSetup.getClusterManagementTool().setResourceIdealState(clusterName, dbName, is);
+  }
+
+  /**
+   * Validate there should be always minimal active replica and top state replica for each partition.
+   * Also make sure there is always some partitions with only active replica count.
+   */
+  protected void validateMinActiveAndTopStateReplica(IdealState is, ExternalView ev,
+      int minActiveReplica, int numNodes) {
+    StateModelDefinition stateModelDef =
+        BuiltInStateModelDefinitions.valueOf(is.getStateModelDefRef()).getStateModelDefinition();
+    String topState = stateModelDef.getStatesPriorityList().get(0);
+    int replica = Integer.valueOf(is.getReplicas());
+
+    Map<String, Integer> stateCount =
+        stateModelDef.getStateCountMap(numNodes, replica);
+    Set<String> activeStates = stateCount.keySet();
+
+    for (String partition : is.getPartitionSet()) {
+      Map<String, String> assignmentMap = ev.getRecord().getMapField(partition);
+      Assert.assertNotNull(assignmentMap,
+          is.getResourceName() + "'s best possible assignment is null for partition " + partition);
+      Assert.assertTrue(!assignmentMap.isEmpty(),
+          is.getResourceName() + "'s partition " + partition + " has no best possible map in IS.");
+
+      boolean hasTopState = false;
+      int activeReplica = 0;
+      for (String state : assignmentMap.values()) {
+        if (topState.equalsIgnoreCase(state)) {
+          hasTopState = true;
+        }
+        if (activeStates.contains(state)) {
+          activeReplica++;
+        }
+      }
+
+      if (activeReplica < minActiveReplica) {
+        int a = 0;
+      }
+
+      Assert.assertTrue(hasTopState, String.format("%s missing %s replica", partition, topState));
+      Assert.assertTrue(activeReplica >= minActiveReplica, String
+          .format("%s has less active replica %d then required %d", partition, activeReplica,
+              minActiveReplica));
+    }
+  }
+
+  protected void runStage(HelixManager manager, ClusterEvent event, Stage stage) throws Exception {
+    event.addAttribute(AttributeName.helixmanager.name(), manager);
+    StageContext context = new StageContext();
+    stage.init(context);
+    stage.preProcess();
+
+    // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in
+    // execute() function call
+    if (stage instanceof AbstractAsyncBaseStage) {
+      ((AbstractAsyncBaseStage) stage).execute(event);
+    } else {
+      stage.process(event);
+    }
+    stage.postProcess();
+  }
+
+  public void verifyInstance(ZkClient zkClient, String clusterName, String instance,
+      boolean wantExists) {
+    // String instanceConfigsPath = HelixUtil.getConfigPath(clusterName);
+    String instanceConfigsPath = PropertyPathBuilder.instanceConfig(clusterName);
+    String instanceConfigPath = instanceConfigsPath + "/" + instance;
+    String instancePath = PropertyPathBuilder.instance(clusterName, instance);
+    AssertJUnit.assertEquals(wantExists, zkClient.exists(instanceConfigPath));
+    AssertJUnit.assertEquals(wantExists, zkClient.exists(instancePath));
+  }
+
+  public void verifyResource(ZkClient zkClient, String clusterName, String resource,
+      boolean wantExists) {
+    String resourcePath = PropertyPathBuilder.idealState(clusterName, resource);
+    AssertJUnit.assertEquals(wantExists, zkClient.exists(resourcePath));
+  }
+
+  public void verifyEnabled(ZkClient zkClient, String clusterName, String instance,
+      boolean wantEnabled) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    InstanceConfig config = accessor.getProperty(keyBuilder.instanceConfig(instance));
+    AssertJUnit.assertEquals(wantEnabled, config.getInstanceEnabled());
+  }
+
+  public void verifyReplication(ZkClient zkClient, String clusterName, String resource, int repl) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resource));
+    for (String partitionName : idealState.getPartitionSet()) {
+      if (idealState.getRebalanceMode() == IdealState.RebalanceMode.SEMI_AUTO) {
+        AssertJUnit.assertEquals(repl, idealState.getPreferenceList(partitionName).size());
+      } else if (idealState.getRebalanceMode() == IdealState.RebalanceMode.CUSTOMIZED) {
+        AssertJUnit.assertEquals(repl, idealState.getInstanceStateMap(partitionName).size());
+      }
+    }
+  }
+
+  protected void simulateSessionExpiry(ZkConnection zkConnection) throws IOException,
+      InterruptedException {
+    ZooKeeper oldZookeeper = zkConnection.getZookeeper();
+    LOG.info("Old sessionId = " + oldZookeeper.getSessionId());
+
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        LOG.info("In New connection, process event:" + event);
+      }
+    };
+
+    ZooKeeper newZookeeper =
+        new ZooKeeper(zkConnection.getServers(), oldZookeeper.getSessionTimeout(), watcher,
+            oldZookeeper.getSessionId(), oldZookeeper.getSessionPasswd());
+    LOG.info("New sessionId = " + newZookeeper.getSessionId());
+    // Thread.sleep(3000);
+    newZookeeper.close();
+    Thread.sleep(10000);
+    oldZookeeper = zkConnection.getZookeeper();
+    LOG.info("After session expiry sessionId = " + oldZookeeper.getSessionId());
+  }
+
+  protected void simulateSessionExpiry(ZkClient zkClient)
+      throws IOException, InterruptedException, IOException {
+    IZkStateListener listener = new IZkStateListener() {
+      @Override
+      public void handleStateChanged(Watcher.Event.KeeperState state) throws Exception {
+        LOG.info("In Old connection, state changed:" + state);
+      }
+
+      @Override
+      public void handleNewSession() throws Exception {
+        LOG.info("In Old connection, new session");
+      }
+
+      @Override
+      public void handleSessionEstablishmentError(Throwable var1) throws Exception {
+      }
+    };
+    zkClient.subscribeStateChanges(listener);
+    ZkConnection connection = ((ZkConnection) zkClient.getConnection());
+    ZooKeeper oldZookeeper = connection.getZookeeper();
+    LOG.info("Old sessionId = " + oldZookeeper.getSessionId());
+
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        LOG.info("In New connection, process event:" + event);
+      }
+    };
+
+    ZooKeeper newZookeeper =
+        new ZooKeeper(connection.getServers(), oldZookeeper.getSessionTimeout(), watcher,
+            oldZookeeper.getSessionId(), oldZookeeper.getSessionPasswd());
+    LOG.info("New sessionId = " + newZookeeper.getSessionId());
+    // Thread.sleep(3000);
+    newZookeeper.close();
+    Thread.sleep(10000);
+    connection = (ZkConnection) zkClient.getConnection();
+    oldZookeeper = connection.getZookeeper();
+    LOG.info("After session expiry sessionId = " + oldZookeeper.getSessionId());
+  }
+
+  protected void setupStateModel(String clusterName) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    StateModelDefinition masterSlave =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave());
+    accessor.setProperty(keyBuilder.stateModelDef(masterSlave.getId()), masterSlave);
+
+    StateModelDefinition leaderStandby =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby());
+    accessor.setProperty(keyBuilder.stateModelDef(leaderStandby.getId()), leaderStandby);
+
+    StateModelDefinition onlineOffline =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline());
+    accessor.setProperty(keyBuilder.stateModelDef(onlineOffline.getId()), onlineOffline);
+
+  }
+
+  protected Message createMessage(Message.MessageType type, String msgId, String fromState, String toState,
+      String resourceName, String tgtName) {
+    Message msg = new Message(type.toString(), msgId);
+    msg.setFromState(fromState);
+    msg.setToState(toState);
+    msg.getRecord().setSimpleField(Message.Attributes.RESOURCE_NAME.toString(), resourceName);
+    msg.setTgtName(tgtName);
+    return msg;
+  }
+
+  protected List<IdealState> setupIdealState(String clusterName, int[] nodes, String[] resources,
+      int partitions, int replicas) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    List<IdealState> idealStates = new ArrayList<IdealState>();
+    List<String> instances = new ArrayList<String>();
+    for (int i : nodes) {
+      instances.add("localhost_" + i);
+    }
+
+    for (String resourceName : resources) {
+      IdealState idealState = new IdealState(resourceName);
+      for (int p = 0; p < partitions; p++) {
+        List<String> value = new ArrayList<String>();
+        for (int r = 0; r < replicas; r++) {
+          int n = nodes[(p + r) % nodes.length];
+          value.add("localhost_" + n);
+        }
+        idealState.getRecord().setListField(resourceName + "_" + p, value);
+      }
+
+      idealState.setReplicas(Integer.toString(replicas));
+      idealState.setStateModelDefRef("MasterSlave");
+      idealState.setRebalanceMode(IdealState.RebalanceMode.SEMI_AUTO);
+      idealState.setNumPartitions(partitions);
+      idealStates.add(idealState);
+
+      // System.out.println(idealState);
+      accessor.setProperty(keyBuilder.idealStates(resourceName), idealState);
+    }
+    return idealStates;
+  }
+
+  protected void setupLiveInstances(String clusterName, int[] liveInstances) {
+    ZKHelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    for (int i = 0; i < liveInstances.length; i++) {
+      String instance = "localhost_" + liveInstances[i];
+      LiveInstance liveInstance = new LiveInstance(instance);
+      liveInstance.setSessionId("session_" + liveInstances[i]);
+      liveInstance.setHelixVersion("0.0.0");
+      accessor.setProperty(keyBuilder.liveInstance(instance), liveInstance);
+    }
+  }
+
+  protected void setupInstances(String clusterName, int[] instances) {
+    HelixAdmin admin = new ZKHelixAdmin(_gZkClient);
+    for (int i = 0; i < instances.length; i++) {
+      String instance = "localhost_" + instances[i];
+      InstanceConfig instanceConfig = new InstanceConfig(instance);
+      instanceConfig.setHostName("localhost");
+      instanceConfig.setPort("" + instances[i]);
+      instanceConfig.setInstanceEnabled(true);
+      admin.addInstance(clusterName, instanceConfig);
+    }
+  }
+
+  protected void runPipeline(ClusterEvent event, Pipeline pipeline) {
+    try {
+      pipeline.handle(event);
+      pipeline.finish();
+    } catch (Exception e) {
+      LOG.error("Exception while executing pipeline:" + pipeline
+          + ". Will not continue to next pipeline", e);
+    }
+  }
+
+  protected void runStage(ClusterEvent event, Stage stage) throws Exception {
+    StageContext context = new StageContext();
+    stage.init(context);
+    stage.preProcess();
+
+    // AbstractAsyncBaseStage will run asynchronously, and it's main logics are implemented in
+    // execute() function call
+    // TODO (harry): duplicated code in ZkIntegrationTestBase, consider moving runStage()
+    // to a shared library
+    if (stage instanceof AbstractAsyncBaseStage) {
+      ((AbstractAsyncBaseStage) stage).execute(event);
+    } else {
+      stage.process(event);
+    }
+    stage.postProcess();
+  }
+
+  /**
+   * Poll for the existence (or lack thereof) of a specific Helix property
+   * @param clazz the HelixProeprty subclass
+   * @param accessor connected HelixDataAccessor
+   * @param key the property key to look up
+   * @param shouldExist true if the property should exist, false otherwise
+   * @return the property if found, or null if it does not exist
+   */
+  protected <T extends HelixProperty> T pollForProperty(Class<T> clazz, HelixDataAccessor accessor,
+      PropertyKey key, boolean shouldExist) throws InterruptedException {
+    final int POLL_TIMEOUT = 5000;
+    final int POLL_INTERVAL = 50;
+    T property = accessor.getProperty(key);
+    int timeWaited = 0;
+    while (((shouldExist && property == null) || (!shouldExist && property != null))
+        && timeWaited < POLL_TIMEOUT) {
+      Thread.sleep(POLL_INTERVAL);
+      timeWaited += POLL_INTERVAL;
+      property = accessor.getProperty(key);
+    }
+    return property;
+  }
+
+  /**
+   * Ensures that external view and current state are empty
+   */
+  protected static class EmptyZkVerifier implements ClusterStateVerifier.ZkVerifier {
+    private final String _clusterName;
+    private final String _resourceName;
+    private final ZkClient _zkClient;
+
+    /**
+     * Instantiate the verifier
+     * @param clusterName the cluster to verify
+     * @param resourceName the resource to verify
+     */
+    public EmptyZkVerifier(String clusterName, String resourceName) {
+      _clusterName = clusterName;
+      _resourceName = resourceName;
+      _zkClient = ZKClientPool.getZkClient(ZK_ADDR);
+    }
+
+    @Override
+    public boolean verify() {
+      BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
+      HelixDataAccessor accessor = new ZKHelixDataAccessor(_clusterName, baseAccessor);
+      PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+      ExternalView externalView = accessor.getProperty(keyBuilder.externalView(_resourceName));
+
+      // verify external view empty
+      if (externalView != null) {
+        for (String partition : externalView.getPartitionSet()) {
+          Map<String, String> stateMap = externalView.getStateMap(partition);
+          if (stateMap != null && !stateMap.isEmpty()) {
+            LOG.error("External view not empty for " + partition);
+            return false;
+          }
+        }
+      }
+
+      // verify current state empty
+      List<String> liveParticipants = accessor.getChildNames(keyBuilder.liveInstances());
+      for (String participant : liveParticipants) {
+        List<String> sessionIds = accessor.getChildNames(keyBuilder.sessions(participant));
+        for (String sessionId : sessionIds) {
+          CurrentState currentState =
+              accessor.getProperty(keyBuilder.currentState(participant, sessionId, _resourceName));
+          Map<String, String> partitionStateMap = currentState.getPartitionStateMap();
+          if (partitionStateMap != null && !partitionStateMap.isEmpty()) {
+            LOG.error("Current state not empty for " + participant);
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public ZkClient getZkClient() {
+      return _zkClient;
+    }
+
+    @Override
+    public String getClusterName() {
+      return _clusterName;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
index 9054a1d..ab62676 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
@@ -119,8 +119,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     Assert.assertEquals(msgThrottleOutput.getMessages("TestDB", new Partition("TestDB_0")).size(),
         1);
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test()
@@ -310,8 +310,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     Assert.assertTrue(throttleMessages.contains(msg3));
     Assert.assertTrue(throttleMessages.contains(msg4));
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   private boolean containsConstraint(Set<ConstraintItem> constraints, ConstraintItem constraint) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
index d38907d..90363c1 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
@@ -118,8 +118,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node1");
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test
@@ -227,8 +227,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     messages = accessor.getChildNames(keyBuilder.messages("localhost_0"));
     Assert.assertTrue(messages.isEmpty());
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test
@@ -323,6 +323,7 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     Assert.assertEquals(message.getToState(), "DROPPED");
     Assert.assertEquals(message.getTgtName(), "localhost_0");
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -398,8 +399,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     messages = msgSelOutput.getMessages(resourceName, new Partition(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node0");
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test
@@ -464,6 +465,7 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     Assert.assertEquals(message.getToState(), "SLAVE");
     Assert.assertEquals(message.getTgtName(), "localhost_1");
 
+    deleteCluster(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java b/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
index 5701211..00509b3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
@@ -25,7 +25,7 @@ import java.util.Date;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -41,7 +41,7 @@ import org.testng.annotations.Test;
  * which helps us write integration tests easily
  */
 
-public class SinglePartitionLeaderStandByTest extends ZkIntegrationTestBase {
+public class SinglePartitionLeaderStandByTest extends ZkTestBase {
   @Test
   public void test()
       throws Exception {
@@ -99,6 +99,7 @@ public class SinglePartitionLeaderStandByTest extends ZkIntegrationTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
index 8be880d..6aca358 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
@@ -21,7 +21,7 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
@@ -33,7 +33,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-public class TestAddClusterV2 extends ZkIntegrationTestBase {
+public class TestAddClusterV2 extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestAddClusterV2.class);
 
   protected static final int CLUSTER_NR = 10;
@@ -54,18 +54,6 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CONTROLLER_CLUSTER;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-
-    for (int i = 0; i < CLUSTER_NR; i++) {
-      namespace = "/" + CLUSTER_PREFIX + "_" + CLASS_NAME + "_" + i;
-      if (_gZkClient.exists(namespace)) {
-        _gZkClient.deleteRecursively(namespace);
-      }
-    }
-
     _setupTool = new ClusterSetup(ZK_ADDR);
 
     // setup CONTROLLER_CLUSTER
@@ -136,6 +124,13 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase {
     for (int i = 0; i < NODE_NR; i++) {
       _participants[i].syncStop();
     }
+
+    // delete clusters
+    for (int i = 0; i < CLUSTER_NR; i++) {
+      String clusterName = CLUSTER_PREFIX + "_" + CLASS_NAME + "_" + i;
+      _setupTool.getClusterManagementTool().dropCluster(clusterName);
+    }
+
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
index 7e83be2..1d5273a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.ClusterDistributedController;
 import org.apache.helix.integration.manager.MockParticipantManager;
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
+public class TestAddNodeAfterControllerStart extends ZkTestBase {
   private static Logger LOG = LoggerFactory.getLogger(TestAddNodeAfterControllerStart.class);
   final String className = getShortClassName();
 
@@ -88,6 +88,7 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     for (int i = 0; i < nodeNr; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -164,6 +165,7 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     for (int i = 0; i < nodeNr; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
index b05b9fb..6209815 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -39,7 +39,7 @@ import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase {
+public class TestAddStateModelFactoryAfterConnect extends ZkTestBase {
   @Test
   public void testBasic() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);
@@ -130,6 +130,7 @@ public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java b/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
index 946eb1c..cd53552 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
@@ -68,17 +68,12 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
 
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
 
     // start controller
@@ -108,19 +103,19 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
 
   @Test (enabled = false)
   public void testParticipantUnavailable() {
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
     HelixClusterVerifier verifier =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(new HashSet<>(Collections.singleton(testDb))).build();
     Assert.assertTrue(verifier.verify());
 
     // disable then enable the resource to ensure no rebalancing error is generated during this process
-    _setupTool.dropResourceFromCluster(CLUSTER_NAME, testDb);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, testDb);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
     Assert.assertTrue(verifier.verify());
 
     // Verify there is no rebalance error logged
@@ -137,7 +132,7 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     checkRebalanceFailureGauge(true);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
     for (int i = 0; i < NODE_NR; i++) {
       _participants[i] =
           new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _participants[i].getInstanceName());
@@ -147,21 +142,21 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
 
   @Test (enabled = false)
   public void testTagSetIncorrect() {
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
     // set expected instance tag
     IdealState is =
-        _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, testDb);
     is.setInstanceGroupTag("RandomTag");
-    _setupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, is);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, is);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
 
     // Verify there is rebalance error logged
     Assert.assertNotNull(pollForError(accessor, errorNodeKey));
     checkRebalanceFailureGauge(true);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
   }
 
   @Test (enabled = false)
@@ -188,10 +183,10 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     // Error may be recorded unexpectedly when a resource from other tests is not cleaned up.
     accessor.removeProperty(errorNodeKey);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name(),
         CrushRebalanceStrategy.class.getName());
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, replicas);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, replicas);
     HelixClusterVerifier verifier =
         new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(new HashSet<>(Collections.singleton(testDb))).build();
@@ -212,13 +207,13 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     for (int i = replicas; i < NODE_NR; i++) {
       setDomainId(_participants[i].getInstanceName(), configAccessor);
     }
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, replicas);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, replicas);
     Thread.sleep(1000);
     // Verify that rebalance error state is removed
     checkRebalanceFailureGauge(false);
 
     // clean up
-    _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
     clusterConfig.setTopologyAwareEnabled(false);
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java b/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
index 0c61b27..2fa8c9f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
@@ -46,8 +46,8 @@ public class TestBasicSpectator extends ZkStandAloneCMTestBase implements
     relayHelixManager.connect();
     relayHelixManager.addExternalViewChangeListener(this);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, "NextDB", 64, STATE_MODEL);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "NextDB", 3);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, "NextDB", 64, STATE_MODEL);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, "NextDB", 3);
 
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageHandling.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageHandling.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageHandling.java
index 3feb2e0..d7c3a2d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageHandling.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageHandling.java
@@ -54,15 +54,15 @@ public class TestBatchMessageHandling extends ZkStandAloneCMTestBase {
     IdealState idealState = new FullAutoModeISBuilder(dbName).setStateModel("OnlineOffline")
         .setStateModelFactoryName("TestFactory").setNumPartitions(10).setNumReplica(1).build();
     idealState.setBatchMessageMode(true);
-    _setupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
+    _gSetupTool.getClusterManagementTool().addResource(CLUSTER_NAME, dbName, idealState);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, 1);
 
     Thread.sleep(1000L);
 
     int numOfOnlines = 0;
     int numOfErrors = 0;
     ExternalView externalView =
-        _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, dbName);
+        _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, dbName);
     for (String partition : externalView.getPartitionSet()) {
       if (externalView.getStateMap(partition).values().contains("ONLINE")) {
         numOfOnlines++;

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
index 1b764d4..7c90e5f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
@@ -30,7 +30,7 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.NotificationContext.Type;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -45,7 +45,7 @@ import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestBucketizedResource extends ZkIntegrationTestBase {
+public class TestBucketizedResource extends ZkTestBase {
 
   private void setupCluster(String clusterName, List<String> instanceNames, String dbName,
       int replica, int partitions, int bucketSize) {

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestCMWithFailParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCMWithFailParticipant.java b/helix-core/src/test/java/org/apache/helix/integration/TestCMWithFailParticipant.java
index 3c3f4ac..7dd477a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCMWithFailParticipant.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCMWithFailParticipant.java
@@ -21,10 +21,10 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.testng.annotations.Test;
 
-public class TestCMWithFailParticipant extends ZkIntegrationTestBase {
+public class TestCMWithFailParticipant extends ZkTestBase {
   // ZkClient _zkClient;
   //
   // @BeforeClass ()

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java b/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java
index 05b3e99..e01afc9 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCarryOverBadCurState.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -33,7 +33,7 @@ import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class TestCarryOverBadCurState extends ZkIntegrationTestBase {
+public class TestCarryOverBadCurState extends ZkTestBase {
   @Test
   public void testCarryOverBadCurState() throws Exception {
     System.out.println("START testCarryOverBadCurState at " + new Date(System.currentTimeMillis()));
@@ -83,7 +83,7 @@ public class TestCarryOverBadCurState extends ZkIntegrationTestBase {
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
+    _gSetupTool.deleteCluster(clusterName);
     System.out.println("END testCarryOverBadCurState at " + new Date(System.currentTimeMillis()));
-
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
index e7a7ea0..61aadde 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
@@ -98,8 +98,8 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
     accessor.removeProperty(keyBuilder.currentState("localhost_12918", liveInstance.getSessionId(),
         "TestDB0"));
     liveInstance = accessor.getProperty(keyBuilder.liveInstance("localhost_12919"));
-    accessor.removeProperty(keyBuilder.currentState("localhost_12919", liveInstance.getSessionId(),
-        "TestDB0"));
+    accessor.removeProperty(
+        keyBuilder.currentState("localhost_12919", liveInstance.getSessionId(), "TestDB0"));
 
     // re-enable controller shall remove orphan external-view
     // System.out.println("re-enabling controller");
@@ -123,6 +123,7 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
+    TestHelper.dropCluster(clusterName, _gZkClient);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/c0d5792b/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java b/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java
index 21a9ac2..9d5ad3e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestClusterStartsup.java
@@ -38,20 +38,14 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
   void setupCluster() throws HelixException {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursively(namespace);
-    }
-    _setupTool = new ClusterSetup(ZK_ADDR);
-
     // setup storage cluster
-    _setupTool.addCluster(CLUSTER_NAME, true);
-    _setupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, 20, STATE_MODEL);
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, 20, STATE_MODEL);
     for (int i = 0; i < NODE_NR; i++) {
       String storageNodeName = "localhost_" + (START_PORT + i);
-      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
     }
-    _setupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, 3);
   }
 
   @Override
@@ -60,11 +54,6 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
 
   }
 
-  @Override
-  @AfterClass()
-  public void afterClass() {
-  }
-
   @Test()
   public void testParticipantStartUp() throws Exception {
     setupCluster();
@@ -135,5 +124,8 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
       AssertJUnit.assertFalse(manager.isConnected());
     }
 
+    if (manager != null) {
+      manager.disconnect();
+    }
   }
 }