You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by zz...@apache.org on 2014/08/04 22:19:21 UTC

[4/7] [HELIX-376] Remove HelixConnection/HelixManager duplicate code

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java b/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java
index 3e69327..0069f3c 100644
--- a/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/TestConfigAccessor.java
@@ -23,10 +23,10 @@ import java.util.Date;
 import java.util.List;
 
 import org.apache.helix.manager.zk.ZKHelixAdmin;
-import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.testutil.ZkTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -44,72 +44,80 @@ public class TestConfigAccessor extends ZkTestBase {
         "MasterSlave", true);
 
     ConfigAccessor configAccessor = new ConfigAccessor(_zkclient);
-    ConfigScope clusterScope = new ConfigScopeBuilder().forCluster(clusterName).build();
+    HelixConfigScope clusterScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build();
 
     // cluster scope config
     String clusterConfigValue = configAccessor.get(clusterScope, "clusterConfigKey");
     Assert.assertNull(clusterConfigValue);
 
-    configAccessor.set(clusterScope, "clusterConfigKey", "clusterConfigValue");
-    clusterConfigValue = configAccessor.get(clusterScope, "clusterConfigKey");
-    Assert.assertEquals(clusterConfigValue, "clusterConfigValue");
+    for (int i = 0; i < 2; i++) {
+      configAccessor.set(clusterScope, "clusterConfigKey" + i, "clusterConfigValue" + i);
+      clusterConfigValue = configAccessor.get(clusterScope, "clusterConfigKey" + i);
+      Assert.assertEquals(clusterConfigValue, "clusterConfigValue" + i);
+    }
 
     // resource scope config
-    ConfigScope resourceScope =
-        new ConfigScopeBuilder().forCluster(clusterName).forResource("testResource").build();
+    HelixConfigScope resourceScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE).forCluster(clusterName)
+            .forResource("testResource").build();
     configAccessor.set(resourceScope, "resourceConfigKey", "resourceConfigValue");
     String resourceConfigValue = configAccessor.get(resourceScope, "resourceConfigKey");
     Assert.assertEquals(resourceConfigValue, "resourceConfigValue");
 
     // partition scope config
-    ConfigScope partitionScope =
-        new ConfigScopeBuilder().forCluster(clusterName).forResource("testResource")
-            .forPartition("testPartition").build();
+    HelixConfigScope partitionScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTITION).forCluster(clusterName)
+            .forResource("testResource").forPartition("testPartition").build();
     configAccessor.set(partitionScope, "partitionConfigKey", "partitionConfigValue");
     String partitionConfigValue = configAccessor.get(partitionScope, "partitionConfigKey");
     Assert.assertEquals(partitionConfigValue, "partitionConfigValue");
 
     // participant scope config
-    ConfigScope participantScope =
-        new ConfigScopeBuilder().forCluster(clusterName).forParticipant("localhost_12918").build();
+    HelixConfigScope participantScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT).forCluster(clusterName)
+            .forParticipant("localhost_12918").build();
     configAccessor.set(participantScope, "participantConfigKey", "participantConfigValue");
     String participantConfigValue = configAccessor.get(participantScope, "participantConfigKey");
     Assert.assertEquals(participantConfigValue, "participantConfigValue");
 
-    List<String> keys = configAccessor.getKeys(ConfigScopeProperty.RESOURCE, clusterName);
+    HelixConfigScope partialResourceScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE).forCluster(clusterName).build();
+    List<String> keys = configAccessor.getKeys(partialResourceScope);
     Assert.assertEquals(keys.size(), 1, "should be [testResource]");
     Assert.assertEquals(keys.get(0), "testResource");
 
-    keys = configAccessor.getKeys(ConfigScopeProperty.CLUSTER, clusterName);
-    Assert.assertEquals(keys.size(), 1, "should be [" + clusterName + "]");
-    Assert.assertEquals(keys.get(0), clusterName);
+    keys = configAccessor.getKeys(clusterScope);
+    Assert.assertEquals(keys.size(), 2, "should be [clusterConfigKey0, clusterConfigKey1]");
+    Assert.assertEquals(keys.get(0), "clusterConfigKey0");
+    Assert.assertEquals(keys.get(1), "clusterConfigKey1");
 
-    keys = configAccessor.getKeys(ConfigScopeProperty.PARTICIPANT, clusterName);
+    HelixConfigScope partialParticipantScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT).forCluster(clusterName)
+            .build();
+    keys = configAccessor.getKeys(partialParticipantScope);
     Assert.assertEquals(keys.size(), 5, "should be [localhost_12918~22] sorted");
     Assert.assertEquals(keys.get(0), "localhost_12918");
     Assert.assertEquals(keys.get(4), "localhost_12922");
 
-    keys = configAccessor.getKeys(ConfigScopeProperty.PARTITION, clusterName, "testResource");
+    HelixConfigScope partialPartitionScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTITION).forCluster(clusterName)
+            .forResource("testResource").build();
+    keys = configAccessor.getKeys(partialPartitionScope);
     Assert.assertEquals(keys.size(), 1, "should be [testPartition]");
     Assert.assertEquals(keys.get(0), "testPartition");
 
-    keys = configAccessor.getKeys(ConfigScopeProperty.RESOURCE, clusterName, "testResource");
+    keys = configAccessor.getKeys(resourceScope);
     Assert.assertEquals(keys.size(), 1, "should be [resourceConfigKey]");
     Assert.assertEquals(keys.get(0), "resourceConfigKey");
 
-    keys = configAccessor.getKeys(ConfigScopeProperty.CLUSTER, clusterName, clusterName);
-    Assert.assertEquals(keys.size(), 1, "should be [clusterConfigKey]");
-    Assert.assertEquals(keys.get(0), "clusterConfigKey");
-
-    keys = configAccessor.getKeys(ConfigScopeProperty.PARTICIPANT, clusterName, "localhost_12918");
+    keys = configAccessor.getKeys(participantScope);
     System.out.println((keys));
     Assert.assertEquals(keys.size(), 4,
         "should be [HELIX_ENABLED, HELIX_HOST, HELIX_PORT, participantConfigKey]");
     Assert.assertEquals(keys.get(3), "participantConfigKey");
 
-    keys =
-        configAccessor.getKeys(ConfigScopeProperty.PARTITION, clusterName, "testResource",
-            "testPartition");
+    keys = configAccessor.getKeys(partitionScope);
     Assert.assertEquals(keys.size(), 1, "should be [partitionConfigKey]");
     Assert.assertEquals(keys.get(0), "partitionConfigKey");
 
@@ -132,21 +140,24 @@ public class TestConfigAccessor extends ZkTestBase {
 
     // negative tests
     try {
-      new ConfigScopeBuilder().forPartition("testPartition").build();
+      new HelixConfigScopeBuilder(ConfigScopeProperty.PARTITION).forPartition("testPartition")
+          .build();
       Assert.fail("Should fail since cluster name is not set");
     } catch (Exception e) {
       // OK
     }
 
     try {
-      new ConfigScopeBuilder().forCluster("testCluster").forPartition("testPartition").build();
+      new HelixConfigScopeBuilder(ConfigScopeProperty.PARTITION).forCluster("testCluster")
+          .forPartition("testPartition").build();
       Assert.fail("Should fail since resource name is not set");
     } catch (Exception e) {
       // OK
     }
 
     try {
-      new ConfigScopeBuilder().forParticipant("testParticipant").build();
+      new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT)
+          .forParticipant("testParticipant").build();
       Assert.fail("Should fail since cluster name is not set");
     } catch (Exception e) {
       // OK
@@ -168,8 +179,9 @@ public class TestConfigAccessor extends ZkTestBase {
     ZKHelixAdmin admin = new ZKHelixAdmin(_zkclient);
     admin.addCluster(clusterName, true);
     ConfigAccessor configAccessor = new ConfigAccessor(_zkclient);
-    ConfigScope participantScope =
-        new ConfigScopeBuilder().forCluster(clusterName).forParticipant("localhost_12918").build();
+    HelixConfigScope participantScope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT).forCluster(clusterName)
+            .forParticipant("localhost_12918").build();
 
     try {
       configAccessor.set(participantScope, "participantConfigKey", "participantConfigValue");

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/TestHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestHelper.java b/helix-core/src/test/java/org/apache/helix/TestHelper.java
index 8328a15..879e727 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelper.java
@@ -48,12 +48,12 @@ import org.apache.helix.api.id.MessageId;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.integration.manager.ZkTestManager;
 import org.apache.helix.manager.zk.CallbackHandler;
 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.ZkCallbackHandler;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
@@ -754,24 +754,22 @@ public class TestHelper {
     } while (true);
   }
 
-  // debug code
-  public static String printHandlers(ZkTestManager manager) {
+  public static void printHandlers(HelixManager manager, List<ZkCallbackHandler> handlers) {
     StringBuilder sb = new StringBuilder();
-    List<CallbackHandler> handlers = manager.getHandlers();
-    sb.append(manager.getInstanceName() + " has " + handlers.size() + " cb-handlers. [");
+    sb.append(manager.getInstanceName() + " has " + handlers.size() + " cb-handlers. [\n");
 
     for (int i = 0; i < handlers.size(); i++) {
-      CallbackHandler handler = handlers.get(i);
+      ZkCallbackHandler handler = handlers.get(i);
       String path = handler.getPath();
       sb.append(path.substring(manager.getClusterName().length() + 1) + ": "
           + handler.getListener());
       if (i < (handlers.size() - 1)) {
-        sb.append(", ");
+        sb.append("\n");
       }
     }
     sb.append("]");
 
-    return sb.toString();
+    System.out.println(sb.toString());
   }
 
   public static void printZkListeners(ZkClient client) throws Exception {

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
index 50ed0df..d09d887 100644
--- a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
+++ b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
@@ -59,7 +59,7 @@ public class TestZKCallback extends ZkTestBase {
   }
 
   public class TestCallbackListener implements MessageListener, LiveInstanceChangeListener,
-      ConfigChangeListener, CurrentStateChangeListener, ExternalViewChangeListener,
+      InstanceConfigChangeListener, CurrentStateChangeListener, ExternalViewChangeListener,
       IdealStateChangeListener {
     boolean externalViewChangeReceived = false;
     boolean liveInstanceChangeReceived = false;
@@ -81,7 +81,7 @@ public class TestZKCallback extends ZkTestBase {
     }
 
     @Override
-    public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
+    public void onInstanceConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
       configChangeReceived = true;
     }
 
@@ -127,7 +127,7 @@ public class TestZKCallback extends ZkTestBase {
     testHelixManager.addMessageListener(testListener, "localhost_8900");
     testHelixManager.addCurrentStateChangeListener(testListener, "localhost_8900",
         testHelixManager.getSessionId());
-    testHelixManager.addConfigChangeListener(testListener);
+    testHelixManager.addInstanceConfigChangeListener(testListener);
     testHelixManager.addIdealStateChangeListener(testListener);
     testHelixManager.addExternalViewChangeListener(testListener);
     testHelixManager.addLiveInstanceChangeListener(testListener);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
index 444c069..b7140cf 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
@@ -20,7 +20,6 @@ package org.apache.helix;
  */
 
 import java.io.BufferedReader;
-import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.net.Socket;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
index b5d218d..cf57c1e 100644
--- a/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
+++ b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
@@ -38,8 +38,8 @@ import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
 import org.apache.helix.controller.stages.BestPossibleStateOutput;
 import org.apache.helix.controller.stages.ClusterEvent;
 import org.apache.helix.controller.stages.ResourceCurrentState;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ResourceAssignment;
@@ -58,8 +58,8 @@ public class TestNewStages extends ZkTestBase {
   final int n = 2;
   final int p = 8;
   final int r = 2;
-  MockParticipantManager[] _participants = new MockParticipantManager[n];
-  ClusterControllerManager _controller;
+  MockParticipant[] _participants = new MockParticipant[n];
+  MockController _controller;
 
   ClusterId _clusterId;
   HelixDataAccessor _dataAccessor;
@@ -221,14 +221,14 @@ public class TestNewStages extends ZkTestBase {
         r, // replicas
         "MasterSlave", true); // do rebalance
 
-    _controller = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    _controller = new MockController(_zkaddr, clusterName, "controller_0");
     _controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      _participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       _participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
index 73ba122..0081c6c 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
@@ -21,7 +21,6 @@ package org.apache.helix.controller.stages;
 
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -85,12 +84,6 @@ public class DummyClusterManager implements HelixManager {
   }
 
   @Override
-  public void addConfigChangeListener(ConfigChangeListener listener) throws Exception {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
   public void addMessageListener(MessageListener listener, String instanceName) throws Exception {
     // TODO Auto-generated method stub
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 922dde6..3349b4f 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
@@ -33,7 +33,7 @@ import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
 import org.apache.helix.controller.pipeline.Pipeline;
-import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.CurrentState;
@@ -152,8 +152,8 @@ public class TestRebalancePipeline extends ZkTestBase {
         0, 1
     });
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // round1: controller sends O->S to both node0 and node1

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/IntegrationTest.java b/helix-core/src/test/java/org/apache/helix/integration/IntegrationTest.java
index bb862f3..1bb9f57 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/IntegrationTest.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/IntegrationTest.java
@@ -21,6 +21,13 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.TestHelper;
+import org.apache.helix.manager.zk.MockController;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.testutil.TestUtil;
+import org.apache.helix.testutil.ZkTestBase;
+import org.apache.helix.tools.ClusterStateVerifier;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 /**
@@ -28,11 +35,48 @@ import org.testng.annotations.Test;
  * which helps us write integration tests easily
  */
 
-public class IntegrationTest extends ZkStandAloneCMTestBase {
+public class IntegrationTest extends ZkTestBase {
   @Test
-  public void integrationTest() throws Exception {
-    System.out.println("START IntegrationTest at " + new Date(System.currentTimeMillis()));
+  public void test() throws Exception {
+    String clusterName = TestUtil.getTestName();
+    int n = 2;
+
+    System.out.println("START " + clusterName +" at " + new Date(System.currentTimeMillis()));
+
     // Thread.currentThread().join();
-    System.out.println("END IntegrationTest at " + new Date(System.currentTimeMillis()));
+    TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
+        "localhost", // participant name prefix
+        "TestDB", // resource name prefix
+        1, // resources
+        2, // partitions per resource
+        n, // number of nodes
+        2, // replicas
+        "MasterSlave", true); // do rebalance
+
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
+    controller.syncStart();
+
+    // start participants
+    MockParticipant[] participants = new MockParticipant[n];
+    for (int i = 0; i < n; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
+      participants[i].syncStart();
+    }
+
+    boolean result =
+        ClusterStateVerifier.verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(_zkaddr,
+            clusterName));
+
+    Assert.assertTrue(result);
+
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 8dca7c8..5d7df59 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
@@ -23,8 +23,8 @@ import java.util.Date;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.integration.manager.ClusterDistributedController;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockMultiClusterController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.ZkTestBase;
@@ -49,8 +49,8 @@ public class TestAddClusterV2 extends ZkTestBase {
 
   protected static final String TEST_DB = "TestDB";
 
-  MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
-  ClusterDistributedController[] _distControllers = new ClusterDistributedController[NODE_NR];
+  MockParticipant[] _participants = new MockParticipant[NODE_NR];
+  MockMultiClusterController[] _distControllers = new MockMultiClusterController[NODE_NR];
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -90,7 +90,7 @@ public class TestAddClusterV2 extends ZkTestBase {
     // start dummy participants for the first cluster
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(_zkaddr, firstCluster, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, firstCluster, instanceName);
       _participants[i].syncStart();
     }
 
@@ -98,7 +98,7 @@ public class TestAddClusterV2 extends ZkTestBase {
     for (int i = 0; i < NODE_NR; i++) {
       String controllerName = "controller_" + i;
       _distControllers[i] =
-          new ClusterDistributedController(_zkaddr, CONTROLLER_CLUSTER, controllerName);
+          new MockMultiClusterController(_zkaddr, CONTROLLER_CLUSTER, controllerName);
       _distControllers[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 01d760a..97acc1c 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
@@ -26,10 +26,10 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.ClusterDistributedController;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.CallbackHandler;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockMultiClusterController;
+import org.apache.helix.manager.zk.MockController;
+import org.apache.helix.manager.zk.ZkCallbackHandler;
 import org.apache.helix.testutil.TestUtil;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -52,15 +52,15 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, "localhost", "TestDB", 1, 20, nodeNr - 1,
         3, "MasterSlave", true);
 
-    MockParticipantManager[] participants = new MockParticipantManager[nodeNr];
+    MockParticipant[] participants = new MockParticipant[nodeNr];
     for (int i = 0; i < nodeNr - 1; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     boolean result;
@@ -76,7 +76,7 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     _setupTool.addInstanceToCluster(clusterName, "localhost_12922");
     _setupTool.rebalanceStorageCluster(clusterName, "TestDB0", 3);
 
-    participants[nodeNr - 1] = new MockParticipantManager(_zkaddr, clusterName, "localhost_12922");
+    participants[nodeNr - 1] = new MockParticipant(_zkaddr, clusterName, "localhost_12922");
     new Thread(participants[nodeNr - 1]).start();
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -106,8 +106,8 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     TestHelper.setupCluster(grandClusterName, _zkaddr, 0, "controller", null, 0, 0, 1, 0, null,
         true);
 
-    ClusterDistributedController distController =
-        new ClusterDistributedController(_zkaddr, grandClusterName, "controller_0");
+    MockMultiClusterController distController =
+        new MockMultiClusterController(_zkaddr, grandClusterName, "controller_0");
     distController.syncStart();
 
     // setup cluster
@@ -130,10 +130,10 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     _setupTool.addResourceToCluster(clusterName, "TestDB0", 1, "LeaderStandby");
     _setupTool.rebalanceStorageCluster(clusterName, "TestDB0", 1);
 
-    MockParticipantManager[] participants = new MockParticipantManager[nodeNr];
+    MockParticipant[] participants = new MockParticipant[nodeNr];
     for (int i = 0; i < nodeNr - 1; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -152,7 +152,7 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     _setupTool.addInstanceToCluster(clusterName, "localhost_12919");
     _setupTool.rebalanceStorageCluster(clusterName, "TestDB0", 2);
 
-    participants[nodeNr - 1] = new MockParticipantManager(_zkaddr, clusterName, "localhost_12919");
+    participants[nodeNr - 1] = new MockParticipant(_zkaddr, clusterName, "localhost_12919");
     participants[nodeNr - 1].syncStart();
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -173,9 +173,9 @@ public class TestAddNodeAfterControllerStart extends ZkTestBase {
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
-  boolean checkHandlers(List<CallbackHandler> handlers, String path) {
+  boolean checkHandlers(List<ZkCallbackHandler> handlers, String path) {
     // System.out.println(handlers.size() + " handlers: ");
-    for (CallbackHandler handler : handlers) {
+    for (ZkCallbackHandler handler : handlers) {
       // System.out.println(handler.getPath());
       if (handler.getPath().equals(path)) {
         return true;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 98e4294..5f37845 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,14 +25,13 @@ import java.util.List;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.id.StateModelFactoryId;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.mock.participant.MockMSModelFactory;
-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.Message;
 import org.apache.helix.testutil.ZkTestBase;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
@@ -49,7 +48,7 @@ public class TestAddStateModelFactoryAfterConnect extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -60,15 +59,15 @@ public class TestAddStateModelFactoryAfterConnect extends ZkTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java
index 8ae722b..1f69a6e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoIsWithEmptyMap.java
@@ -28,8 +28,8 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -75,16 +75,16 @@ public class TestAutoIsWithEmptyMap extends ZkTestBase {
     _zkclient.writeData(idealPath, curIdealState);
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
index 9802e1c..c9c0821 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
@@ -30,8 +30,8 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
@@ -89,8 +89,8 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
       participant.syncStart();
       _participants[i] = participant;
 
@@ -98,7 +98,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
 
     // start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     boolean result =
@@ -163,8 +163,8 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBase {
       String storageNodeName = "localhost_" + (1000 + i);
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, CLUSTER_NAME, storageNodeName.replace(':', '_'));
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, CLUSTER_NAME, storageNodeName.replace(':', '_'));
       participant.syncStart();
     }
     Thread.sleep(5000);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
index eef3826..fdce7a4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
@@ -29,11 +29,9 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.stages.ClusterDataCache;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -77,7 +75,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBase {
 
     // start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     HelixManager manager = _controller; // _startCMResultMap.get(controllerName)._manager;
@@ -85,7 +83,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBase {
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
       _participants[i].syncStart();
       Thread.sleep(2000);
       boolean result =
@@ -147,8 +145,8 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBase {
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
       String newInstanceName = storageNodeName.replace(':', '_');
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, CLUSTER_NAME, newInstanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, CLUSTER_NAME, newInstanceName);
       participant.syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 54dd97c..8c62b3f 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
@@ -38,7 +38,7 @@ public class TestBasicSpectator extends ZkStandAloneCMTestBase implements
   Map<String, Integer> _externalViewChanges = new HashMap<String, Integer>();
 
   @Test
-  public void TestSpectator() throws Exception {
+  public void testSpectator() throws Exception {
     HelixManager relayHelixManager =
         HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, null, InstanceType.SPECTATOR, _zkaddr);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
index 099d47f..eca6bac 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
@@ -30,10 +30,10 @@ import org.apache.helix.HelixProperty.HelixPropertyAttribute;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
@@ -89,16 +89,16 @@ public class TestBatchMessage extends ZkTestBase {
     TestZkChildListener listener = new TestZkChildListener();
     _zkclient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -139,16 +139,16 @@ public class TestBatchMessage extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -178,7 +178,7 @@ public class TestBatchMessage extends ZkTestBase {
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -206,7 +206,7 @@ public class TestBatchMessage extends ZkTestBase {
     String clusterName = className + "_" + methodName;
 
     final int n = 5;
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -227,8 +227,8 @@ public class TestBatchMessage extends ZkTestBase {
     final String hostToFail = "localhost_12921";
     final String partitionToFail = "TestDB0_4";
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     for (int i = 0; i < n; i++) {
@@ -237,10 +237,10 @@ public class TestBatchMessage extends ZkTestBase {
       if (instanceName.equals(hostToFail)) {
         Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>();
         errPartitions.put("SLAVE-MASTER", TestHelper.setOf("TestDB0_4"));
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -297,8 +297,8 @@ public class TestBatchMessage extends ZkTestBase {
     TestZkChildListener listener = new TestZkChildListener();
     _zkclient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // pause controller
@@ -308,11 +308,11 @@ public class TestBatchMessage extends ZkTestBase {
     });
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java
index a1f2b4d..6a6837a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessageWrapper.java
@@ -24,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.messaging.handling.BatchMessageWrapper;
 import org.apache.helix.mock.participant.MockMSModelFactory;
@@ -88,18 +88,18 @@ public class TestBatchMessageWrapper extends ZkTestBase {
     idealState.setBatchMessageMode(true);
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     TestMockMSModelFactory[] ftys = new TestMockMSModelFactory[n];
 
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
       ftys[i] = new TestMockMSModelFactory();
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", ftys[i]);
       participants[i].syncStart();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 0c97b13..7234658 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
@@ -24,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.IdealState;
@@ -46,7 +46,7 @@ public class TestBucketizedResource extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
     // ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
@@ -67,15 +67,15 @@ public class TestBucketizedResource extends ZkTestBase {
     idealState.setBucketSize(1);
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 aa0bf33..1fc7d19 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
@@ -25,8 +25,8 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.testutil.TestUtil;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -42,7 +42,7 @@ public class TestCarryOverBadCurState extends ZkTestBase {
 
     String testName = TestUtil.getTestName();
     String clusterName = testName;
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -61,15 +61,15 @@ public class TestCarryOverBadCurState extends ZkTestBase {
     _zkclient.createPersistent(path, true);
     _zkclient.writeData(path, badCurState);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 521864b..8f3257b 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
@@ -24,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.ExternalView;
@@ -59,16 +59,16 @@ public class TestCleanupExternalView extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 afe35f5..d7a905b 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
@@ -26,7 +26,6 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyType;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.util.HelixUtil;
 import org.testng.Assert;
 import org.testng.AssertJUnit;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 dba7014..abb2a7b 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
@@ -29,7 +29,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.Message;
 import org.apache.helix.participant.statemachine.HelixStateModelFactory;
@@ -51,7 +51,7 @@ public class TestCorrectnessOnConnectivityLoss {
   private static final String ZK_ADDR = "localhost:2189";
   private ZkServer _zkServer;
   private String _clusterName;
-  private ClusterControllerManager _controller;
+  private MockController _controller;
 
   @BeforeMethod
   public void beforeMethod() throws Exception {
@@ -71,7 +71,7 @@ public class TestCorrectnessOnConnectivityLoss {
         RebalanceMode.FULL_AUTO, // automatic assignment
         true); // rebalance
 
-    _controller = new ClusterControllerManager(ZK_ADDR, _clusterName, "controller0");
+    _controller = new MockController(ZK_ADDR, _clusterName, "controller0");
     _controller.connect();
   }
 
@@ -124,7 +124,7 @@ public class TestCorrectnessOnConnectivityLoss {
         HelixManagerFactory.getZKHelixManager(_clusterName, "spectator", InstanceType.SPECTATOR,
             ZK_ADDR);
     spectator.connect();
-    spectator.addConfigChangeListener(routingTableProvider);
+    spectator.addInstanceConfigChangeListener(routingTableProvider);
     spectator.addExternalViewChangeListener(routingTableProvider);
     Thread.sleep(1000);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java
index 610d574..e1be10c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomIdealState.java
@@ -23,7 +23,6 @@ import java.util.Date;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.testutil.ZkTestBase;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.log4j.Logger;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
index b03a48b..316bcb6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
@@ -40,7 +40,6 @@ import org.apache.helix.controller.rebalancer.HelixRebalancer;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 ce88a34..4ab71cb 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,8 +27,8 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.IdealState;
@@ -53,7 +53,7 @@ public class TestDisable extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -73,15 +73,15 @@ public class TestDisable extends ZkTestBase {
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -141,7 +141,7 @@ public class TestDisable extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -153,15 +153,15 @@ public class TestDisable extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -220,7 +220,7 @@ public class TestDisable extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -240,15 +240,15 @@ public class TestDisable extends ZkTestBase {
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -312,7 +312,7 @@ public class TestDisable extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -324,15 +324,15 @@ public class TestDisable extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
index 08ffd54..ba642f4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
@@ -32,8 +32,8 @@ 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.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -95,13 +95,13 @@ public class TestDisableCustomCodeRunner extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
     Map<String, HelixCustomCodeRunner> customCodeRunners =
         new HashMap<String, HelixCustomCodeRunner>();
     Map<String, DummyCallback> callbacks = new HashMap<String, DummyCallback>();
@@ -109,7 +109,7 @@ public class TestDisableCustomCodeRunner extends ZkTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       participants
-          .put(instanceName, new MockParticipantManager(_zkaddr, clusterName, instanceName));
+          .put(instanceName, new MockParticipant(_zkaddr, clusterName, instanceName));
 
       customCodeRunners.put(instanceName, new HelixCustomCodeRunner(participants.get(instanceName),
           _zkaddr));
@@ -252,7 +252,7 @@ public class TestDisableCustomCodeRunner extends ZkTestBase {
 
     // Clean up
     controller.syncStop();
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 1419084..4da69d5 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
@@ -23,17 +23,14 @@ import java.util.Date;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
@@ -63,16 +60,16 @@ public class TestDisableResource extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager participants[] = new MockParticipantManager[N];
+    MockParticipant participants[] = new MockParticipant[N];
     for (int i = 0; i < N; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -118,16 +115,16 @@ public class TestDisableResource extends ZkTestBase {
         2, // replicas
         "MasterSlave", RebalanceMode.FULL_AUTO, true); // do rebalance
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager participants[] = new MockParticipantManager[N];
+    MockParticipant participants[] = new MockParticipant[N];
     for (int i = 0; i < N; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -182,16 +179,16 @@ public class TestDisableResource extends ZkTestBase {
     idealState.setPartitionState("TestDB0_0", "localhost_12919", "SLAVE");
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager participants[] = new MockParticipantManager[N];
+    MockParticipant participants[] = new MockParticipant[N];
     for (int i = 0; i < N; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 0d7d33c..cf32d73 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
@@ -23,12 +23,11 @@ import java.util.Date;
 
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.ClusterDistributedController;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockMultiClusterController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.ZkTestBase;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
@@ -75,10 +74,10 @@ public class TestDistributedCMMain extends ZkTestBase {
         "LeaderStandby", true); // do rebalance
 
     // start distributed cluster controllers
-    ClusterDistributedController[] controllers = new ClusterDistributedController[n + n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[n + n];
     for (int i = 0; i < n; i++) {
       controllers[i] =
-          new ClusterDistributedController(_zkaddr, controllerClusterName, "controller_" + i);
+          new MockMultiClusterController(_zkaddr, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 
@@ -89,11 +88,11 @@ public class TestDistributedCMMain extends ZkTestBase {
     Assert.assertTrue(result, "Controller cluster NOT in ideal state");
 
     // start first cluster
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     final String firstClusterName = clusterNamePrefix + "0_0";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost0_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, firstClusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, firstClusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -110,7 +109,7 @@ public class TestDistributedCMMain extends ZkTestBase {
     _setupTool.rebalanceStorageCluster(controllerClusterName, clusterNamePrefix + "0", 6);
     for (int i = n; i < 2 * n; i++) {
       controllers[i] =
-          new ClusterDistributedController(_zkaddr, controllerClusterName, "controller_" + i);
+          new MockMultiClusterController(_zkaddr, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 cf7884a..c2f9a5c 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
@@ -23,8 +23,8 @@ import java.util.Date;
 
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.ClusterDistributedController;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockMultiClusterController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.ZkTestBase;
@@ -74,10 +74,10 @@ public class TestDistributedClusterController extends ZkTestBase {
         "LeaderStandby", true); // do rebalance
 
     // start distributed cluster controllers
-    ClusterDistributedController[] controllers = new ClusterDistributedController[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[n];
     for (int i = 0; i < n; i++) {
       controllers[i] =
-          new ClusterDistributedController(_zkaddr, controllerClusterName, "controller_" + i);
+          new MockMultiClusterController(_zkaddr, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 
@@ -88,11 +88,11 @@ public class TestDistributedClusterController extends ZkTestBase {
     Assert.assertTrue(result, "Controller cluster NOT in ideal state");
 
     // start first cluster
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     final String firstClusterName = clusterNamePrefix + "0_0";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost0_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, firstClusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, firstClusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -110,11 +110,11 @@ public class TestDistributedClusterController extends ZkTestBase {
     controllers[j].syncStop();
 
     // setup the second cluster
-    MockParticipantManager[] participants2 = new MockParticipantManager[n];
+    MockParticipant[] participants2 = new MockParticipant[n];
     final String secondClusterName = clusterNamePrefix + "0_1";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost1_" + (12918 + i);
-      participants2[i] = new MockParticipantManager(_zkaddr, secondClusterName, instanceName);
+      participants2[i] = new MockParticipant(_zkaddr, secondClusterName, instanceName);
       participants2[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 9660f10..0e2a25b 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
@@ -31,8 +31,8 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.IdealState.IdealStateProperty;
@@ -187,8 +187,8 @@ public class TestDriver {
         LOG.warn("Dummy participant:" + instanceName + " has already started; skip starting it");
       } else {
         // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, clusterName, instanceName);
-        MockParticipantManager participant =
-            new MockParticipantManager(zkAddr, clusterName, instanceName);
+        MockParticipant participant =
+            new MockParticipant(zkAddr, clusterName, instanceName);
         participant.syncStart();
         testInfo._managers.put(instanceName, participant);
         // testInfo._instanceStarted.countDown();
@@ -216,8 +216,8 @@ public class TestDriver {
       if (testInfo._managers.containsKey(controllerName)) {
         LOG.warn("Controller:" + controllerName + " has already started; skip starting it");
       } else {
-        ClusterControllerManager controller =
-            new ClusterControllerManager(zkAddr, clusterName, controllerName);
+        MockController controller =
+            new MockController(zkAddr, clusterName, controllerName);
         controller.syncStart();
         testInfo._managers.put(controllerName, controller);
       }
@@ -252,8 +252,8 @@ public class TestDriver {
     // stop controller first
     for (String instanceName : testInfo._managers.keySet()) {
       if (instanceName.startsWith(CONTROLLER_PREFIX)) {
-        ClusterControllerManager controller =
-            (ClusterControllerManager) testInfo._managers.get(instanceName);
+        MockController controller =
+            (MockController) testInfo._managers.get(instanceName);
         controller.syncStop();
       }
     }
@@ -262,8 +262,8 @@ public class TestDriver {
 
     for (String instanceName : testInfo._managers.keySet()) {
       if (!instanceName.startsWith(CONTROLLER_PREFIX)) {
-        MockParticipantManager participant =
-            (MockParticipantManager) testInfo._managers.get(instanceName);
+        MockParticipant participant =
+            (MockParticipant) testInfo._managers.get(instanceName);
         participant.syncStop();
       }
     }
@@ -282,8 +282,8 @@ public class TestDriver {
     TestInfo testInfo = _testInfoMap.get(uniqClusterName);
 
     String failHost = PARTICIPANT_PREFIX + "_" + (START_PORT + instanceId);
-    MockParticipantManager participant =
-        (MockParticipantManager) testInfo._managers.remove(failHost);
+    MockParticipant participant =
+        (MockParticipant) testInfo._managers.remove(failHost);
 
     // TODO need sync
     if (participant == null) {