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:18 UTC

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

Repository: helix
Updated Branches:
  refs/heads/master 8a3705714 -> 593918880


http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java
index 54f81cd..c441381 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKLiveInstanceData.java
@@ -106,19 +106,6 @@ public class TestZKLiveInstanceData extends ZkTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    ZkClient zkClient = null;
-    try {
-      zkClient = new ZkClient(_zkaddr);
-      zkClient.setZkSerializer(new ZNRecordSerializer());
-      if (zkClient.exists("/" + clusterName)) {
-        zkClient.deleteRecursive("/" + clusterName);
-      }
-    } finally {
-      if (zkClient != null) {
-        zkClient.close();
-      }
-    }
-
     ClusterSetup.processCommandLineArgs(getArgs("-zkSvr", _zkaddr, "-addCluster", clusterName));
     ClusterSetup.processCommandLineArgs(getArgs("-zkSvr", _zkaddr, "-addNode", clusterName,
         "localhost:54321"));

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
index 0a770d0..0a325d6 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
@@ -27,12 +27,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.AccessOption;
-import org.apache.helix.integration.manager.MockParticipantManager;
-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.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
@@ -59,16 +56,9 @@ public class TestZkClusterManager extends ZkTestBase {
   public void testController() throws Exception {
     final String clusterName = TestUtil.getTestName();
 
-    System.out.println("START " + clusterName + ".testController() at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    // basic test
-    if (_zkclient.exists("/" + clusterName)) {
-      _zkclient.deleteRecursive("/" + clusterName);
-    }
-
-    ZKHelixManager controller =
-        new ZKHelixManager(clusterName, null, InstanceType.CONTROLLER, _zkaddr);
+    MockController controller = new MockController(_zkaddr, clusterName, "controller");
 
     try {
       controller.connect();
@@ -110,19 +100,17 @@ public class TestZkClusterManager extends ZkTestBase {
     controller.getMessagingService();
     controller.getClusterManagmentTool();
 
-    controller.handleNewSession();
+    controller.getConn().handleNewSession();
     controller.disconnect();
     AssertJUnit.assertFalse(controller.isConnected());
 
-    System.out.println("END " + clusterName + ".testController() at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test
   public void testLiveInstanceInfoProvider() throws Exception {
     final String clusterName = TestUtil.getTestName();
-    System.out.println("START " + clusterName + ".testLiveInstanceInfoProvider() at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
     class provider implements LiveInstanceInfoProvider {
       boolean _flag = false;
 
@@ -197,8 +185,7 @@ public class TestZkClusterManager extends ZkTestBase {
 
     // //////////////////////////////////
 
-    MockParticipantManager manager2 =
-        new MockParticipantManager(_zkaddr, clusterName, "localhost_3");
+    MockParticipant manager2 = new MockParticipant(_zkaddr, clusterName, "localhost_3");
 
     manager2.setLiveInstanceInfoProvider(new provider(true));
 
@@ -224,20 +211,13 @@ public class TestZkClusterManager extends ZkTestBase {
     Assert.assertFalse(liveInstance.getLiveInstance().equals("value"));
     Assert.assertFalse(sessionId.equals(liveInstance.getTypedSessionId().stringify()));
 
-    System.out.println("END " + clusterName + ".testLiveInstanceInfoProvider() at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test()
   public void testAdministrator() throws Exception {
     final String clusterName = TestUtil.getTestName();
-    System.out.println("START " + clusterName + ".testAdministrator() at "
-        + new Date(System.currentTimeMillis()));
-
-    // basic test
-    if (_zkclient.exists("/" + clusterName)) {
-      _zkclient.deleteRecursive("/" + clusterName);
-    }
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
     ZKHelixManager admin =
         new ZKHelixManager(clusterName, null, InstanceType.ADMINISTRATOR, _zkaddr);
@@ -265,8 +245,7 @@ public class TestZkClusterManager extends ZkTestBase {
     admin.disconnect();
     AssertJUnit.assertFalse(admin.isConnected());
 
-    System.out.println("END " + clusterName + ".testAdministrator() at "
-        + new Date(System.currentTimeMillis()));
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   private void setupInstances(String clusterName, int[] instances) {

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java
index 6a5f002..89eb323 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkFlapping.java
@@ -31,8 +31,6 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.Verifier;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
@@ -153,8 +151,8 @@ public class TestZkFlapping extends ZkTestBase {
         "MasterSlave", false);
 
     final String instanceName = "localhost_12918";
-    MockParticipantManager participant =
-        new MockParticipantManager(_zkaddr, clusterName, instanceName);
+    MockParticipant participant =
+        new MockParticipant(_zkaddr, clusterName, instanceName);
     participant.syncStart();
 
     final ZkClient client = participant.getZkClient();
@@ -232,8 +230,8 @@ public class TestZkFlapping extends ZkTestBase {
         1, // replicas
         "MasterSlave", false);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     final ZkClient client = controller.getZkClient();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java
index 96952d0..a34dc66 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAutoController.java
@@ -21,7 +21,7 @@ package org.apache.helix.manager.zk;
 
 import java.util.Date;
 
-import org.apache.helix.HelixAutoController;
+import org.apache.helix.HelixMultiClusterController;
 import org.apache.helix.HelixConnection;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
@@ -59,7 +59,7 @@ public class TestZkHelixAutoController extends ZkTestBase {
 
     // start auto-controller
     ClusterId clusterId = ClusterId.from(clusterName);
-    final HelixAutoController[] controllers = new HelixAutoController[n];
+    final HelixMultiClusterController[] controllers = new HelixMultiClusterController[n];
     for (int i = 0; i < n; i++) {
       int port = 12918 + i;
       ControllerId controllerId = ControllerId.from("localhost_" + port);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 86aa6e3..63bf1c5 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
@@ -21,8 +21,6 @@ package org.apache.helix.manager.zk;
 
 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.testutil.ZkTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -44,7 +42,7 @@ public class TestZkManagerFlappingDetection extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     String instanceName = "localhost_" + (12918 + 0);
-    MockParticipantManager manager = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+    MockParticipant manager = new MockParticipant(_zkaddr, clusterName, instanceName);
 
     manager.connect();
     ZkClient zkClient = manager.getZkClient();
@@ -139,7 +137,7 @@ public class TestZkManagerFlappingDetection extends ZkTestBase {
     // flapping time window to 5 sec
     System.setProperty("helixmanager.flappingTimeWindow", "5000");
     System.setProperty("helixmanager.maxDisconnectThreshold", "3");
-    ClusterControllerManager manager2 = new ClusterControllerManager(_zkaddr, clusterName, null);
+    MockController manager2 = new MockController(_zkaddr, clusterName, null);
     manager2.connect();
     Thread.sleep(100);
     ZkClient zkClient = manager2.getZkClient();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java
index a9c028c..c1972eb 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkStateChangeListener.java
@@ -20,7 +20,6 @@ package org.apache.helix.manager.zk;
  */
 
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.testng.Assert;
 
 public class TestZkStateChangeListener extends ZkStandAloneCMTestBase {
@@ -36,7 +35,7 @@ public class TestZkStateChangeListener extends ZkStandAloneCMTestBase {
     // 11 disconnects in 5 sec
     for (int i = 0; i < 11; i++) {
       Thread.sleep(200);
-      _controller.handleStateChanged(KeeperState.Disconnected);
+      // _controller.handleStateChanged(KeeperState.Disconnected);
       if (i < 10) {
         Assert.assertTrue(_controller.isConnected());
       } else {
@@ -51,7 +50,7 @@ public class TestZkStateChangeListener extends ZkStandAloneCMTestBase {
     // ZkStateChangeListener listener2 = new ZkStateChangeListener(_participants[0], 5000, 0);
     for (int i = 0; i < 2; i++) {
       Thread.sleep(200);
-      _participants[0].handleStateChanged(KeeperState.Disconnected);
+      // _participants[0].handleStateChanged(KeeperState.Disconnected);
       if (i < 1) {
         Assert.assertTrue(_participants[0].isConnected());
       } else {
@@ -66,22 +65,22 @@ public class TestZkStateChangeListener extends ZkStandAloneCMTestBase {
     // ZkStateChangeListener listener3 = new ZkStateChangeListener(_participants[1], 5000, 5);
     for (int i = 0; i < 3; i++) {
       Thread.sleep(200);
-      _participants[1].handleStateChanged(KeeperState.Disconnected);
+      // _participants[1].handleStateChanged(KeeperState.Disconnected);
       Assert.assertTrue(_participants[1].isConnected());
     }
     Thread.sleep(5000);
     // Old entries should be cleaned up
     for (int i = 0; i < 3; i++) {
       Thread.sleep(200);
-      _participants[1].handleStateChanged(KeeperState.Disconnected);
+      // _participants[1].handleStateChanged(KeeperState.Disconnected);
       Assert.assertTrue(_participants[1].isConnected());
     }
     for (int i = 0; i < 2; i++) {
       Thread.sleep(200);
-      _participants[1].handleStateChanged(KeeperState.Disconnected);
+      // _participants[1].handleStateChanged(KeeperState.Disconnected);
       Assert.assertTrue(_participants[1].isConnected());
     }
-    _participants[1].handleStateChanged(KeeperState.Disconnected);
+    // _participants[1].handleStateChanged(KeeperState.Disconnected);
     Assert.assertFalse(_participants[1].isConnected());
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/ZkConnTestHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/ZkConnTestHelper.java b/helix-core/src/test/java/org/apache/helix/manager/zk/ZkConnTestHelper.java
new file mode 100644
index 0000000..c2c4d42
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/ZkConnTestHelper.java
@@ -0,0 +1,34 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.
+ */
+
+/**
+ * Helper class for testing ZkHelixConnection
+ */
+public class ZkConnTestHelper extends ZkHelixConnection {
+
+  public ZkConnTestHelper(String zkAddr) {
+    super(zkAddr);
+  }
+
+  public ZkClient getZkClient() {
+    return _zkclient;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java
index 80a46fa..a1c55d5 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestConfigThreadpoolSize.java
@@ -27,9 +27,10 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
 import org.apache.helix.messaging.DefaultMessagingService;
-import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -80,12 +81,14 @@ public class TestConfigThreadpoolSize extends ZkStandAloneCMTestBase {
     HelixManager manager = _participants[0];
 
     ConfigAccessor accessor = manager.getConfigAccessor();
-    ConfigScope scope =
-        new ConfigScopeBuilder().forCluster(manager.getClusterName()).forParticipant(instanceName)
-            .build();
+    HelixConfigScope scope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT)
+            .forCluster(manager.getClusterName()).forParticipant(instanceName).build();
     accessor.set(scope, "TestMsg." + HelixTaskExecutor.MAX_THREADS, "" + 12);
 
-    scope = new ConfigScopeBuilder().forCluster(manager.getClusterName()).build();
+    scope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(
+            manager.getClusterName()).build();
     accessor.set(scope, "TestMsg." + HelixTaskExecutor.MAX_THREADS, "" + 8);
 
     for (int i = 0; i < NODE_NR; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 97a56be..694af9c 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
@@ -25,9 +25,10 @@ import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
 import org.apache.helix.messaging.DefaultMessagingService;
-import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -37,8 +38,9 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
   public void TestThreadPoolSizeConfig() {
     HelixManager manager = _participants[0];
     ConfigAccessor accessor = manager.getConfigAccessor();
-    ConfigScope scope =
-        new ConfigScopeBuilder().forCluster(manager.getClusterName()).forResource("NextDB").build();
+    HelixConfigScope scope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE)
+            .forCluster(manager.getClusterName()).forResource("NextDB").build();
     accessor.set(scope, HelixTaskExecutor.MAX_THREADS, "" + 12);
 
     _setupTool.addResourceToCluster(CLUSTER_NAME, "NextDB", 64, STATE_MODEL);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
deleted file mode 100644
index 4ddaac4..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
+++ /dev/null
@@ -1,140 +0,0 @@
-package org.apache.helix.mock.controller;
-
-/*
- * 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.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.TreeMap;
-
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.api.State;
-import org.apache.helix.api.id.MessageId;
-import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.api.id.SessionId;
-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.ExternalView;
-import org.apache.helix.model.IdealState.IdealStateProperty;
-import org.apache.helix.model.LiveInstance.LiveInstanceProperty;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageState;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.util.HelixUtil;
-import org.codehaus.jackson.JsonGenerationException;
-import org.codehaus.jackson.map.JsonMappingException;
-import org.codehaus.jackson.map.ObjectMapper;
-
-public class MockController {
-  private final ZkClient client;
-  private final String srcName;
-  private final String clusterName;
-
-  public MockController(String src, String zkServer, String cluster) {
-    srcName = src;
-    clusterName = cluster;
-    client = new ZkClient(zkServer);
-    client.setZkSerializer(new ZNRecordSerializer());
-  }
-
-  void sendMessage(MessageId msgId, String instanceName, String fromState, String toState,
-      String partitionKey, int partitionId) throws InterruptedException, JsonGenerationException,
-      JsonMappingException, IOException {
-    Message message = new Message(MessageType.STATE_TRANSITION, msgId);
-    message.setMessageId(msgId);
-    message.setSrcName(srcName);
-    message.setTgtName(instanceName);
-    message.setMsgState(MessageState.NEW);
-    message.setFromState(State.from(fromState));
-    message.setToState(State.from(toState));
-    // message.setPartitionId(partitionId);
-    message.setPartitionId(PartitionId.from(partitionKey));
-
-    String path = HelixUtil.getMessagePath(clusterName, instanceName) + "/" + message.getId();
-    ObjectMapper mapper = new ObjectMapper();
-    StringWriter sw = new StringWriter();
-    mapper.writeValueUsingView(sw, message, Message.class);
-    System.out.println(sw.toString());
-    client.delete(path);
-
-    Thread.sleep(10000);
-    ZNRecord record = client.readData(HelixUtil.getLiveInstancePath(clusterName, instanceName));
-    message.setTgtSessionId(SessionId.from(record.getSimpleField(
-        LiveInstanceProperty.SESSION_ID.toString()).toString()));
-    client.createPersistent(path, message);
-  }
-
-  public void createExternalView(List<String> instanceNames, int partitions, int replicas,
-      String dbName, long randomSeed) {
-    ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(client));
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ExternalView externalView =
-        new ExternalView(computeRoutingTable(instanceNames, partitions, replicas, dbName,
-            randomSeed));
-
-    accessor.setProperty(keyBuilder.externalView(dbName), externalView);
-  }
-
-  public ZNRecord computeRoutingTable(List<String> instanceNames, int partitions, int replicas,
-      String dbName, long randomSeed) {
-    assert (instanceNames.size() > replicas);
-    Collections.sort(instanceNames);
-
-    ZNRecord result = new ZNRecord(dbName);
-
-    Map<String, Object> externalView = new TreeMap<String, Object>();
-
-    List<Integer> partitionList = new ArrayList<Integer>(partitions);
-    for (int i = 0; i < partitions; i++) {
-      partitionList.add(new Integer(i));
-    }
-    Random rand = new Random(randomSeed);
-    // Shuffle the partition list
-    Collections.shuffle(partitionList, rand);
-
-    for (int i = 0; i < partitionList.size(); i++) {
-      int partitionId = partitionList.get(i);
-      Map<String, String> partitionAssignment = new TreeMap<String, String>();
-      int masterNode = i % instanceNames.size();
-      // the first in the list is the node that contains the master
-      partitionAssignment.put(instanceNames.get(masterNode), "MASTER");
-
-      // for the jth replica, we put it on (masterNode + j) % nodes-th
-      // node
-      for (int j = 1; j <= replicas; j++) {
-        partitionAssignment
-            .put(instanceNames.get((masterNode + j) % instanceNames.size()), "SLAVE");
-      }
-      String partitionName = dbName + ".partition-" + partitionId;
-      result.setMapField(partitionName, partitionAssignment);
-    }
-    result.setSimpleField(IdealStateProperty.NUM_PARTITIONS.toString(), "" + partitions);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
deleted file mode 100644
index 193abd3..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
+++ /dev/null
@@ -1,76 +0,0 @@
-package org.apache.helix.mock.controller;
-
-/*
- * 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.util.ArrayList;
-
-import org.apache.helix.api.id.MessageId;
-import org.codehaus.jackson.JsonGenerationException;
-import org.codehaus.jackson.map.JsonMappingException;
-
-public class MockControllerProcess {
-
-  /**
-   * @param args
-   * @throws IOException
-   * @throws JsonMappingException
-   * @throws JsonGenerationException
-   * @throws InterruptedException
-   */
-  public static void main(String[] args) throws JsonGenerationException, JsonMappingException,
-      InterruptedException, IOException {
-
-    MockController storageController =
-        new MockController("cm-instance-0", "localhost:2181", "storage-cluster");
-    MockController relayController =
-        new MockController("cm-instance-0", "localhost:2181", "relay-cluster");
-
-    ArrayList<String> instanceNames = new ArrayList<String>();
-    instanceNames.add("relay0");
-    instanceNames.add("relay1");
-    instanceNames.add("relay2");
-    instanceNames.add("relay3");
-    instanceNames.add("relay4");
-
-    relayController.createExternalView(instanceNames, 10, 2, "EspressoDB", 0);
-
-    // Messages to initiate offline->slave->master->slave transitions
-
-    storageController.sendMessage(MessageId.from("TestMessageId1"), "localhost_8900", "Offline",
-        "Slave", "EspressoDB.partition-0", 0);
-    Thread.sleep(10000);
-    storageController.sendMessage(MessageId.from("TestMessageId2"), "localhost_8900", "Slave",
-        "Master", "EspressoDB.partition-0", 0);
-    Thread.sleep(10000);
-    storageController.sendMessage(MessageId.from("TestMessageId3"), "localhost_8900", "Master",
-        "Slave", "EspressoDB.partition-0", 0);
-    Thread.sleep(10000);
-
-    // Change the external view to trigger the consumer to listen from
-    // another relay
-    relayController.createExternalView(instanceNames, 10, 2, "EspressoDB", 10);
-
-    storageController.sendMessage(MessageId.from("TestMessageId4"), "localhost_8900", "Slave",
-        "Offline", "EspressoDB.partition-0", 0);
-    Thread.sleep(10000);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
index 4d5dd95..6f78427 100644
--- a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
@@ -28,13 +28,11 @@ import java.util.TreeMap;
 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.api.id.MessageId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
-import org.apache.helix.model.Message.Attributes;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.testutil.HelixTestUtil;
 import org.apache.helix.testutil.TestUtil;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 c31b641..2b8d5e5 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
@@ -29,12 +29,11 @@ import javax.management.MalformedObjectNameException;
 
 import org.apache.helix.HelixDataAccessor;
 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.model.IdealState;
 import org.apache.helix.monitoring.mbeans.ClusterMBeanObserver;
 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.apache.log4j.Logger;
@@ -46,8 +45,8 @@ import org.testng.annotations.Test;
 public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
   private static final Logger LOG = Logger.getLogger(TestClusterStatusMonitorLifecycle.class);
 
-  MockParticipantManager[] _participants;
-  ClusterDistributedController[] _controllers;
+  MockParticipant[] _participants;
+  MockMultiClusterController[] _controllers;
   String _controllerClusterName;
   String _clusterNamePrefix;
   String _firstClusterName;
@@ -92,10 +91,10 @@ public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
         "LeaderStandby", true); // do rebalance
 
     // start distributed cluster controllers
-    _controllers = new ClusterDistributedController[n + n];
+    _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();
     }
 
@@ -106,11 +105,11 @@ public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
     Assert.assertTrue(result, "Controller cluster NOT in ideal state");
 
     // start first cluster
-    _participants = new MockParticipantManager[n];
+    _participants = new MockParticipant[n];
     _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();
     }
 
@@ -127,7 +126,7 @@ public class TestClusterStatusMonitorLifecycle 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();
     }
 
@@ -206,8 +205,8 @@ public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
     String firstControllerName =
         accessor.getProperty(accessor.keyBuilder().controllerLeader()).getId();
 
-    ClusterDistributedController firstController = null;
-    for (ClusterDistributedController controller : _controllers) {
+    MockMultiClusterController firstController = null;
+    for (MockMultiClusterController controller : _controllers) {
       if (controller.getInstanceName().equals(firstControllerName)) {
         firstController = controller;
       }
@@ -222,7 +221,7 @@ public class TestClusterStatusMonitorLifecycle extends ZkTestBase {
     Assert.assertEquals(nMbeansRegistered, listener._nMbeansRegistered - 12);
 
     String instanceName = "localhost0_" + (12918 + 0);
-    _participants[0] = new MockParticipantManager(_zkaddr, _firstClusterName, instanceName);
+    _participants[0] = new MockParticipant(_zkaddr, _firstClusterName, instanceName);
     _participants[0].syncStart();
 
     // 1 participant comes back

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 17e1837..b8a97cc 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
@@ -31,8 +31,8 @@ import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
 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.model.IdealState.RebalanceMode;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterSetup;
@@ -71,14 +71,14 @@ public class TestDropResourceMetricsReset extends ZkTestBase {
 
     // Start participants and controller
     ClusterSetup setupTool = new ClusterSetup(_zkclient);
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       participants[i] =
-          new MockParticipantManager(_zkaddr, clusterName, "localhost_" + (12918 + i));
+          new MockParticipant(_zkaddr, clusterName, "localhost_" + (12918 + i));
       participants[i].syncStart();
     }
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // Verify that the bean was created
@@ -95,7 +95,7 @@ public class TestDropResourceMetricsReset extends ZkTestBase {
     // Clean up
     listener.disconnect();
     controller.syncStop();
-    for (MockParticipantManager participant : participants) {
+    for (MockParticipant participant : participants) {
       participant.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/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 5497138..c382f55 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
@@ -27,8 +27,8 @@ import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 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.model.IdealState.RebalanceMode;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -50,13 +50,13 @@ public class TestResetClusterMetrics extends ZkTestBase {
         "OnlineOffline", RebalanceMode.FULL_AUTO, true);
 
     // Add a participant
-    MockParticipantManager participant =
-        new MockParticipantManager(_zkaddr, clusterName, "localhost_12918");
+    MockParticipant participant =
+        new MockParticipant(_zkaddr, clusterName, "localhost_12918");
     participant.syncStart();
 
     // Add a controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // Make sure everything gets assigned

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
index 11cdd34..efa30da 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
@@ -23,7 +23,6 @@ import java.util.UUID;
 
 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;
@@ -93,12 +92,6 @@ public class MockZKHelixManager 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/store/zk/TestZkManagerWithAutoFallbackStore.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
index 3e5e068..69f676e 100644
--- a/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
+++ b/helix-core/src/test/java/org/apache/helix/store/zk/TestZkManagerWithAutoFallbackStore.java
@@ -26,7 +26,7 @@ import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.helix.testutil.ZkTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -52,11 +52,11 @@ public class TestZkManagerWithAutoFallbackStore extends ZkTestBase {
         "MasterSlave", false); // do rebalance
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < 1; 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/testutil/TestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/testutil/TestUtil.java b/helix-core/src/test/java/org/apache/helix/testutil/TestUtil.java
index bc62fca..6ff2b90 100644
--- a/helix-core/src/test/java/org/apache/helix/testutil/TestUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/testutil/TestUtil.java
@@ -27,7 +27,7 @@ import org.apache.log4j.Logger;
 public class TestUtil {
   static Logger logger = Logger.getLogger(TestUtil.class);
 
-  public static boolean isTestNGAnnotated(Class clazz, String methodName) {
+  public static boolean isTestNGAnnotated(Class<? extends Object> clazz, String methodName) {
     final String annotationsPkgName = "org.testng.annotations";
 
     // Check if the class itself is annotated.
@@ -61,7 +61,7 @@ public class TestUtil {
 
       // The first 2 elements in the stack are getStackTrace and this method itself, so ignore them.
       for (int i = 2; i < stackTrace.length; i++) {
-        Class clazz = Class.forName(stackTrace[i].getClassName());
+        Class<? extends Object> clazz = Class.forName(stackTrace[i].getClassName());
         if (isTestNGAnnotated(clazz, stackTrace[i].getMethodName())) {
           String testName = String.format("%s_%s", clazz.getSimpleName(), stackTrace[i].getMethodName());
           logger.debug("Detected " + testName + " as the test name");

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/tools/TestClusterStateVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestClusterStateVerifier.java b/helix-core/src/test/java/org/apache/helix/tools/TestClusterStateVerifier.java
index ec43664..e648a14 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestClusterStateVerifier.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestClusterStateVerifier.java
@@ -23,8 +23,8 @@ import java.util.Arrays;
 
 import org.apache.helix.HelixAdmin;
 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.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.testutil.ZkTestBase;
@@ -41,8 +41,8 @@ public class TestClusterStateVerifier extends ZkTestBase {
       "resource0", "resource1"
   };
   private HelixAdmin _admin;
-  private MockParticipantManager[] _participants;
-  private ClusterControllerManager _controller;
+  private MockParticipant[] _participants;
+  private MockController _controller;
   private String _clusterName;
 
   @BeforeMethod
@@ -62,13 +62,13 @@ public class TestClusterStateVerifier extends ZkTestBase {
         RebalanceMode.SEMI_AUTO.toString());
 
     // Configure and start the participants
-    _participants = new MockParticipantManager[RESOURCES.length];
+    _participants = new MockParticipant[RESOURCES.length];
     for (int i = 0; i < _participants.length; i++) {
       String host = "localhost";
       int port = 12918 + i;
       String id = host + '_' + port;
       _setupTool.addInstanceToCluster(_clusterName, id);
-      _participants[i] = new MockParticipantManager(_zkaddr, _clusterName, id);
+      _participants[i] = new MockParticipant(_zkaddr, _clusterName, id);
       _participants[i].syncStart();
     }
 
@@ -82,7 +82,7 @@ public class TestClusterStateVerifier extends ZkTestBase {
     }
 
     // Start the controller
-    _controller = new ClusterControllerManager(_zkaddr, _clusterName, "controller_0");
+    _controller = new MockController(_zkaddr, _clusterName, "controller_0");
     _controller.syncStart();
     Thread.sleep(1000);
   }
@@ -91,7 +91,7 @@ public class TestClusterStateVerifier extends ZkTestBase {
   public void afterMethod() {
     // Cleanup
     _controller.syncStop();
-    for (MockParticipantManager participant : _participants) {
+    for (MockParticipant participant : _participants) {
       participant.syncStop();
     }
     _admin.dropCluster(_clusterName);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 6d27dcb..9e95b51 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
@@ -31,8 +31,8 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
-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.manager.zk.ZKUtil;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -239,8 +239,8 @@ public class TestHelixAdminCli extends ZkTestBase {
     final int n = 6;
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
+    MockParticipant[] participants = new MockParticipant[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
 
     // activate clusters
@@ -310,7 +310,7 @@ public class TestHelixAdminCli extends ZkTestBase {
     Assert.assertTrue(verifyResult);
 
     // clean up
-    for (ClusterDistributedController controller : controllers) {
+    for (MockMultiClusterController controller : controllers) {
       controller.syncStop();
     }
     for (int i = 0; i < participants.length; i++) {
@@ -330,8 +330,8 @@ public class TestHelixAdminCli extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
+    MockParticipant[] participants = new MockParticipant[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
         "-zkSvr " + _zkaddr + " -activateCluster " + clusterName + " " + grandClusterName + " true";
@@ -387,7 +387,7 @@ public class TestHelixAdminCli extends ZkTestBase {
   }
 
   private void setupCluster(String clusterName, String grandClusterName, final int n,
-      MockParticipantManager[] participants, ClusterDistributedController[] controllers)
+      MockParticipant[] participants, MockMultiClusterController[] controllers)
       throws Exception, InterruptedException {
     // add cluster
     String command = "-zkSvr " + _zkaddr + " -addCluster " + clusterName;
@@ -419,14 +419,14 @@ public class TestHelixAdminCli extends ZkTestBase {
     // start mock nodes
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_123" + i;
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
     // start controller nodes
     for (int i = 0; i < 2; i++) {
       controllers[i] =
-          new ClusterDistributedController(_zkaddr, grandClusterName, "controller_900" + i);
+          new MockMultiClusterController(_zkaddr, grandClusterName, "controller_900" + i);
       controllers[i].syncStart();
     }
 
@@ -443,8 +443,8 @@ public class TestHelixAdminCli extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
+    MockParticipant[] participants = new MockParticipant[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
         "-zkSvr " + _zkaddr + " -activateCluster " + clusterName + " " + grandClusterName + " true";
@@ -522,8 +522,8 @@ public class TestHelixAdminCli extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
+    MockParticipant[] participants = new MockParticipant[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
         "-zkSvr " + _zkaddr + " -activateCluster " + clusterName + " " + grandClusterName + " true";
@@ -538,10 +538,10 @@ public class TestHelixAdminCli extends ZkTestBase {
     command = "-zkSvr localhost:2183 -expandCluster " + clusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
-    MockParticipantManager[] newParticipants = new MockParticipantManager[4];
+    MockParticipant[] newParticipants = new MockParticipant[4];
     for (int i = 3; i <= 6; i++) {
       String instanceName = "localhost_123" + i + "1";
-      newParticipants[i - 3] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      newParticipants[i - 3] = new MockParticipant(_zkaddr, clusterName, instanceName);
       newParticipants[i - 3].syncStart();
     }
 
@@ -579,8 +579,8 @@ public class TestHelixAdminCli extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
-    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
+    MockParticipant[] participants = new MockParticipant[n];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
         "-zkSvr " + _zkaddr + " -activateCluster " + clusterName + " " + grandClusterName + " true";

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
----------------------------------------------------------------------
diff --git a/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java b/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
index 4064e10..9d56cdf 100644
--- a/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
@@ -49,7 +49,7 @@ import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.rebalancer.config.FullAutoRebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.PartitionedRebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
-import org.apache.helix.manager.zk.HelixConnectionAdaptor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkHelixConnection;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.provisioning.ApplicationSpec;
@@ -190,7 +190,7 @@ public class AppMasterLauncher {
               readFromHDFS(fs, taskConfig.name, yamlUri, applicationSpec,
                   appAttemptID.getApplicationId());
           Workflow workflow = Workflow.parse(is);
-          TaskDriver taskDriver = new TaskDriver(new HelixConnectionAdaptor(controller));
+          TaskDriver taskDriver = new TaskDriver(new ZKHelixManager(controller));
           taskDriver.start(workflow);
         }
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/JobRunnerMain.java
----------------------------------------------------------------------
diff --git a/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/JobRunnerMain.java b/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/JobRunnerMain.java
index e588ea8..59a2a92 100644
--- a/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/JobRunnerMain.java
+++ b/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/JobRunnerMain.java
@@ -29,6 +29,7 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.HelixConnection;
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRole;
 import org.apache.helix.InstanceType;
@@ -38,7 +39,7 @@ import org.apache.helix.api.accessor.ClusterAccessor;
 import org.apache.helix.api.config.ContainerConfig;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.Id;
-import org.apache.helix.manager.zk.HelixConnectionAdaptor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.provisioning.ApplicationSpec;
 import org.apache.helix.provisioning.ApplicationSpecFactory;
@@ -82,7 +83,7 @@ public class JobRunnerMain {
     final HelixConnection connection = launcher.pollForConnection();
     final ClusterId clusterId = ClusterId.from(appSpec.getAppName());
     // TODO: this is a hack -- TaskDriver should accept a connection instead of a manager
-    HelixManager manager = new HelixConnectionAdaptor(new HelixRole() {
+    HelixManager manager = new ZKHelixManager(new HelixRole() {
       @Override
       public HelixConnection getConnection() {
         return connection;
@@ -107,6 +108,11 @@ public class JobRunnerMain {
       public ClusterMessagingService getMessagingService() {
         return null;
       }
+
+      @Override
+      public HelixDataAccessor getAccessor() {
+        return null;
+      }
     });
 
     // Get all submitted jobs

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/MyTaskService.java
----------------------------------------------------------------------
diff --git a/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/MyTaskService.java b/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/MyTaskService.java
index 7c50e53..d77a71e 100644
--- a/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/MyTaskService.java
+++ b/recipes/jobrunner-yarn/src/main/java/org/apache/helix/provisioning/yarn/example/MyTaskService.java
@@ -27,7 +27,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.manager.zk.HelixConnectionAdaptor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.participant.AbstractParticipantService;
 import org.apache.helix.provisioning.ServiceConfig;
 import org.apache.helix.provisioning.participant.StatelessParticipantService;
@@ -56,7 +56,7 @@ public class MyTaskService extends StatelessParticipantService {
     LOG.info("Initialized service with config " + serviceConfig);
 
     // Register for callbacks for tasks
-    HelixManager manager = new HelixConnectionAdaptor(getParticipant());
+    HelixManager manager = new ZKHelixManager(getParticipant());
     Map<String, TaskFactory> taskFactoryReg = new HashMap<String, TaskFactory>();
     taskFactoryReg.put("RunTask", new TaskFactory() {
       @Override


[7/7] git commit: merge master

Posted by zz...@apache.org.
merge master


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

Branch: refs/heads/master
Commit: 593918880be3b3ab139d7043105ec78fb759ddb2
Parents: 02165c5 8a37057
Author: zzhang <zz...@apache.org>
Authored: Mon Aug 4 13:18:48 2014 -0700
Committer: zzhang <zz...@apache.org>
Committed: Mon Aug 4 13:18:48 2014 -0700

----------------------------------------------------------------------
 .../helix/api/accessor/ClusterAccessor.java     |  78 +--
 .../config/BasicRebalancerConfig.java           | 256 ---------
 .../config/CustomRebalancerConfig.java          | 169 ------
 .../config/FullAutoRebalancerConfig.java        |  69 ---
 .../config/PartitionedRebalancerConfig.java     | 523 -------------------
 .../rebalancer/config/RebalancerConfig.java     |  57 +-
 .../config/RebalancerConfigHolder.java          |  15 -
 .../config/ReplicatedRebalancerConfig.java      |  40 --
 .../config/SemiAutoRebalancerConfig.java        | 183 -------
 .../stages/BestPossibleStateCalcStage.java      |  70 +--
 .../stages/ContainerProvisioningStage.java      |  88 ++--
 .../stages/ExternalViewComputeStage.java        |   8 +-
 .../stages/MessageGenerationStage.java          |  17 +-
 .../stages/MessageSelectionStage.java           |  21 +-
 .../stages/ResourceComputationStage.java        |  51 +-
 .../java/org/apache/helix/model/IdealState.java |  83 +--
 .../helix/model/builder/IdealStateBuilder.java  |  10 +-
 .../monitoring/mbeans/ClusterStatusMonitor.java |   4 +-
 .../java/org/apache/helix/task/JobContext.java  |   6 +-
 .../java/org/apache/helix/task/TaskDriver.java  |  61 +--
 .../org/apache/helix/task/TaskRebalancer.java   |  21 +-
 .../java/org/apache/helix/task/TaskState.java   |   5 +
 .../org/apache/helix/api/TestNewStages.java     | 253 ---------
 .../context/TestSerializeRebalancerContext.java | 105 ----
 .../helix/controller/stages/BaseStageTest.java  |   5 +-
 .../stages/TestResourceComputationStage.java    |  14 +-
 .../stages/TestStagesWithLogicalAccessors.java  | 255 +++++++++
 .../TestCustomizedIdealStateRebalancer.java     |   6 +-
 .../helix/integration/TestHelixConnection.java  |  20 +-
 .../integration/TestLocalContainerProvider.java |  20 +-
 .../TestUserDefRebalancerCompatibility.java     |   3 +-
 .../mbeans/TestClusterStatusMonitor.java        |  20 +-
 .../helix/examples/LogicalModelExample.java     |  66 +--
 .../tools/UpdateProvisionerConfig.java          |   1 +
 .../provisioning/yarn/AppMasterLauncher.java    |  22 +-
 .../provisioning/yarn/YarnProvisioner.java      |   6 +-
 36 files changed, 522 insertions(+), 2109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/59391888/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/helix/blob/59391888/helix-core/src/test/java/org/apache/helix/controller/stages/TestStagesWithLogicalAccessors.java
----------------------------------------------------------------------
diff --cc helix-core/src/test/java/org/apache/helix/controller/stages/TestStagesWithLogicalAccessors.java
index 0000000,ccd8c67..ad89985
mode 000000,100644..100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestStagesWithLogicalAccessors.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestStagesWithLogicalAccessors.java
@@@ -1,0 -1,255 +1,255 @@@
+ package org.apache.helix.controller.stages;
+ 
+ /*
+  * 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.util.Date;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.helix.HelixDataAccessor;
+ import org.apache.helix.TestHelper;
+ import org.apache.helix.api.Cluster;
+ import org.apache.helix.api.Participant;
+ import org.apache.helix.api.Resource;
+ import org.apache.helix.api.State;
+ import org.apache.helix.api.accessor.ClusterAccessor;
+ import org.apache.helix.api.config.ResourceConfig;
+ import org.apache.helix.api.id.ClusterId;
+ import org.apache.helix.api.id.ParticipantId;
+ import org.apache.helix.api.id.PartitionId;
+ import org.apache.helix.api.id.ResourceId;
+ import org.apache.helix.controller.stages.AttributeName;
+ 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.IdealState;
+ import org.apache.helix.model.IdealState.RebalanceMode;
+ import org.apache.helix.model.ResourceAssignment;
+ import org.apache.helix.testutil.ZkTestBase;
+ 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.BeforeClass;
+ import org.testng.annotations.Test;
+ 
+ import com.google.common.base.Function;
+ import com.google.common.collect.Maps;
+ 
+ public class TestStagesWithLogicalAccessors 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;
+ 
+   @Test
+   public void testReadClusterDataStage() {
+     String className = TestHelper.getTestClassName();
+     String methodName = TestHelper.getTestMethodName();
+     String testName = className + "_" + methodName;
+ 
+     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
+ 
+     ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor);
+     Cluster cluster = clusterAccessor.readCluster();
+ 
+     ClusterId id = cluster.getId();
+     Assert.assertEquals(id, _clusterId);
+     Map<ParticipantId, Participant> liveParticipantMap = cluster.getLiveParticipantMap();
+     Assert.assertEquals(liveParticipantMap.size(), n);
+ 
+     for (ParticipantId participantId : liveParticipantMap.keySet()) {
+       Participant participant = liveParticipantMap.get(participantId);
+       Map<ResourceId, CurrentState> curStateMap = participant.getCurrentStateMap();
+       Assert.assertEquals(curStateMap.size(), 1);
+ 
+       ResourceId resourceId = ResourceId.from("TestDB0");
+       Assert.assertTrue(curStateMap.containsKey(resourceId));
+       CurrentState curState = curStateMap.get(resourceId);
+       Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
+       Assert.assertEquals(partitionStateMap.size(), p);
+     }
+ 
+     Map<ResourceId, Resource> resourceMap = cluster.getResourceMap();
+     Assert.assertEquals(resourceMap.size(), 1);
+ 
+     ResourceId resourceId = ResourceId.from("TestDB0");
+     Assert.assertTrue(resourceMap.containsKey(resourceId));
+     Resource resource = resourceMap.get(resourceId);
+     Assert.assertNotNull(resource.getIdealState());
+     Assert.assertEquals(resource.getIdealState().getRebalanceMode(), RebalanceMode.SEMI_AUTO);
+ 
+     System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis()));
+   }
+ 
+   @Test
+   public void testBasicBestPossibleStateCalcStage() {
+     String className = TestHelper.getTestClassName();
+     String methodName = TestHelper.getTestMethodName();
+     String testName = className + "_" + methodName;
+ 
+     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
+ 
+     // Set up the event
+     ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor);
+     Cluster cluster = clusterAccessor.readCluster();
+     ClusterEvent event = new ClusterEvent(testName);
+     event.addAttribute(AttributeName.CURRENT_STATE.toString(), new ResourceCurrentState());
+     Map<ResourceId, ResourceConfig> resourceConfigMap =
+         Maps.transformValues(cluster.getResourceMap(), new Function<Resource, ResourceConfig>() {
+           @Override
+           public ResourceConfig apply(Resource resource) {
+             return resource.getConfig();
+           }
+         });
+     event.addAttribute(AttributeName.RESOURCES.toString(), resourceConfigMap);
+     event.addAttribute("Cluster", cluster);
+ 
+     // Run the stage
+     try {
+       new BestPossibleStateCalcStage().process(event);
+     } catch (Exception e) {
+       Assert.fail(e.toString());
+     }
+ 
+     // Verify the result
+     BestPossibleStateOutput bestPossibleStateOutput =
+         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
+     Assert.assertNotNull(bestPossibleStateOutput);
+     ResourceId resourceId = ResourceId.from("TestDB0");
+     ResourceAssignment assignment = bestPossibleStateOutput.getResourceAssignment(resourceId);
+     Assert.assertNotNull(assignment);
+     Resource resource = cluster.getResource(resourceId);
+     verifySemiAutoRebalance(resource, assignment);
+ 
+     System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis()));
+   }
+ 
+   @Test
+   public void testClusterRebalancers() {
+     String className = TestHelper.getTestClassName();
+     String methodName = TestHelper.getTestMethodName();
+     String testName = className + "_" + methodName;
+ 
+     System.out.println("START " + testName + " at " + new Date(System.currentTimeMillis()));
+ 
+     ClusterAccessor clusterAccessor = new ClusterAccessor(_clusterId, _dataAccessor);
+     Cluster cluster = clusterAccessor.readCluster();
+ 
+     ResourceId resourceId = ResourceId.from("TestDB0");
+     Resource resource = cluster.getResource(resourceId);
+     ResourceCurrentState currentStateOutput = new ResourceCurrentState();
+     ResourceAssignment semiAutoResult =
+         resource
+             .getIdealState()
+             .getRebalancerRef()
+             .getRebalancer()
+             .computeResourceMapping(resource.getIdealState(), resource.getRebalancerConfig(), null,
+                 cluster, currentStateOutput);
+     verifySemiAutoRebalance(resource, semiAutoResult);
+ 
+     System.out.println("END " + testName + " at " + new Date(System.currentTimeMillis()));
+   }
+ 
+   /**
+    * Check that a semi auto rebalance is run, and all partitions are mapped by preference list
+    * @param resource the resource to verify
+    * @param assignment the assignment to verify
+    */
+   private void verifySemiAutoRebalance(Resource resource, ResourceAssignment assignment) {
+     Assert.assertEquals(assignment.getMappedPartitionIds().size(), resource.getSubUnitSet().size());
+     IdealState idealState = resource.getIdealState();
+     for (PartitionId partitionId : assignment.getMappedPartitionIds()) {
+       List<ParticipantId> preferenceList = idealState.getPreferenceList(partitionId);
+       Map<ParticipantId, State> replicaMap = assignment.getReplicaMap(partitionId);
+       Assert.assertEquals(replicaMap.size(), preferenceList.size());
+       Assert.assertEquals(replicaMap.size(), r);
+       boolean hasMaster = false;
+       for (ParticipantId participant : preferenceList) {
+         Assert.assertTrue(replicaMap.containsKey(participant));
+         State state = replicaMap.get(participant);
+         if (state.equals(State.from("MASTER"))) {
+           Assert.assertFalse(hasMaster);
+           hasMaster = true;
+         }
+       }
+       Assert.assertEquals(replicaMap.get(preferenceList.get(0)), State.from("MASTER"));
+     }
+   }
+ 
+   @BeforeClass
+   public void beforeClass() throws Exception {
+     // set up a running class
+     // Logger.getRootLogger().setLevel(Level.INFO);
+     String className = TestHelper.getTestClassName();
+     String methodName = TestHelper.getTestMethodName();
+     String clusterName = className + "_" + methodName;
+     _clusterId = ClusterId.from(clusterName);
+ 
+     System.out.println("START " + _clusterId + " at " + new Date(System.currentTimeMillis()));
+ 
+     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
+         "localhost", // participant name prefix
+         "TestDB", // resource name prefix
+         1, // resources
+         p, // partitions per resource
+         n, // number of nodes
+         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();
+     }
+ 
+     boolean result =
+         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(_zkaddr,
+             clusterName));
+     Assert.assertTrue(result);
+ 
+     _dataAccessor = new ZKHelixDataAccessor(clusterName, _baseAccessor);
+   }
+ 
+   @AfterClass
+   public void afterClass() {
+     // tear down the cluster
+     _controller.syncStop();
+     for (int i = 0; i < n; i++) {
+       _participants[i].syncStop();
+     }
+ 
+     System.out.println("END " + _clusterId + " at " + new Date(System.currentTimeMillis()));
+   }
+ }

http://git-wip-us.apache.org/repos/asf/helix/blob/59391888/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
----------------------------------------------------------------------
diff --cc helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
index 9d56cdf,6d96849..9a527a2
--- a/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
+++ b/helix-provisioning/src/main/java/org/apache/helix/provisioning/yarn/AppMasterLauncher.java
@@@ -45,13 -45,12 +45,12 @@@ import org.apache.helix.api.config.Clus
  import org.apache.helix.api.config.ResourceConfig;
  import org.apache.helix.api.id.ClusterId;
  import org.apache.helix.api.id.ControllerId;
+ import org.apache.helix.api.id.PartitionId;
  import org.apache.helix.api.id.ResourceId;
- import org.apache.helix.controller.rebalancer.config.FullAutoRebalancerConfig;
- import org.apache.helix.controller.rebalancer.config.PartitionedRebalancerConfig;
- import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
 -import org.apache.helix.manager.zk.HelixConnectionAdaptor;
 +import org.apache.helix.manager.zk.ZKHelixManager;
  import org.apache.helix.manager.zk.ZkHelixConnection;
  import org.apache.helix.model.StateModelDefinition;
+ import org.apache.helix.model.builder.AutoRebalanceModeISBuilder;
  import org.apache.helix.provisioning.ApplicationSpec;
  import org.apache.helix.provisioning.ApplicationSpecFactory;
  import org.apache.helix.provisioning.HelixYarnUtil;


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

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 04c0352..495d37c 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
@@ -25,11 +25,9 @@ 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.manager.zk.MockController;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.testutil.ZkTestBase;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -54,9 +52,9 @@ public class TestStartMultipleControllersWithSameName extends ZkTestBase {
     // rebalance
 
     // start controller
-    ClusterControllerManager[] controllers = new ClusterControllerManager[4];
+    MockController[] controllers = new MockController[4];
     for (int i = 0; i < 4; i++) {
-      controllers[i] = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+      controllers[i] = new MockController(_zkaddr, clusterName, "controller_0");
       controllers[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
index e5ff171..c4304b0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
@@ -32,8 +32,8 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.api.State;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
-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.messaging.handling.MessageHandler.ErrorCode;
 import org.apache.helix.mock.participant.MockMSStateModel;
 import org.apache.helix.mock.participant.MockTransition;
@@ -172,7 +172,7 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
   @Test
   public void testStateTransitionTimeOut() throws Exception {
     Map<String, SleepStateModelFactory> factories = new HashMap<String, SleepStateModelFactory>();
-    // MockParticipantManager[] participants = new MockParticipantManager[NODE_NR];
+    // MockParticipant[] participants = new MockParticipant[NODE_NR];
     IdealState idealState =
         _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     for (int i = 0; i < NODE_NR; i++) {
@@ -185,12 +185,12 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
         }
       }
 
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
       _participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", factory);
       _participants[i].syncStart();
     }
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     boolean result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 283055c..1927d72 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
@@ -22,8 +22,7 @@ package org.apache.helix.integration;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -47,7 +46,6 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
     idealStateOld2.merge(is2.getRecord());
 
     String instanceName = "localhost_" + (START_PORT + 0);
-    ZKHelixAdmin tool = new ZKHelixAdmin(_zkclient);
     _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instanceName, false);
 
     boolean result =
@@ -77,8 +75,8 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
       exception = true;
     }
     Assert.assertFalse(exception);
-    MockParticipantManager newParticipant =
-        new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName2);
+    MockParticipant newParticipant =
+        new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName2);
     newParticipant.syncStart();
 
     result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
index 111c4d2..9e8fd85 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
@@ -31,11 +31,8 @@ import org.apache.helix.NotificationContext;
 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.integration.manager.ZkTestManager;
-import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -63,16 +60,16 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    final 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();
     }
 
@@ -81,7 +78,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
             .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(_zkaddr,
                 clusterName));
     Assert.assertTrue(result);
-    final MockParticipantManager participantManagerToExpire = participants[1];
+    final MockParticipant participantManagerToExpire = participants[1];
 
     // check controller zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -116,12 +113,12 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertTrue(result, "Participant should have 1 zk-watcher. MESSAGES->HelixTaskExecutor");
 
     // check HelixManager#_handlers
-    // printHandlers(controllerManager);
-    // printHandlers(participantManagerToExpire);
+    TestHelper.printHandlers(controller, controller.getHandlers());
+    TestHelper.printHandlers(participantManagerToExpire, participantManagerToExpire.getHandlers());
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManagerToExpire.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, 9,
-        "HelixController should have 9 (5+2n) callback handlers for 2 (n) participant");
+    Assert.assertEquals(controllerHandlerNb, 10,
+        "HelixController should have 10 (6+2n) callback handlers for 2 (n) participant");
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handlers");
 
@@ -208,16 +205,16 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    final 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();
     }
 
@@ -228,7 +225,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertTrue(result);
     // final ZkHelixTestManager controllerManager = controller.getManager();
     // final ZkHelixTestManager participantManager = participants[0].getManager();
-    final MockParticipantManager participantManager = participants[0];
+    final MockParticipant participantManager = participants[0];
 
     // wait until we get all the listeners registered
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -237,7 +234,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
       public boolean verify() throws Exception {
         int controllerHandlerNb = controller.getHandlers().size();
         int particHandlerNb = participantManager.getHandlers().size();
-        if (controllerHandlerNb == 9 && particHandlerNb == 2)
+        if (controllerHandlerNb == 10 && particHandlerNb == 2)
           return true;
         else
           return false;
@@ -246,12 +243,14 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
 
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManager.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, 9,
-        "HelixController should have 9 (5+2n) callback handlers for 2 participant, but was "
-            + controllerHandlerNb + ", " + printHandlers(controller));
+    TestHelper.printHandlers(controller, controller.getHandlers());
+    Assert.assertEquals(controllerHandlerNb, 10,
+        "HelixController should have 10 (6+2n) callback handlers for 2 participant, but was "
+            + controllerHandlerNb);
+    TestHelper.printHandlers(participantManager, participantManager.getHandlers());
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handler, but was "
-            + particHandlerNb + ", " + printHandlers(participantManager));
+            + particHandlerNb);
 
     // expire controller
     System.out.println("Expiring controller session...");
@@ -301,13 +300,13 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     // check HelixManager#_handlers
     // printHandlers(controllerManager);
     int handlerNb = controller.getHandlers().size();
+    TestHelper.printHandlers(controller, controller.getHandlers());
     Assert.assertEquals(handlerNb, controllerHandlerNb,
-        "controller callback handlers should not increase after participant session expiry, but was "
-            + printHandlers(controller));
+        "controller callback handlers should not increase after participant session expiry");
     handlerNb = participantManager.getHandlers().size();
+    TestHelper.printHandlers(participantManager, participantManager.getHandlers());
     Assert.assertEquals(handlerNb, particHandlerNb,
-        "participant callback handlers should not increase after participant session expiry, but was "
-            + printHandlers(participantManager));
+        "participant callback handlers should not increase after participant session expiry");
 
     // clean up
     controller.syncStop();
@@ -333,20 +332,19 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         2, // replicas
         "MasterSlave", true);
 
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(zkAddr, clusterName, "controller_0");
+    final MockController controller = new MockController(zkAddr, clusterName, "controller_0");
     controller.syncStart();
 
-    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();
 
       // register a controller listener on participant_0
       if (i == 0) {
         // ZkHelixTestManager manager = participants[0].getManager();
-        MockParticipantManager manager = participants[0];
+        MockParticipant manager = participants[0];
         manager.addCurrentStateChangeListener(new CurrentStateChangeListener() {
           @Override
           public void onStateChange(String instanceName, List<CurrentState> statesInfo,
@@ -365,7 +363,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
                 clusterName));
     Assert.assertTrue(result);
 
-    MockParticipantManager participantToExpire = participants[0];
+    MockParticipant participantToExpire = participants[0];
     String oldSessionId = participantToExpire.getSessionId();
     PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName);
 
@@ -446,8 +444,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertEquals(childListeners.get(path).size(), 1,
         "Should have 1 child-listener on path: " + path);
     path = keyBuilder.controller().getPath();
-    Assert.assertNull(childListeners.get(path),
-        "Should have no child-listener on path: " + path);
+    Assert.assertNull(childListeners.get(path), "Should have no child-listener on path: " + path);
 
     // check zookeeper#watches on client side
     watchPaths = ZkTestHelper.getZkWatch(participantToExpire.getZkClient());
@@ -494,51 +491,4 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
-
-  // debug code
-  static String printHandlers(ZkTestManager manager) {
-    StringBuilder sb = new StringBuilder();
-    List<CallbackHandler> handlers = manager.getHandlers();
-    sb.append(manager.getInstanceName() + " has " + handlers.size() + " cb-handlers. [");
-
-    for (int i = 0; i < handlers.size(); i++) {
-      CallbackHandler 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("]");
-
-    return sb.toString();
-  }
-
-  void printZkListeners(ZkClient client) throws Exception {
-    Map<String, Set<IZkDataListener>> datalisteners = ZkTestHelper.getZkDataListener(client);
-    Map<String, Set<IZkChildListener>> childListeners = ZkTestHelper.getZkChildListener(client);
-
-    System.out.println("dataListeners {");
-    for (String path : datalisteners.keySet()) {
-      System.out.println("\t" + path + ": ");
-      Set<IZkDataListener> set = datalisteners.get(path);
-      for (IZkDataListener listener : set) {
-        CallbackHandler handler = (CallbackHandler) listener;
-        System.out.println("\t\t" + handler.getListener());
-      }
-    }
-    System.out.println("}");
-
-    System.out.println("childListeners {");
-    for (String path : childListeners.keySet()) {
-      System.out.println("\t" + path + ": ");
-      Set<IZkChildListener> set = childListeners.get(path);
-      for (IZkChildListener listener : set) {
-        CallbackHandler handler = (CallbackHandler) listener;
-        System.out.println("\t\t" + handler.getListener());
-      }
-    }
-    System.out.println("}");
-  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
index 8b4e889..d1ae897 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkSessionExpiry.java
@@ -30,8 +30,8 @@ import org.apache.helix.NotificationContext;
 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.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
@@ -110,17 +110,17 @@ public class TestZkSessionExpiry 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
     Set<String> handledMsgSet = new HashSet<String>();
-    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].getMessagingService().registerMessageHandlerFactory(DUMMY_MSG_TYPE,
           new DummyMessageHandlerFactory(handledMsgSet));
       participants[i].syncStart();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java
index 8eaf2e7..3098b5c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBase.java
@@ -21,8 +21,8 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
-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.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -49,8 +49,8 @@ public class ZkStandAloneCMTestBase extends ZkTestBase {
   protected final String CLASS_NAME = this.getClass().getSimpleName();
   protected final String CLUSTER_NAME = CLASS_NAME;
 
-  protected MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
-  protected ClusterControllerManager _controller;
+  protected MockParticipant[] _participants = new MockParticipant[NODE_NR];
+  protected MockController _controller;
 
   int _replica = 3;
 
@@ -76,13 +76,13 @@ public class ZkStandAloneCMTestBase extends ZkTestBase {
     // 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();
     }
 
     // start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     boolean result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java
deleted file mode 100644
index b8f0f2b..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterControllerManager.java
+++ /dev/null
@@ -1,90 +0,0 @@
-package org.apache.helix.integration.manager;
-
-/*
- * 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.util.List;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.helix.HelixTimerTask;
-import org.apache.helix.InstanceType;
-import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.log4j.Logger;
-
-public class ClusterControllerManager extends ZKHelixManager implements Runnable, ZkTestManager {
-  private static Logger LOG = Logger.getLogger(ClusterControllerManager.class);
-
-  private final CountDownLatch _startCountDown = new CountDownLatch(1);
-  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
-
-  public ClusterControllerManager(String zkAddr, String clusterName, String controllerName) {
-    super(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr);
-  }
-
-  public void syncStop() {
-    _stopCountDown.countDown();
-    try {
-      _waitStopFinishCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("Interrupted waiting for finish", e);
-    }
-  }
-
-  public void syncStart() {
-    // TODO: prevent start multiple times
-    new Thread(this).start();
-    try {
-      _startCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("Interrupted waiting for start", e);
-    }
-  }
-
-  @Override
-  public void run() {
-    try {
-      connect();
-      _startCountDown.countDown();
-      _stopCountDown.await();
-    } catch (Exception e) {
-      LOG.error("exception running controller-manager", e);
-    } finally {
-      _startCountDown.countDown();
-      disconnect();
-      _waitStopFinishCountDown.countDown();
-    }
-  }
-
-  @Override
-  public ZkClient getZkClient() {
-    return _zkclient;
-  }
-
-  @Override
-  public List<CallbackHandler> getHandlers() {
-    return _handlers;
-  }
-
-  public List<HelixTimerTask> getControllerTimerTasks() {
-    return _controllerTimerTasks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java b/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java
deleted file mode 100644
index a17ccc1..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/ClusterDistributedController.java
+++ /dev/null
@@ -1,92 +0,0 @@
-package org.apache.helix.integration.manager;
-
-/*
- * 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.util.List;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.helix.InstanceType;
-import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.participant.DistClusterControllerStateModelFactory;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-
-public class ClusterDistributedController extends ZKHelixManager implements Runnable, ZkTestManager {
-  private static Logger LOG = Logger.getLogger(ClusterDistributedController.class);
-
-  private final CountDownLatch _startCountDown = new CountDownLatch(1);
-  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
-
-  public ClusterDistributedController(String zkAddr, String clusterName, String controllerName) {
-    super(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT, zkAddr);
-  }
-
-  public void syncStop() {
-    _stopCountDown.countDown();
-    try {
-      _waitStopFinishCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("Interrupted waiting for finish", e);
-    }
-  }
-
-  public void syncStart() {
-    // TODO: prevent start multiple times
-    new Thread(this).start();
-    try {
-      _startCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("Interrupted waiting for start", e);
-    }
-  }
-
-  @Override
-  public void run() {
-    try {
-      StateMachineEngine stateMach = getStateMachineEngine();
-      DistClusterControllerStateModelFactory lsModelFactory =
-          new DistClusterControllerStateModelFactory(_zkAddress);
-      stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory);
-
-      connect();
-      _startCountDown.countDown();
-      _stopCountDown.await();
-    } catch (Exception e) {
-      LOG.error("exception running controller-manager", e);
-    } finally {
-      _startCountDown.countDown();
-      disconnect();
-      _waitStopFinishCountDown.countDown();
-    }
-  }
-
-  @Override
-  public ZkClient getZkClient() {
-    return _zkclient;
-  }
-
-  @Override
-  public List<CallbackHandler> getHandlers() {
-    return _handlers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
deleted file mode 100644
index 917be17..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
+++ /dev/null
@@ -1,116 +0,0 @@
-package org.apache.helix.integration.manager;
-
-/*
- * 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.util.List;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.helix.InstanceType;
-import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory;
-import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory;
-import org.apache.helix.mock.participant.MockMSModelFactory;
-import org.apache.helix.mock.participant.MockSchemataModelFactory;
-import org.apache.helix.mock.participant.MockTransition;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-
-public class MockParticipantManager extends ZKHelixManager implements Runnable, ZkTestManager {
-  private static Logger LOG = Logger.getLogger(MockParticipantManager.class);
-
-  private final CountDownLatch _startCountDown = new CountDownLatch(1);
-  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private final CountDownLatch _waitStopCompleteCountDown = new CountDownLatch(1);
-
-  private final MockMSModelFactory _msModelFactory = new MockMSModelFactory(null);
-
-  public MockParticipantManager(String zkAddr, String clusterName, String instanceName) {
-    super(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr);
-  }
-
-  public void setTransition(MockTransition transition) {
-    _msModelFactory.setTrasition(transition);
-  }
-
-  public void syncStop() {
-    _stopCountDown.countDown();
-    try {
-      _waitStopCompleteCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("exception in syncStop participant-manager", e);
-    }
-  }
-
-  public void syncStart() {
-    try {
-      new Thread(this).start();
-      _startCountDown.await();
-    } catch (InterruptedException e) {
-      LOG.error("exception in syncStart participant-manager", e);
-    }
-  }
-
-  @Override
-  public void run() {
-    try {
-      StateMachineEngine stateMach = getStateMachineEngine();
-      stateMach.registerStateModelFactory("MasterSlave", _msModelFactory);
-
-      DummyLeaderStandbyStateModelFactory lsModelFactory =
-          new DummyLeaderStandbyStateModelFactory(10);
-      DummyOnlineOfflineStateModelFactory ofModelFactory =
-          new DummyOnlineOfflineStateModelFactory(10);
-      stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory);
-      stateMach.registerStateModelFactory("OnlineOffline", ofModelFactory);
-
-      MockSchemataModelFactory schemataFactory = new MockSchemataModelFactory();
-      stateMach.registerStateModelFactory("STORAGE_DEFAULT_SM_SCHEMATA", schemataFactory);
-
-      connect();
-      _startCountDown.countDown();
-
-      _stopCountDown.await();
-    } catch (InterruptedException e) {
-      String msg =
-          "participant: " + getInstanceName() + ", " + Thread.currentThread().getName()
-              + " is interrupted";
-      LOG.info(msg);
-    } catch (Exception e) {
-      LOG.error("exception running participant-manager", e);
-    } finally {
-      _startCountDown.countDown();
-
-      disconnect();
-      _waitStopCompleteCountDown.countDown();
-    }
-  }
-
-  @Override
-  public ZkClient getZkClient() {
-    return _zkclient;
-  }
-
-  @Override
-  public List<CallbackHandler> getHandlers() {
-    return _handlers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
index 877cf3c..99986ef 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
@@ -24,13 +24,15 @@ import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixProperty;
 import org.apache.helix.PreConnectCallback;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-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.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkCallbackHandler;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.HelixTestUtil;
@@ -97,19 +99,18 @@ public class TestConsecutiveZkSessionExpiry extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    final MockController controller = new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
     CountDownLatch startCountdown = new CountDownLatch(1);
     CountDownLatch endCountdown = new CountDownLatch(1);
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
 
       if (i == 0) {
         participants[i].addPreConnectCallback(new PreConnectTestCallback(instanceName,
@@ -178,14 +179,13 @@ public class TestConsecutiveZkSessionExpiry extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterDistributedController[] distributedControllers = new ClusterDistributedController[n];
+    MockMultiClusterController[] distributedControllers = new MockMultiClusterController[n];
     CountDownLatch startCountdown = new CountDownLatch(1);
     CountDownLatch endCountdown = new CountDownLatch(1);
 
     for (int i = 0; i < n; i++) {
       String contrllerName = "localhost_" + (12918 + i);
-      distributedControllers[i] =
-          new ClusterDistributedController(_zkaddr, clusterName, contrllerName);
+      distributedControllers[i] = new MockMultiClusterController(_zkaddr, clusterName, contrllerName);
       distributedControllers[i].getStateMachineEngine().registerStateModelFactory("MasterSlave",
           new MockMSModelFactory());
       if (i == 0) {
@@ -237,12 +237,15 @@ public class TestConsecutiveZkSessionExpiry extends ZkTestBase {
     Assert.assertNotNull(leader);
     Assert.assertEquals(leader.getId(), "localhost_12919");
 
-    // check localhost_12918 has 2 handlers: message and data-accessor
-    LOG.debug("handlers: " + TestHelper.printHandlers(distributedControllers[0]));
-    List<CallbackHandler> handlers = distributedControllers[0].getHandlers();
-    Assert.assertEquals(handlers.size(), 1,
-        "Distributed controller should have 1 handler (message) after lose leadership, but was "
-            + handlers.size());
+    // check localhost_12918 has 2 handlers: message and leader-election
+    TestHelper.printHandlers(distributedControllers[0], distributedControllers[0].getHandlers());
+    List<ZkCallbackHandler> handlers = distributedControllers[0].getHandlers();
+    Assert
+        .assertEquals(
+            handlers.size(),
+            2,
+            "Distributed controller should have 2 handler (message and leader election) after lose leadership, but was "
+                + handlers.size());
 
     // clean up
     distributedControllers[0].disconnect();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/TestControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestControllerManager.java
index 1544dc8..69d1dbe 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestControllerManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestControllerManager.java
@@ -24,6 +24,8 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -53,10 +55,10 @@ public class TestControllerManager extends ZkTestBase {
 
     // start multiple controllers of same name
     int m = 3;
-    ClusterControllerManager[] controllers = new ClusterControllerManager[m];
+    MockController[] controllers = new MockController[m];
 
     for (int i = 0; i < m; i++) {
-      controllers[i] = new ClusterControllerManager(_zkaddr, clusterName, "controller");
+      controllers[i] = new MockController(_zkaddr, clusterName, "controller");
       controllers[i].syncStart();
     }
 
@@ -69,11 +71,11 @@ public class TestControllerManager extends ZkTestBase {
     }
     Assert.assertEquals(leaderCnt, 1, "Should have only 1 leader but was " + leaderCnt);
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
 
       participants[i].syncStart();
     }
@@ -107,7 +109,7 @@ public class TestControllerManager 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
@@ -119,14 +121,14 @@ public class TestControllerManager extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     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/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 f915c4f..18234b5 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
@@ -27,9 +27,10 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.manager.zk.CallbackHandler;
+import org.apache.helix.manager.zk.MockMultiClusterController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKHelixManager;
+import org.apache.helix.manager.zk.ZkCallbackHandler;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.testutil.ZkTestBase;
@@ -109,8 +110,8 @@ public class TestDistributedControllerManager extends ZkTestBase {
    * @param newController
    * @throws Exception
    */
-  void expireController(ClusterDistributedController expireController,
-      ClusterDistributedController newController) throws Exception {
+  void expireController(MockMultiClusterController expireController,
+      MockMultiClusterController newController) throws Exception {
     String clusterName = expireController.getClusterName();
     LOG.info("Expiring distributedController: " + expireController.getInstanceName()
         + ", session: " + expireController.getSessionId() + " ...");
@@ -136,13 +137,12 @@ public class TestDistributedControllerManager extends ZkTestBase {
     Assert.assertNotNull(leader);
     Assert.assertEquals(leader.getId(), newController.getInstanceName());
 
-    // check expired-controller has 2 handlers: message and data-accessor
-    LOG.debug(expireController.getInstanceName() + " handlers: "
-        + TestHelper.printHandlers(expireController));
+    // check expired-controller has 2 handlers: message and leader-election
+    TestHelper.printHandlers(expireController, expireController.getHandlers());
 
-    List<CallbackHandler> handlers = expireController.getHandlers();
-    Assert.assertEquals(handlers.size(), 1,
-        "Distributed controller should have 1 handler (message) after lose leadership, but was "
+    List<ZkCallbackHandler> handlers = expireController.getHandlers();
+    Assert.assertEquals(handlers.size(), 2,
+        "Distributed controller should have 2 handler (message and leader-election) after lose leadership, but was "
             + handlers.size());
   }
 
@@ -165,12 +165,12 @@ public class TestDistributedControllerManager extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterDistributedController[] distributedControllers = new ClusterDistributedController[n];
+    MockMultiClusterController[] distributedControllers = new MockMultiClusterController[n];
 
     for (int i = 0; i < n; i++) {
       String contrllerName = "localhost_" + (12918 + i);
       distributedControllers[i] =
-          new ClusterDistributedController(_zkaddr, clusterName, contrllerName);
+          new MockMultiClusterController(_zkaddr, clusterName, contrllerName);
       distributedControllers[i].getStateMachineEngine().registerStateModelFactory("MasterSlave",
           new MockMSModelFactory());
       distributedControllers[i].connect();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 4d46883..309ab18 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
@@ -34,6 +34,8 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.api.id.PartitionId;
+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.ZKHelixManager;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
@@ -110,7 +112,7 @@ public class TestParticipantManager 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
@@ -122,14 +124,14 @@ public class TestParticipantManager 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();
     }
 
@@ -197,7 +199,7 @@ public class TestParticipantManager 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
@@ -209,14 +211,14 @@ public class TestParticipantManager 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].setTransition(new SessionExpiryTransition(startCountdown, endCountdown));
       participants[i].syncStart();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/TestStateModelLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestStateModelLeak.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestStateModelLeak.java
index b82f156..d6d7bab 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestStateModelLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestStateModelLeak.java
@@ -28,6 +28,8 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.TestHelper;
+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.mock.participant.ErrTransition;
 import org.apache.helix.participant.HelixStateMachineEngine;
@@ -70,15 +72,15 @@ public class TestStateModelLeak extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -144,15 +146,15 @@ public class TestStateModelLeak extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       if (i == 0) {
         Map<String, Set<String>> errTransitionMap = new HashMap<String, Set<String>>();
         Set<String> partitions = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
index 650f13f..1393231 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
@@ -31,6 +31,8 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -61,16 +63,15 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    final MockController controller = new MockController(_zkaddr, clusterName, "controller");
     controller.connect();
 
     // 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();
     }
 
@@ -98,7 +99,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertTrue(result, "Controller should have 6 + 5*n zk-watchers.");
 
     // check participant zk-watchers
-    final MockParticipantManager participantManagerToExpire = participants[0];
+    final MockParticipant participantManagerToExpire = participants[0];
     result = TestHelper.verify(new TestHelper.Verifier() {
 
       @Override
@@ -118,8 +119,8 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     // printHandlers(participantManagerToExpire);
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManagerToExpire.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, (5 + 2 * n),
-        "HelixController should have 9 (5+2n) callback handlers for 2 (n) participant");
+    Assert.assertEquals(controllerHandlerNb, (6 + 2 * n),
+        "HelixController should have 10 (5+2n) callback handlers for 2 (n) participant");
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handlers");
 
@@ -200,16 +201,15 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    final 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();
     }
 
@@ -220,14 +220,14 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertTrue(result);
 
     // wait until we get all the listeners registered
-    final MockParticipantManager participantManager = participants[0];
+    final MockParticipant participantManager = participants[0];
     result = TestHelper.verify(new TestHelper.Verifier() {
 
       @Override
       public boolean verify() throws Exception {
         int controllerHandlerNb = controller.getHandlers().size();
         int particHandlerNb = participantManager.getHandlers().size();
-        if (controllerHandlerNb == 9 && particHandlerNb == 2)
+        if (controllerHandlerNb == 10 && particHandlerNb == 2)
           return true;
         else
           return false;
@@ -236,12 +236,14 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
 
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManager.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, (5 + 2 * n),
-        "HelixController should have 9 (5+2n) callback handlers for 2 participant, but was "
-            + controllerHandlerNb + ", " + TestHelper.printHandlers(controller));
+    TestHelper.printHandlers(controller, controller.getHandlers());
+    TestHelper.printHandlers(participantManager, participantManager.getHandlers());
+    Assert.assertEquals(controllerHandlerNb, (6 + 2 * n),
+        "HelixController should have 10 (6+2n) callback handlers for 2 participant, but was "
+            + controllerHandlerNb);
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handler, but was "
-            + particHandlerNb + ", " + TestHelper.printHandlers(participantManager));
+            + particHandlerNb);
 
     // expire controller
     LOG.debug("Expiring controller session...");
@@ -292,13 +294,13 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     // check HelixManager#_handlers
     // printHandlers(controllerManager);
     int handlerNb = controller.getHandlers().size();
+    TestHelper.printHandlers(controller, controller.getHandlers());
     Assert.assertEquals(handlerNb, controllerHandlerNb,
-        "controller callback handlers should not increase after participant session expiry, but was "
-            + TestHelper.printHandlers(controller));
+        "controller callback handlers should not increase after participant session expiry");
     handlerNb = participantManager.getHandlers().size();
+    TestHelper.printHandlers(participantManager, participantManager.getHandlers());
     Assert.assertEquals(handlerNb, particHandlerNb,
-        "participant callback handlers should not increase after participant session expiry, but was "
-            + TestHelper.printHandlers(participantManager));
+        "participant callback handlers should not increase after participant session expiry");
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -318,19 +320,18 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
         2, // replicas
         "MasterSlave", true);
 
-    final ClusterControllerManager controller =
-        new ClusterControllerManager(zkAddr, clusterName, "controller");
+    final MockController controller = new MockController(zkAddr, clusterName, "controller");
     controller.syncStart();
 
-    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();
 
       // register a controller listener on participant_0
       if (i == 0) {
-        MockParticipantManager manager = participants[0];
+        MockParticipant manager = participants[0];
         manager.addCurrentStateChangeListener(new CurrentStateChangeListener() {
           @Override
           public void onStateChange(String instanceName, List<CurrentState> statesInfo,
@@ -349,7 +350,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
                 clusterName));
     Assert.assertTrue(result);
 
-    MockParticipantManager participantToExpire = participants[0];
+    MockParticipant participantToExpire = participants[0];
     String oldSessionId = participantToExpire.getSessionId();
     PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName);
 
@@ -371,9 +372,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertEquals(dataListeners.get(path).size(), 1, "Should have 1 data-listeners on path: "
         + path);
     Assert
-        .assertEquals(
-            childListeners.size(),
-            2,
+        .assertEquals(childListeners.size(), 2,
             "Should have 2 paths (CURRENTSTATE/{sessionId}, and MESSAGES) each of which has 1 child-listener");
     path = keyBuilder.currentStates(participantToExpire.getInstanceName(), oldSessionId).getPath();
     Assert.assertEquals(childListeners.get(path).size(), 1,
@@ -382,17 +381,14 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertEquals(childListeners.get(path).size(), 1,
         "Should have 1 child-listener on path: " + path);
     path = keyBuilder.controller().getPath();
-    Assert.assertNull(childListeners.get(path),
-        "Should have no child-listener on path: " + path);
+    Assert.assertNull(childListeners.get(path), "Should have no child-listener on path: " + path);
 
     // check zookeeper#watches on client side
     Map<String, List<String>> watchPaths =
         ZkTestHelper.getZkWatch(participantToExpire.getZkClient());
     LOG.debug("localhost_12918 zk-client side watchPaths: " + watchPaths);
     Assert
-        .assertEquals(
-            watchPaths.get("dataWatches").size(),
-            3,
+        .assertEquals(watchPaths.get("dataWatches").size(), 3,
             "Should have 3 data-watches: CURRENTSTATE/{sessionId}, CURRENTSTATE/{sessionId}/TestDB, MESSAGES");
     Assert.assertEquals(watchPaths.get("childWatches").size(), 2,
         "Should have 2 child-watches: MESSAGES, and CURRENTSTATE/{sessionId}");
@@ -435,8 +431,7 @@ public class TestZkCallbackHandlerLeak extends ZkTestBase {
     Assert.assertEquals(childListeners.get(path).size(), 1,
         "Should have 1 child-listener on path: " + path);
     path = keyBuilder.controller().getPath();
-    Assert.assertNull(childListeners.get(path),
-        "Should have no child-listener on path: " + path);
+    Assert.assertNull(childListeners.get(path), "Should have no child-listener on path: " + path);
 
     // check zookeeper#watches on client side
     watchPaths = ZkTestHelper.getZkWatch(participantToExpire.getZkClient());

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 2e2c8b6..22beed1 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
@@ -30,9 +30,9 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.integration.task.TestTaskRebalancerStopResume.ReindexTask;
+import org.apache.helix.manager.zk.MockParticipant;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.ScheduleConfig;
@@ -63,8 +63,8 @@ public class TestIndependentTaskRebalancer extends ZkTestBase {
   private static final int n = 5;
   private static final int START_PORT = 12918;
   private final String CLUSTER_NAME = "TestIndependentTaskRebalancer";
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
+  private final MockParticipant[] _participants = new MockParticipant[n];
+  private MockController _controller;
   private Set<String> _invokedClasses = Sets.newHashSet();
   private Map<String, Integer> _runCounts = Maps.newHashMap();
 
@@ -104,7 +104,7 @@ public class TestIndependentTaskRebalancer extends ZkTestBase {
         }
       });
 
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
 
       // Register a Task state model factory.
       StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
@@ -115,7 +115,7 @@ public class TestIndependentTaskRebalancer extends ZkTestBase {
 
     // Start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     // Start an admin connection

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
index e39615d..84c0e1d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancer.java
@@ -30,8 +30,8 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.api.id.StateModelDefId;
-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.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobContext;
@@ -66,8 +66,8 @@ public class TestTaskRebalancer extends ZkTestBase {
   private static final int NUM_PARTITIONS = 20;
   private static final int NUM_REPLICAS = 3;
   private final String CLUSTER_NAME = "TestTaskRebalancer";
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
+  private final MockParticipant[] _participants = new MockParticipant[n];
+  private MockController _controller;
 
   private HelixManager _manager;
   private TaskDriver _driver;
@@ -102,7 +102,7 @@ public class TestTaskRebalancer extends ZkTestBase {
     // start dummy participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
 
       // Register a Task state model factory.
       StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
@@ -113,7 +113,7 @@ public class TestTaskRebalancer extends ZkTestBase {
 
     // start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     // create cluster manager

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
index 6de361d..e3e8b89 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/TestTaskRebalancerStopResume.java
@@ -27,8 +27,8 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.api.id.StateModelDefId;
-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.participant.StateMachineEngine;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.Task;
@@ -60,8 +60,8 @@ public class TestTaskRebalancerStopResume extends ZkTestBase {
   private static final int NUM_PARTITIONS = 20;
   private static final int NUM_REPLICAS = 3;
   private final String CLUSTER_NAME = "TestTaskRebalancerStopResume";
-  private final MockParticipantManager[] _participants = new MockParticipantManager[n];
-  private ClusterControllerManager _controller;
+  private final MockParticipant[] _participants = new MockParticipant[n];
+  private MockController _controller;
 
   private HelixManager _manager;
   private TaskDriver _driver;
@@ -94,7 +94,7 @@ public class TestTaskRebalancerStopResume extends ZkTestBase {
     // start dummy participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
 
       // Register a Task state model factory.
       StateMachineEngine stateMachine = _participants[i].getStateMachineEngine();
@@ -106,7 +106,7 @@ public class TestTaskRebalancerStopResume extends ZkTestBase {
 
     // start controller
     String controllerName = "controller_0";
-    _controller = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+    _controller = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
     // create cluster manager

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/MockListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/MockListener.java b/helix-core/src/test/java/org/apache/helix/manager/MockListener.java
index 376481e..d6edf17 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/MockListener.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/MockListener.java
@@ -21,11 +21,11 @@ package org.apache.helix.manager;
 
 import java.util.List;
 
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.IdealStateChangeListener;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.NotificationContext;
@@ -37,7 +37,7 @@ import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 
 public class MockListener implements IdealStateChangeListener, LiveInstanceChangeListener,
-    ConfigChangeListener, CurrentStateChangeListener, ExternalViewChangeListener,
+    InstanceConfigChangeListener, CurrentStateChangeListener, ExternalViewChangeListener,
     ControllerChangeListener, MessageListener
 
 {
@@ -71,7 +71,7 @@ public class MockListener implements IdealStateChangeListener, LiveInstanceChang
   }
 
   @Override
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
+  public void onInstanceConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
     isConfigChangeListenerInvoked = true;
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/MockController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/MockController.java b/helix-core/src/test/java/org/apache/helix/manager/zk/MockController.java
new file mode 100644
index 0000000..f4d2159
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/MockController.java
@@ -0,0 +1,86 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.helix.InstanceType;
+import org.apache.log4j.Logger;
+
+public class MockController extends ZKHelixManager implements Runnable {
+  private static Logger LOG = Logger.getLogger(MockController.class);
+
+  private final CountDownLatch _startCountDown = new CountDownLatch(1);
+  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
+  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
+
+  public MockController(String zkAddr, String clusterName, String controllerName) {
+    super(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr);
+  }
+
+  public void syncStop() {
+    _stopCountDown.countDown();
+    try {
+      _waitStopFinishCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted waiting for finish", e);
+    }
+  }
+
+  public void syncStart() {
+    // TODO: prevent start multiple times
+    new Thread(this).start();
+    try {
+      _startCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted waiting for start", e);
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      connect();
+      _startCountDown.countDown();
+      _stopCountDown.await();
+    } catch (Exception e) {
+      LOG.error("exception running controller-manager", e);
+    } finally {
+      _startCountDown.countDown();
+      disconnect();
+      _waitStopFinishCountDown.countDown();
+    }
+  }
+
+  public ZkClient getZkClient() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    return conn._zkclient;
+  }
+
+  public ZkHelixConnection getConn() {
+    return (ZkHelixConnection)_role.getConnection();
+  }
+
+  public List<ZkCallbackHandler> getHandlers() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    return conn._handlers.get(_role);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/MockMultiClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/MockMultiClusterController.java b/helix-core/src/test/java/org/apache/helix/manager/zk/MockMultiClusterController.java
new file mode 100644
index 0000000..7f8b1a3
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/MockMultiClusterController.java
@@ -0,0 +1,99 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.helix.InstanceType;
+import org.apache.helix.participant.DistClusterControllerStateModelFactory;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.log4j.Logger;
+
+public class MockMultiClusterController extends ZKHelixManager implements Runnable {
+  private static Logger LOG = Logger.getLogger(MockMultiClusterController.class);
+
+  private final CountDownLatch _startCountDown = new CountDownLatch(1);
+  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
+  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
+
+  public MockMultiClusterController(String zkAddr, String clusterName, String controllerName) {
+    super(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT, zkAddr);
+  }
+
+  public void syncStop() {
+    _stopCountDown.countDown();
+    try {
+      _waitStopFinishCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted waiting for finish", e);
+    }
+  }
+
+  public void syncStart() {
+    // TODO: prevent start multiple times
+    new Thread(this).start();
+    try {
+      _startCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted waiting for start", e);
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      StateMachineEngine stateMach = getStateMachineEngine();
+      DistClusterControllerStateModelFactory lsModelFactory =
+          new DistClusterControllerStateModelFactory(_zkAddress);
+      stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory);
+
+      connect();
+      _startCountDown.countDown();
+      _stopCountDown.await();
+    } catch (Exception e) {
+      LOG.error("exception running controller-manager", e);
+    } finally {
+      _startCountDown.countDown();
+      disconnect();
+      _waitStopFinishCountDown.countDown();
+    }
+  }
+
+  public ZkHelixConnection getConn() {
+    return (ZkHelixConnection)_role.getConnection();
+  }
+
+  public ZkClient getZkClient() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    return conn._zkclient;
+  }
+
+  public List<ZkCallbackHandler> getHandlers() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    List<ZkCallbackHandler> handlers = new ArrayList<ZkCallbackHandler>();
+    for (List<ZkCallbackHandler> handlerList : conn._handlers.values()) {
+      handlers.addAll(handlerList);
+    }
+
+    return handlers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/MockParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/MockParticipant.java b/helix-core/src/test/java/org/apache/helix/manager/zk/MockParticipant.java
new file mode 100644
index 0000000..f107d3d
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/MockParticipant.java
@@ -0,0 +1,119 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.helix.HelixConnection;
+import org.apache.helix.InstanceType;
+import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory;
+import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory;
+import org.apache.helix.mock.participant.MockMSModelFactory;
+import org.apache.helix.mock.participant.MockSchemataModelFactory;
+import org.apache.helix.mock.participant.MockTransition;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.log4j.Logger;
+
+public class MockParticipant extends ZKHelixManager implements Runnable {
+  private static Logger LOG = Logger.getLogger(MockParticipant.class);
+
+  private final CountDownLatch _startCountDown = new CountDownLatch(1);
+  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
+  private final CountDownLatch _waitStopCompleteCountDown = new CountDownLatch(1);
+
+  private final MockMSModelFactory _msModelFactory = new MockMSModelFactory(null);
+
+
+  public MockParticipant(String zkAddress, String clusterName, String instanceName) {
+    super(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddress);
+  }
+
+  public void setTransition(MockTransition transition) {
+    _msModelFactory.setTrasition(transition);
+  }
+
+  public void syncStop() {
+    _stopCountDown.countDown();
+    try {
+      _waitStopCompleteCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("exception in syncStop participant-manager", e);
+    }
+  }
+
+  public void syncStart() {
+    try {
+      new Thread(this).start();
+      _startCountDown.await();
+    } catch (InterruptedException e) {
+      LOG.error("exception in syncStart participant-manager", e);
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      StateMachineEngine stateMach = getStateMachineEngine();
+      stateMach.registerStateModelFactory("MasterSlave", _msModelFactory);
+
+      DummyLeaderStandbyStateModelFactory lsModelFactory =
+          new DummyLeaderStandbyStateModelFactory(10);
+      DummyOnlineOfflineStateModelFactory ofModelFactory =
+          new DummyOnlineOfflineStateModelFactory(10);
+      stateMach.registerStateModelFactory("LeaderStandby", lsModelFactory);
+      stateMach.registerStateModelFactory("OnlineOffline", ofModelFactory);
+
+      MockSchemataModelFactory schemataFactory = new MockSchemataModelFactory();
+      stateMach.registerStateModelFactory("STORAGE_DEFAULT_SM_SCHEMATA", schemataFactory);
+
+      connect();
+      _startCountDown.countDown();
+
+      _stopCountDown.await();
+    } catch (InterruptedException e) {
+      String msg =
+          "participant: " + getInstanceName() + ", " + Thread.currentThread().getName()
+              + " is interrupted";
+      LOG.info(msg);
+    } catch (Exception e) {
+      LOG.error("exception running participant-manager", e);
+    } finally {
+      _startCountDown.countDown();
+
+      disconnect();
+      _waitStopCompleteCountDown.countDown();
+    }
+  }
+
+  public HelixConnection getConn() {
+    return _role.getConnection();
+  }
+
+  public ZkClient getZkClient() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    return conn._zkclient;
+  }
+
+  public List<ZkCallbackHandler> getHandlers() {
+    ZkHelixConnection conn = (ZkHelixConnection)getConn();
+    return conn._handlers.get(_role);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 ca0d4ab..3b5b24e 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,7 +23,6 @@ import java.util.Date;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.testutil.ZkTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -47,8 +46,8 @@ public class TestHandleNewSession extends ZkTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    MockParticipantManager participant =
-        new MockParticipantManager(_zkaddr, clusterName, "localhost_12918");
+    MockParticipant participant =
+        new MockParticipant(_zkaddr, clusterName, "localhost_12918");
     participant.syncStart();
 
     // Logger.getRootLogger().setLevel(Level.INFO);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java
index e59dd0c..7e2f4a7 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestLiveInstanceBounce.java
@@ -20,7 +20,6 @@ package org.apache.helix.manager.zk;
  */
 
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -40,7 +39,7 @@ public class TestLiveInstanceBounce extends ZkStandAloneCMTestBase {
         e.printStackTrace();
       }
       // restart the participant
-      _participants[i] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+      _participants[i] = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
       _participants[i].syncStart();
       Thread.sleep(100);
     }


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

Posted by zz...@apache.org.
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) {


[6/7] git commit: [HELIX-376] Remove HelixConnection/HelixManager duplicate code

Posted by zz...@apache.org.
[HELIX-376] Remove HelixConnection/HelixManager duplicate code


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

Branch: refs/heads/master
Commit: 02165c5203ffd226ad23d310ee3e027c9b761f65
Parents: 961b930
Author: zzhang <zz...@apache.org>
Authored: Tue Jul 29 17:04:22 2014 -0700
Committer: zzhang <zz...@apache.org>
Committed: Mon Aug 4 13:09:34 2014 -0700

----------------------------------------------------------------------
 .../helix/webapp/resources/ClusterResource.java |   5 +-
 .../helix/webapp/resources/ConfigResource.java  |   3 -
 .../webapp/resources/ConstraintResource.java    |   3 -
 .../webapp/resources/ControllerResource.java    |   5 +-
 .../webapp/TestHelixAdminScenariosRest.java     | 112 +--
 .../apache/helix/webapp/TestResetInstance.java  |  12 +-
 .../helix/webapp/TestResetPartitionState.java   |  14 +-
 .../apache/helix/webapp/TestResetResource.java  |  12 +-
 .../org/apache/helix/agent/TestHelixAgent.java  |   4 +-
 .../java/org/apache/helix/ConfigAccessor.java   | 234 ------
 .../org/apache/helix/ConfigChangeListener.java  |  38 -
 .../org/apache/helix/HelixAutoController.java   |  62 --
 .../java/org/apache/helix/HelixConnection.java  |  15 +-
 .../java/org/apache/helix/HelixManager.java     |  12 +-
 .../helix/HelixMultiClusterController.java      |  62 ++
 .../main/java/org/apache/helix/HelixRole.java   |   5 +
 .../java/org/apache/helix/HelixService.java     |   6 +
 .../apache/helix/api/id/AdministratorId.java    |  51 ++
 .../controller/GenericHelixController.java      |  15 +-
 .../helix/controller/HelixControllerMain.java   |   2 +-
 .../stages/BestPossibleStateCalcStage.java      |   2 +-
 .../controller/stages/TaskAssignmentStage.java  |   8 +-
 .../helix/manager/zk/CallbackHandler.java       |  13 +-
 .../manager/zk/ControllerManagerHelper.java     |   2 +-
 .../manager/zk/HelixConnectionAdaptor.java      | 304 -------
 .../apache/helix/manager/zk/ZKHelixManager.java | 808 ++++---------------
 .../helix/manager/zk/ZkCallbackHandler.java     |  16 +-
 .../helix/manager/zk/ZkHelixAutoController.java | 133 ---
 .../helix/manager/zk/ZkHelixConnection.java     | 160 ++--
 .../helix/manager/zk/ZkHelixController.java     |  20 +-
 .../zk/ZkHelixMultiClusterController.java       | 145 ++++
 .../helix/manager/zk/ZkHelixParticipant.java    |  15 +-
 .../manager/zk/ZkHelixRoleDefaultImpl.java      | 126 +++
 .../messaging/DefaultMessagingService.java      |  16 +-
 .../messaging/handling/HelixTaskExecutor.java   |   1 -
 .../org/apache/helix/model/ConfigScope.java     | 137 ----
 .../helix/model/builder/ConfigScopeBuilder.java | 127 ---
 .../helix/participant/CustomCodeInvoker.java    |   6 +-
 .../participant/GenericLeaderStandbyModel.java  |   2 +-
 .../helix/spectator/RoutingTableProvider.java   |   6 +-
 .../src/test/java/org/apache/helix/Mocks.java   |   6 -
 .../org/apache/helix/TestConfigAccessor.java    |  78 +-
 .../test/java/org/apache/helix/TestHelper.java  |  14 +-
 .../java/org/apache/helix/TestZKCallback.java   |   6 +-
 .../java/org/apache/helix/ZkTestHelper.java     |   1 -
 .../org/apache/helix/api/TestNewStages.java     |  12 +-
 .../controller/stages/DummyClusterManager.java  |   7 -
 .../stages/TestRebalancePipeline.java           |   6 +-
 .../helix/integration/IntegrationTest.java      |  52 +-
 .../helix/integration/TestAddClusterV2.java     |  12 +-
 .../TestAddNodeAfterControllerStart.java        |  32 +-
 .../TestAddStateModelFactoryAfterConnect.java   |  13 +-
 .../integration/TestAutoIsWithEmptyMap.java     |  12 +-
 .../helix/integration/TestAutoRebalance.java    |  14 +-
 .../TestAutoRebalancePartitionLimit.java        |  14 +-
 .../helix/integration/TestBasicSpectator.java   |   2 +-
 .../helix/integration/TestBatchMessage.java     |  40 +-
 .../integration/TestBatchMessageWrapper.java    |  12 +-
 .../integration/TestBucketizedResource.java     |  12 +-
 .../integration/TestCarryOverBadCurState.java   |  12 +-
 .../integration/TestCleanupExternalView.java    |  12 +-
 .../helix/integration/TestClusterStartsup.java  |   1 -
 .../TestCorrectnessOnConnectivityLoss.java      |   8 +-
 .../helix/integration/TestCustomIdealState.java |   1 -
 .../TestCustomizedIdealStateRebalancer.java     |   1 -
 .../apache/helix/integration/TestDisable.java   |  36 +-
 .../TestDisableCustomCodeRunner.java            |  16 +-
 .../helix/integration/TestDisableResource.java  |  31 +-
 .../integration/TestDistributedCMMain.java      |  15 +-
 .../TestDistributedClusterController.java       |  16 +-
 .../apache/helix/integration/TestDriver.java    |  24 +-
 .../org/apache/helix/integration/TestDrop.java  |  62 +-
 .../helix/integration/TestDropResource.java     |   4 +-
 .../TestEnablePartitionDuringDisable.java       |  14 +-
 .../integration/TestEntropyFreeNodeBounce.java  |  15 +-
 .../helix/integration/TestErrorPartition.java   |  16 +-
 .../integration/TestExternalViewUpdates.java    |  12 +-
 .../integration/TestFullAutoNodeTagging.java    |  28 +-
 .../integration/TestHelixCustomCodeRunner.java  |  12 +-
 .../helix/integration/TestInstanceAutoJoin.java |  15 +-
 .../integration/TestInvalidAutoIdealState.java  |  12 +-
 .../TestInvalidResourceRebalance.java           |  12 +-
 .../helix/integration/TestMessageThrottle.java  |  13 +-
 .../helix/integration/TestMessageThrottle2.java |  26 +-
 .../helix/integration/TestMessagingService.java |  10 +-
 .../integration/TestNonOfflineInitState.java    |  13 +-
 .../helix/integration/TestNullReplica.java      |  12 +-
 .../TestParticipantNameCollision.java           |   6 +-
 .../TestPartitionLevelTransitionConstraint.java |  17 +-
 .../helix/integration/TestPauseSignal.java      |  12 +-
 .../integration/TestPreferenceListAsQueue.java  |   1 -
 .../integration/TestRedefineStateModelDef.java  |  14 +-
 .../TestReelectedPipelineCorrectness.java       |  13 +-
 .../helix/integration/TestRenamePartition.java  |  26 +-
 .../helix/integration/TestResetInstance.java    |  14 +-
 .../integration/TestResetPartitionState.java    |  14 +-
 .../helix/integration/TestResetResource.java    |  14 +-
 .../integration/TestRestartParticipant.java     |  26 +-
 .../helix/integration/TestSchemataSM.java       |  14 +-
 .../TestSessionExpiryInTransition.java          |  22 +-
 .../helix/integration/TestSharedConnection.java |   5 +-
 .../helix/integration/TestStandAloneCMMain.java |   6 +-
 .../TestStandAloneCMSessionExpiry.java          |  15 +-
 ...estStartMultipleControllersWithSameName.java |   8 +-
 .../integration/TestStateTransitionTimeout.java |  10 +-
 .../helix/integration/TestSwapInstance.java     |   8 +-
 .../integration/TestZkCallbackHandlerLeak.java  | 116 +--
 .../helix/integration/TestZkSessionExpiry.java  |  12 +-
 .../integration/ZkStandAloneCMTestBase.java     |  12 +-
 .../manager/ClusterControllerManager.java       |  90 ---
 .../manager/ClusterDistributedController.java   |  92 ---
 .../manager/MockParticipantManager.java         | 116 ---
 .../manager/TestConsecutiveZkSessionExpiry.java |  33 +-
 .../manager/TestControllerManager.java          |  18 +-
 .../TestDistributedControllerManager.java       |  22 +-
 .../manager/TestParticipantManager.java         |  18 +-
 .../integration/manager/TestStateModelLeak.java |  18 +-
 .../manager/TestZkCallbackHandlerLeak.java      |  69 +-
 .../task/TestIndependentTaskRebalancer.java     |  12 +-
 .../integration/task/TestTaskRebalancer.java    |  12 +-
 .../task/TestTaskRebalancerStopResume.java      |  12 +-
 .../org/apache/helix/manager/MockListener.java  |   6 +-
 .../apache/helix/manager/zk/MockController.java |  86 ++
 .../manager/zk/MockMultiClusterController.java  |  99 +++
 .../helix/manager/zk/MockParticipant.java       | 119 +++
 .../helix/manager/zk/TestHandleNewSession.java  |   5 +-
 .../manager/zk/TestLiveInstanceBounce.java      |   3 +-
 .../manager/zk/TestZKLiveInstanceData.java      |  13 -
 .../helix/manager/zk/TestZkClusterManager.java  |  39 +-
 .../apache/helix/manager/zk/TestZkFlapping.java |  10 +-
 .../manager/zk/TestZkHelixAutoController.java   |   4 +-
 .../zk/TestZkManagerFlappingDetection.java      |   6 +-
 .../manager/zk/TestZkStateChangeListener.java   |  13 +-
 .../helix/manager/zk/ZkConnTestHelper.java      |  34 +
 .../handling/TestConfigThreadpoolSize.java      |  15 +-
 .../handling/TestResourceThreadpoolSize.java    |  10 +-
 .../helix/mock/controller/MockController.java   | 140 ----
 .../mock/controller/MockControllerProcess.java  |  76 --
 .../org/apache/helix/model/TestConstraint.java  |   2 -
 .../TestClusterStatusMonitorLifecycle.java      |  25 +-
 .../mbeans/TestDropResourceMetricsReset.java    |  14 +-
 .../mbeans/TestResetClusterMetrics.java         |  12 +-
 .../helix/participant/MockZKHelixManager.java   |   7 -
 .../zk/TestZkManagerWithAutoFallbackStore.java  |   6 +-
 .../org/apache/helix/testutil/TestUtil.java     |   4 +-
 .../helix/tools/TestClusterStateVerifier.java   |  16 +-
 .../apache/helix/tools/TestHelixAdminCli.java   |  36 +-
 .../provisioning/yarn/AppMasterLauncher.java    |   4 +-
 .../yarn/example/JobRunnerMain.java             |  10 +-
 .../yarn/example/MyTaskService.java             |   4 +-
 150 files changed, 1905 insertions(+), 3323 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java
index b22d801..dda0794 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ClusterResource.java
@@ -20,9 +20,7 @@ package org.apache.helix.webapp.resources;
  */
 
 import java.io.IOException;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
@@ -35,7 +33,6 @@ import org.apache.helix.webapp.RestAdminApplication;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.restlet.data.MediaType;
-import org.restlet.data.Method;
 import org.restlet.data.Status;
 import org.restlet.representation.Representation;
 import org.restlet.representation.StringRepresentation;
@@ -43,7 +40,7 @@ import org.restlet.representation.Variant;
 import org.restlet.resource.ServerResource;
 
 public class ClusterResource extends ServerResource {
-    
+
   public ClusterResource() {
     getVariants().add(new Variant(MediaType.TEXT_PLAIN));
     getVariants().add(new Variant(MediaType.APPLICATION_JSON));

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java
index 3c384d4..3a0d83a 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConfigResource.java
@@ -33,9 +33,6 @@ import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.webapp.RestAdminApplication;
 import org.apache.log4j.Logger;
-import org.restlet.Context;
-import org.restlet.Request;
-import org.restlet.Response;
 import org.restlet.data.MediaType;
 import org.restlet.data.Status;
 import org.restlet.representation.Representation;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java
index 675d0ec..afa2454 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ConstraintResource.java
@@ -29,9 +29,6 @@ import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.webapp.RestAdminApplication;
 import org.apache.log4j.Logger;
-import org.restlet.Context;
-import org.restlet.Request;
-import org.restlet.Response;
 import org.restlet.data.MediaType;
 import org.restlet.data.Status;
 import org.restlet.representation.Representation;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java
index ea7be42..3ac4151 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/ControllerResource.java
@@ -40,9 +40,6 @@ import org.apache.helix.util.StatusUpdateUtil.Level;
 import org.apache.helix.webapp.RestAdminApplication;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
-import org.restlet.Context;
-import org.restlet.Request;
-import org.restlet.Response;
 import org.restlet.data.MediaType;
 import org.restlet.data.Status;
 import org.restlet.representation.Representation;
@@ -135,7 +132,7 @@ public class ControllerResource extends ServerResource {
           MediaType.APPLICATION_JSON);
       getResponse().setStatus(Status.SUCCESS_OK);
     }
-    
+
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestHelixAdminScenariosRest.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestHelixAdminScenariosRest.java b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestHelixAdminScenariosRest.java
index 66065c3..5aa26d2 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestHelixAdminScenariosRest.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestHelixAdminScenariosRest.java
@@ -31,9 +31,9 @@ import java.util.Map;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.ZNRecord;
-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.MockParticipant;
+import org.apache.helix.manager.zk.MockMultiClusterController;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -330,10 +330,10 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     final String clusterName = "clusterTestDeactivateCluster";
     final String controllerClusterName = "controllerClusterTestDeactivateCluster";
 
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
-    Map<String, ClusterDistributedController> distControllers =
-        new HashMap<String, ClusterDistributedController>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
+    Map<String, MockMultiClusterController> distControllers =
+        new HashMap<String, MockMultiClusterController>();
 
     // setup cluster
     addCluster(clusterName);
@@ -347,8 +347,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     // start mock nodes
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -356,8 +356,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     // start controller nodes
     for (int i = 0; i < 2; i++) {
       String controllerName = "controller_900" + i;
-      ClusterDistributedController distController =
-          new ClusterDistributedController(_zkaddr, controllerClusterName, controllerName);
+      MockMultiClusterController distController =
+          new MockMultiClusterController(_zkaddr, controllerClusterName, controllerName);
       distController.syncStart();
       distControllers.put(controllerName, distController);
     }
@@ -390,7 +390,7 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Assert.assertTrue(_zkclient.exists("/" + clusterName));
 
     // leader node should be gone
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
     deleteUrl(clusterUrl, false);
@@ -398,11 +398,11 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Assert.assertFalse(_zkclient.exists("/" + clusterName));
 
     // clean up
-    for (ClusterDistributedController controller : distControllers.values()) {
+    for (MockMultiClusterController controller : distControllers.values()) {
       controller.syncStop();
     }
 
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
   }
@@ -433,17 +433,17 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     pw.write(x);
     pw.close();
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_9900");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_9900");
     controller.syncStart();
 
     // start mock nodes
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -478,7 +478,7 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     // clean up
     controller.syncStop();
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
   }
@@ -509,17 +509,17 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     addResource(clusterName, "db_11", 22);
     rebalanceResource(clusterName, "db_11");
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_9900");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_9900");
     controller.syncStart();
 
     // start mock nodes
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -543,8 +543,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     for (int i = 3; i <= 6; i++) {
       String instanceName = "localhost_123" + i + "1";
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -561,7 +561,7 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     // clean up
     controller.syncStop();
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
   }
@@ -587,17 +587,17 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     addResource(clusterName, "db_11", 22);
     rebalanceResource(clusterName, "db_11");
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_9900");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_9900");
     controller.syncStart();
 
     // start mock nodes
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -641,7 +641,7 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     // clean up
     controller.syncStop();
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
   }
@@ -673,17 +673,17 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     addResource(clusterName, "db_11", 8);
     rebalanceResource(clusterName, "db_11");
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_9900");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_9900");
     controller.syncStart();
 
     // start mock nodes
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -723,8 +723,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     String path = accessor.keyBuilder().instanceConfig("localhost_1232").getPath();
     Assert.assertFalse(_zkclient.exists(path));
 
-    MockParticipantManager newParticipant =
-        new MockParticipantManager(_zkaddr, clusterName, "localhost_12320");
+    MockParticipant newParticipant =
+        new MockParticipant(_zkaddr, clusterName, "localhost_12320");
     newParticipant.syncStart();
     participants.put("localhost_12320", newParticipant);
 
@@ -735,7 +735,7 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     // clean up
     controller.syncStop();
-    for (MockParticipantManager participant : participants.values()) {
+    for (MockParticipant participant : participants.values()) {
       participant.syncStop();
     }
   }
@@ -745,10 +745,10 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     final String clusterName = "clusterTestStartCluster";
     final String controllerClusterName = "controllerClusterTestStartCluster";
 
-    Map<String, MockParticipantManager> participants =
-        new HashMap<String, MockParticipantManager>();
-    Map<String, ClusterDistributedController> distControllers =
-        new HashMap<String, ClusterDistributedController>();
+    Map<String, MockParticipant> participants =
+        new HashMap<String, MockParticipant>();
+    Map<String, MockMultiClusterController> distControllers =
+        new HashMap<String, MockMultiClusterController>();
 
     // setup cluster
     addCluster(clusterName);
@@ -762,8 +762,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     // start mock nodes
     for (int i = 0; i < 6; i++) {
       String instanceName = "localhost_123" + i;
-      MockParticipantManager participant =
-          new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      MockParticipant participant =
+          new MockParticipant(_zkaddr, clusterName, instanceName);
       participant.syncStart();
       participants.put(instanceName, participant);
     }
@@ -771,8 +771,8 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     // start controller nodes
     for (int i = 0; i < 2; i++) {
       String controllerName = "controller_900" + i;
-      ClusterDistributedController distController =
-          new ClusterDistributedController(_zkaddr, controllerClusterName, controllerName);
+      MockMultiClusterController distController =
+          new MockMultiClusterController(_zkaddr, controllerClusterName, controllerName);
       distController.syncStart();
       distControllers.put(controllerName, distController);
     }
@@ -821,10 +821,10 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Thread.sleep(1000);
 
     // clean up
-    for (ClusterDistributedController controller : distControllers.values()) {
+    for (MockMultiClusterController controller : distControllers.values()) {
       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-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetInstance.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetInstance.java b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetInstance.java
index b89a067..a9ecaa0 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetInstance.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetInstance.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 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.mock.participant.ErrTransition;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -54,7 +54,7 @@ public class TestResetInstance extends AdminTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller = new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -65,16 +65,16 @@ public class TestResetInstance extends AdminTestBase {
     };
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipantManager(_zkaddr, clusterName, instanceName);
+            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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetPartitionState.java b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetPartitionState.java
index 8cd6f42..d44f3bb 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetPartitionState.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetPartitionState.java
@@ -28,12 +28,10 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.helix.NotificationContext;
 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.mock.participant.ErrTransition;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
@@ -98,7 +96,7 @@ public class TestResetPartitionState extends AdminTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller = new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>();
@@ -106,16 +104,16 @@ public class TestResetPartitionState extends AdminTestBase {
     errPartitions.put("OFFLINE-SLAVE", TestHelper.setOf("TestDB0_8"));
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipantManager(_zkaddr, clusterName, instanceName);
+            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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetResource.java b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetResource.java
index 464edc4..a54b0a3 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetResource.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/webapp/TestResetResource.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 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.mock.participant.ErrTransition;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -54,7 +54,7 @@ public class TestResetResource extends AdminTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller = new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -65,16 +65,16 @@ public class TestResetResource extends AdminTestBase {
     };
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipantManager(_zkaddr, clusterName, instanceName);
+            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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java
----------------------------------------------------------------------
diff --git a/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java b/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java
index cbf0582..2904803 100644
--- a/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java
+++ b/helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java
@@ -28,7 +28,7 @@ import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ExternalCommand;
 import org.apache.helix.ScriptTestHelper;
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
@@ -147,7 +147,7 @@ public class TestHelixAgent extends ZkTestBase {
     configAccessor.set(scope, cmdConfig.toKeyValueMap());
 
     // start controller
-    ClusterControllerManager controller = new ClusterControllerManager(zkAddr, clusterName, "controller_0");
+    MockController controller = new MockController(zkAddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start helix-agent

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java b/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java
index 3589165..406aeb1 100644
--- a/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/ConfigAccessor.java
@@ -30,7 +30,6 @@ import java.util.TreeMap;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.util.StringTemplate;
@@ -73,66 +72,6 @@ public class ConfigAccessor {
   }
 
   /**
-   * get config
-   * @deprecated replaced by {@link #get(HelixConfigScope, String)}
-   * @param scope
-   * @param key
-   * @return value or null if doesn't exist
-   */
-  @Deprecated
-  public String get(ConfigScope scope, String key) {
-    Map<String, String> map = get(scope, Arrays.asList(key));
-    return map.get(key);
-  }
-
-  /**
-   * get configs
-   * @deprecated replaced by {@link #get(HelixConfigScope, List<String>)}
-   * @param scope
-   * @param keys
-   * @return
-   */
-  @Deprecated
-  public Map<String, String> get(ConfigScope scope, List<String> keys) {
-    if (scope == null || scope.getScope() == null) {
-      LOG.error("Scope can't be null");
-      return null;
-    }
-
-    // String value = null;
-    Map<String, String> map = new HashMap<String, String>();
-    String clusterName = scope.getClusterName();
-    if (!ZKUtil.isClusterSetup(clusterName, zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    String scopeStr = scope.getScopeStr();
-    String[] splits = scopeStr.split("\\|");
-
-    ZNRecord record = zkClient.readData(splits[0], true);
-
-    if (record != null) {
-      if (splits.length == 1) {
-        for (String key : keys) {
-          if (record.getSimpleFields().containsKey(key)) {
-            map.put(key, record.getSimpleField(key));
-          }
-        }
-      } else if (splits.length == 2) {
-        if (record.getMapField(splits[1]) != null) {
-          for (String key : keys) {
-            if (record.getMapField(splits[1]).containsKey(key)) {
-              map.put(key, record.getMapField(splits[1]).get(key));
-            }
-          }
-        }
-      }
-    }
-    return map;
-
-  }
-
-  /**
    * get a single config entry
    * @param scope specification of the entity set to query
    *          (e.g. cluster, resource, participant, etc.)
@@ -198,72 +137,6 @@ public class ConfigAccessor {
   }
 
   /**
-   * Set config, create if not exist
-   * @deprecated replaced by {@link #set(HelixConfigScope, String, String)}
-   * @param scope
-   * @param key
-   * @param value
-   */
-  @Deprecated
-  public void set(ConfigScope scope, String key, String value) {
-    Map<String, String> map = new HashMap<String, String>();
-    map.put(key, value);
-    set(scope, map);
-  }
-
-  /**
-   * Set configs, create if not exist
-   * @deprecated replaced by {@link #set(HelixConfigScope, Map<String, String>)}
-   * @param scope
-   * @param keyValueMap
-   */
-  @Deprecated
-  public void set(ConfigScope scope, Map<String, String> keyValueMap) {
-    if (scope == null || scope.getScope() == null) {
-      LOG.error("Scope can't be null");
-      return;
-    }
-
-    String clusterName = scope.getClusterName();
-    if (!ZKUtil.isClusterSetup(clusterName, zkClient)) {
-      throw new HelixException("cluster: " + clusterName + " is NOT setup.");
-    }
-
-    if (scope.getScope() == ConfigScopeProperty.PARTICIPANT) {
-      String scopeStr = scope.getScopeStr();
-      String instanceName = scopeStr.substring(scopeStr.lastIndexOf('/') + 1);
-      if (!ZKUtil.isInstanceSetup(zkClient, scope.getClusterName(), instanceName,
-          InstanceType.PARTICIPANT)) {
-        throw new HelixException("instance: " + instanceName + " is NOT setup in cluster: "
-            + clusterName);
-      }
-    }
-
-    // use "|" to delimit resource and partition. e.g. /MyCluster/CONFIGS/PARTICIPANT/MyDB|MyDB_0
-    String scopeStr = scope.getScopeStr();
-    String[] splits = scopeStr.split("\\|");
-
-    String id = splits[0].substring(splits[0].lastIndexOf('/') + 1);
-    ZNRecord update = new ZNRecord(id);
-    if (splits.length == 1) {
-      for (String key : keyValueMap.keySet()) {
-        String value = keyValueMap.get(key);
-        update.setSimpleField(key, value);
-      }
-    } else if (splits.length == 2) {
-      if (update.getMapField(splits[1]) == null) {
-        update.setMapField(splits[1], new TreeMap<String, String>());
-      }
-      for (String key : keyValueMap.keySet()) {
-        String value = keyValueMap.get(key);
-        update.getMapField(splits[1]).put(key, value);
-      }
-    }
-    ZKUtil.createOrUpdate(zkClient, splits[0], update, true, true);
-    return;
-  }
-
-  /**
    * Set config, creating it if it doesn't exist
    * @param scope scope specification of the entity set to query
    *          (e.g. cluster, resource, participant, etc.)
@@ -315,59 +188,6 @@ public class ConfigAccessor {
   }
 
   /**
-   * Remove config
-   * @deprecated replaced by {@link #remove(HelixConfigScope, String)}
-   * @param scope
-   * @param key
-   */
-  @Deprecated
-  public void remove(ConfigScope scope, String key) {
-    remove(scope, Arrays.asList(key));
-  }
-
-  /**
-   * remove configs
-   * @deprecated replaced by {@link #remove(HelixConfigScope, List<String>)}
-   * @param scope
-   * @param keys
-   */
-  @Deprecated
-  public void remove(ConfigScope scope, List<String> keys) {
-    if (scope == null || scope.getScope() == null) {
-      LOG.error("Scope can't be null");
-      return;
-    }
-
-    String clusterName = scope.getClusterName();
-    if (!ZKUtil.isClusterSetup(clusterName, zkClient)) {
-      throw new HelixException("cluster " + clusterName + " is not setup yet");
-    }
-
-    String scopeStr = scope.getScopeStr();
-    String[] splits = scopeStr.split("\\|");
-
-    String id = splits[0].substring(splits[0].lastIndexOf('/') + 1);
-    ZNRecord update = new ZNRecord(id);
-    if (splits.length == 1) {
-      // subtract doesn't care about value, use empty string
-      for (String key : keys) {
-        update.setSimpleField(key, "");
-      }
-    } else if (splits.length == 2) {
-      if (update.getMapField(splits[1]) == null) {
-        update.setMapField(splits[1], new TreeMap<String, String>());
-      }
-      // subtract doesn't care about value, use empty string
-      for (String key : keys) {
-        update.getMapField(splits[1]).put(key, "");
-      }
-    }
-
-    ZKUtil.subtract(zkClient, splits[0], update);
-    return;
-  }
-
-  /**
    * Remove a single config
    * @param scope scope specification of the entity set to query
    *          (e.g. cluster, resource, participant, etc.)
@@ -416,60 +236,6 @@ public class ConfigAccessor {
   }
 
   /**
-   * get config keys
-   * @deprecated replaced by {@link #getKeys(HelixConfigScope)}
-   * @param type
-   * @param clusterName
-   * @param keys
-   * @return
-   */
-  @Deprecated
-  public List<String> getKeys(ConfigScopeProperty type, String clusterName, String... keys) {
-    if (type == null || clusterName == null) {
-      LOG.error("clusterName|scope can't be null");
-      return Collections.emptyList();
-    }
-
-    try {
-      if (!ZKUtil.isClusterSetup(clusterName, zkClient)) {
-        LOG.error("cluster " + clusterName + " is not setup yet");
-        return Collections.emptyList();
-      }
-
-      String[] args = new String[1 + keys.length];
-      args[0] = clusterName;
-      System.arraycopy(keys, 0, args, 1, keys.length);
-      String scopeStr = template.instantiate(type, args);
-      String[] splits = scopeStr.split("\\|");
-      List<String> retKeys = null;
-      if (splits.length == 1) {
-        retKeys = zkClient.getChildren(splits[0]);
-      } else {
-        ZNRecord record = zkClient.readData(splits[0]);
-
-        if (splits[1].startsWith("SIMPLEKEYS")) {
-          retKeys = new ArrayList<String>(record.getSimpleFields().keySet());
-
-        } else if (splits[1].startsWith("MAPKEYS")) {
-          retKeys = new ArrayList<String>(record.getMapFields().keySet());
-        } else if (splits[1].startsWith("MAPMAPKEYS")) {
-          retKeys = new ArrayList<String>(record.getMapField(splits[2]).keySet());
-        }
-      }
-      if (retKeys == null) {
-        LOG.error("Invalid scope: " + type + " or keys: " + Arrays.toString(args));
-        return Collections.emptyList();
-      }
-
-      Collections.sort(retKeys);
-      return retKeys;
-    } catch (Exception e) {
-      return Collections.emptyList();
-    }
-
-  }
-
-  /**
    * Get list of config keys for a scope
    * @param scope
    * @return a list of configuration keys

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java b/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
deleted file mode 100644
index 1dbf2fe..0000000
--- a/helix-core/src/main/java/org/apache/helix/ConfigChangeListener.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package org.apache.helix;
-
-/*
- * 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.util.List;
-
-import org.apache.helix.model.InstanceConfig;
-
-/**
- * @deprecated replaced by InstanceConfigChangeListener
- */
-public interface ConfigChangeListener {
-
-  /**
-   * Invoked when participant config changes
-   * @param configs
-   * @param changeContext
-   */
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext);
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixAutoController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixAutoController.java b/helix-core/src/main/java/org/apache/helix/HelixAutoController.java
deleted file mode 100644
index 91ec809..0000000
--- a/helix-core/src/main/java/org/apache/helix/HelixAutoController.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.helix;
-
-/*
- * 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 org.apache.helix.api.id.ControllerId;
-import org.apache.helix.participant.StateMachineEngine;
-
-/**
- * Autonomous controller
- */
-public interface HelixAutoController extends HelixRole, HelixService, HelixConnectionStateListener {
-  /**
-   * get controller id
-   * @return controller id
-   */
-  ControllerId getControllerId();
-
-  /**
-   * get state machine engine
-   * @return state machine engine
-   */
-  StateMachineEngine getStateMachineEngine();
-
-  /**
-   * add pre-connect callback
-   * @param callback
-   */
-  void addPreConnectCallback(PreConnectCallback callback);
-
-  /**
-   * Add a LiveInstanceInfoProvider that is invoked before creating liveInstance.</br>
-   * This allows applications to provide additional information that will be published to zookeeper
-   * and become available for discovery</br>
-   * @see LiveInstanceInfoProvider#getAdditionalLiveInstanceInfo()
-   * @param liveInstanceInfoProvider
-   */
-  void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider);
-
-  /**
-   * tell if this controller is leader of cluster
-   * @return
-   */
-  boolean isLeader();
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixConnection.java b/helix-core/src/main/java/org/apache/helix/HelixConnection.java
index ff5f458..4b7487b 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixConnection.java
@@ -82,7 +82,7 @@ public interface HelixConnection {
    * @param controllerId
    * @return
    */
-  HelixAutoController createAutoController(ClusterId clusterId, ControllerId controllerId);
+  HelixMultiClusterController createAutoController(ClusterId clusterId, ControllerId controllerId);
 
   /**
    * create a cluster-accessor
@@ -113,10 +113,8 @@ public interface HelixConnection {
 
   /**
    * get config accessor
-   * TODO replace with new ConfigAccessor
    * @return config accessor
    */
-  @Deprecated
   ConfigAccessor getConfigAccessor();
 
   /**
@@ -164,15 +162,6 @@ public interface HelixConnection {
       ParticipantId participantId);
 
   /**
-   * add config change listener
-   * @param role
-   * @param listener
-   * @param clusterId
-   */
-  @Deprecated
-  void addConfigChangeListener(HelixRole role, ConfigChangeListener listener, ClusterId clusterId);
-
-  /**
    * add instance config change listener
    * @see InstanceConfigChangeListener#onInstanceConfigChange(List, NotificationContext)
    * @param role
@@ -234,7 +223,7 @@ public interface HelixConnection {
   void removeConnectionStateListener(HelixConnectionStateListener listener);
 
   /**
-   * create messasing service using this connection
+   * create messaging service using this connection
    * @param role
    * @return messaging-service
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixManager.java b/helix-core/src/main/java/org/apache/helix/HelixManager.java
index 73313c0..9e2bd14 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -33,7 +33,7 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
  * Class that represents the Helix Agent.
  * First class Object any process will interact with<br/>
  * General flow <blockquote>
- * 
+ *
  * <pre>
  * manager = HelixManagerFactory.getZKHelixManager(
  *    clusterName, instanceName, ROLE, zkAddr);
@@ -49,7 +49,7 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
  * FINALIZE -> will be invoked when listener is removed or session expires
  * manager.disconnect()
  * </pre>
- * 
+ *
  * </blockquote> Default implementations available
  * @see HelixStateMachineEngine HelixStateMachineEngine for participant
  * @see RoutingTableProvider RoutingTableProvider for spectator
@@ -98,14 +98,6 @@ public interface HelixManager {
   void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception;
 
   /**
-   * @see ConfigChangeListener#onConfigChange(List, NotificationContext)
-   * @param listener
-   * @deprecated replaced by addInstanceConfigChangeListener()
-   */
-  @Deprecated
-  void addConfigChangeListener(ConfigChangeListener listener) throws Exception;
-
-  /**
    * @see InstanceConfigChangeListener#onInstanceConfigChange(List, NotificationContext)
    * @param listener
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixMultiClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixMultiClusterController.java b/helix-core/src/main/java/org/apache/helix/HelixMultiClusterController.java
new file mode 100644
index 0000000..c4a587e
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/HelixMultiClusterController.java
@@ -0,0 +1,62 @@
+package org.apache.helix;
+
+/*
+ * 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 org.apache.helix.api.id.ControllerId;
+import org.apache.helix.participant.StateMachineEngine;
+
+/**
+ * Autonomous controller
+ */
+public interface HelixMultiClusterController extends HelixRole, HelixService, HelixConnectionStateListener {
+  /**
+   * get controller id
+   * @return controller id
+   */
+  ControllerId getControllerId();
+
+  /**
+   * get state machine engine
+   * @return state machine engine
+   */
+  StateMachineEngine getStateMachineEngine();
+
+  /**
+   * add pre-connect callback
+   * @param callback
+   */
+  void addPreConnectCallback(PreConnectCallback callback);
+
+  /**
+   * Add a LiveInstanceInfoProvider that is invoked before creating liveInstance.</br>
+   * This allows applications to provide additional information that will be published to zookeeper
+   * and become available for discovery</br>
+   * @see LiveInstanceInfoProvider#getAdditionalLiveInstanceInfo()
+   * @param liveInstanceInfoProvider
+   */
+  void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider);
+
+  /**
+   * tell if this controller is leader of cluster
+   * @return
+   */
+  boolean isLeader();
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixRole.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixRole.java b/helix-core/src/main/java/org/apache/helix/HelixRole.java
index ffcb700..dd77cc3 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixRole.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixRole.java
@@ -56,4 +56,9 @@ public interface HelixRole {
    */
   ClusterMessagingService getMessagingService();
 
+  /**
+   * get data accessor
+   * @return
+   */
+  HelixDataAccessor getAccessor();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/HelixService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixService.java b/helix-core/src/main/java/org/apache/helix/HelixService.java
index 40e9bae..37baa42 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixService.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixService.java
@@ -32,4 +32,10 @@ public interface HelixService {
    * stop helix service
    */
   void stop();
+
+  /**
+   * is service started
+   * @return
+   */
+  boolean isStarted();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/api/id/AdministratorId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/AdministratorId.java b/helix-core/src/main/java/org/apache/helix/api/id/AdministratorId.java
new file mode 100644
index 0000000..de6bcda
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/AdministratorId.java
@@ -0,0 +1,51 @@
+package org.apache.helix.api.id;
+
+/*
+ * 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 org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class AdministratorId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a spectator id
+   * @param id string representing a spectator id
+   */
+  @JsonCreator
+  public AdministratorId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Create a spectator id from a string
+   * @param spectatorId string representing a spectator id
+   * @return SpectatorId
+   */
+  public static AdministratorId from(String spectatorId) {
+    return new AdministratorId(spectatorId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index 7bb214e..f1c2583 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -29,7 +29,6 @@ import java.util.TimerTask;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.I0Itec.zkclient.exception.ZkInterruptedException;
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -86,7 +85,7 @@ import org.apache.log4j.Logger;
  * 4. select the messages that can be sent, needs messages and state model constraints <br>
  * 5. send messages
  */
-public class GenericHelixController implements ConfigChangeListener, IdealStateChangeListener,
+public class GenericHelixController implements IdealStateChangeListener,
     LiveInstanceChangeListener, MessageListener, CurrentStateChangeListener,
     ExternalViewChangeListener, ControllerChangeListener, InstanceConfigChangeListener {
   private static final Logger logger = Logger.getLogger(GenericHelixController.class.getName());
@@ -433,8 +432,8 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
   }
 
   @Override
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
-    logger.info("START: GenericClusterController.onConfigChange()");
+  public void onInstanceConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
+    logger.info("START: GenericClusterController.onInstanceConfigChange()");
     if (changeContext == null || changeContext.getType() != Type.CALLBACK) {
       _cache.requireFullRefresh();
     }
@@ -449,14 +448,6 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
     event.addAttribute("helixmanager", changeContext.getManager());
     event.addAttribute("eventData", configs);
     _eventQueue.put(event);
-    logger.info("END: GenericClusterController.onConfigChange()");
-  }
-
-  @Override
-  public void onInstanceConfigChange(List<InstanceConfig> instanceConfigs,
-      NotificationContext changeContext) {
-    logger.info("START: GenericClusterController.onInstanceConfigChange()");
-    onConfigChange(instanceConfigs, changeContext);
     logger.info("END: GenericClusterController.onInstanceConfigChange()");
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
index b6c16b5..6aa3ab9 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
@@ -132,7 +132,7 @@ public class HelixControllerMain {
   public static void addListenersToController(HelixManager manager,
       GenericHelixController controller) {
     try {
-      manager.addConfigChangeListener(controller);
+      manager.addInstanceConfigChangeListener(controller);
       manager.addLiveInstanceChangeListener(controller);
       manager.addIdealStateChangeListener(controller);
       // no need for controller to listen on external-view

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 6f34953..d23e011 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -157,7 +157,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       Set<ParticipantId> errorParticipants = Sets.newHashSet();
       for (ParticipantId participantId : currentStateMap.keySet()) {
         State state = currentStateMap.get(participantId);
-        if (state.equals(State.from(HelixDefinedState.ERROR))) {
+        if (State.from(HelixDefinedState.ERROR).equals(state)) {
           errorParticipants.add(participantId);
         }
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
index 9d6228e..4057389 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
@@ -149,10 +149,10 @@ public class TaskAssignmentStage extends AbstractBaseStage {
           + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:"
           + message.getTypedFromState() + " to:" + message.getTypedToState());
 
-      // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to "
-      // + message.getTgtName() + " transit " + message.getPartitionId() + "|"
-      // + message.getPartitionIds() + " from: " + message.getFromState() + " to: "
-      // + message.getToState());
+//       System.out.println("[dbg] Sending Message " + message.getMsgId() + " to "
+//       + message.getTgtName() + " transit " + message.getPartitionId() + "|"
+//       + message.getPartitionIds() + " from: " + message.getFromState() + " to: "
+//       + message.getToState());
 
       keys.add(keyBuilder.message(message.getTgtName(), message.getId()));
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
index 65fe2f9..cdb2845 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
@@ -37,7 +37,6 @@ import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -150,15 +149,9 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener
 
       } else if (_changeType == ChangeType.INSTANCE_CONFIG) {
         subscribeForChanges(changeContext, _path, true, true);
-        if (_listener instanceof ConfigChangeListener) {
-          ConfigChangeListener configChangeListener = (ConfigChangeListener) _listener;
-          List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
-          configChangeListener.onConfigChange(configs, changeContext);
-        } else if (_listener instanceof InstanceConfigChangeListener) {
-          InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener;
-          List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
-          listener.onInstanceConfigChange(configs, changeContext);
-        }
+        InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener;
+        List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
+        listener.onInstanceConfigChange(configs, changeContext);
       } else if (_changeType == CONFIG) {
         subscribeForChanges(changeContext, _path, true, true);
         ScopedConfigChangeListener listener = (ScopedConfigChangeListener) _listener;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
index 9a817e3..623b874 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
@@ -72,7 +72,7 @@ public class ControllerManagerHelper {
       /**
        * setup generic-controller
        */
-      _manager.addConfigChangeListener(controller);
+      _manager.addInstanceConfigChangeListener(controller);
       _manager.addLiveInstanceChangeListener(controller);
       _manager.addIdealStateChangeListener(controller);
       // no need for controller to listen on external-view

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
deleted file mode 100644
index ef17715..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixConnectionAdaptor.java
+++ /dev/null
@@ -1,304 +0,0 @@
-package org.apache.helix.manager.zk;
-
-/*
- * 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 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;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixAutoController;
-import org.apache.helix.HelixConnection;
-import org.apache.helix.HelixController;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerProperties;
-import org.apache.helix.HelixParticipant;
-import org.apache.helix.HelixRole;
-import org.apache.helix.IdealStateChangeListener;
-import org.apache.helix.InstanceConfigChangeListener;
-import org.apache.helix.InstanceType;
-import org.apache.helix.LiveInstanceChangeListener;
-import org.apache.helix.LiveInstanceInfoProvider;
-import org.apache.helix.MessageListener;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ScopedConfigChangeListener;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.api.id.ClusterId;
-import org.apache.helix.api.id.Id;
-import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.api.id.SessionId;
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.apache.log4j.Logger;
-
-/**
- * Adapt helix-connection to helix-manager, so we can pass to callback-handler and
- * notification-context
- */
-public class HelixConnectionAdaptor implements HelixManager {
-  private static Logger LOG = Logger.getLogger(HelixConnectionAdaptor.class);
-
-  final HelixRole _role;
-  final HelixConnection _connection;
-  final ClusterId _clusterId;
-  final Id _instanceId;
-  final InstanceType _instanceType;
-  final HelixDataAccessor _accessor;
-  final ClusterMessagingService _messagingService;
-
-  public HelixConnectionAdaptor(HelixRole role) {
-    _role = role;
-    _connection = role.getConnection();
-    _clusterId = role.getClusterId();
-    _accessor = _connection.createDataAccessor(_clusterId);
-
-    _instanceId = role.getId();
-    _instanceType = role.getType();
-    _messagingService = role.getMessagingService();
-  }
-
-  @Override
-  public void connect() throws Exception {
-    _connection.connect();
-  }
-
-  @Override
-  public boolean isConnected() {
-    return _connection.isConnected();
-  }
-
-  @Override
-  public void disconnect() {
-    _connection.disconnect();
-  }
-
-  @Override
-  public void addIdealStateChangeListener(IdealStateChangeListener listener) throws Exception {
-    _connection.addIdealStateChangeListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception {
-    _connection.addLiveInstanceChangeListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public void addConfigChangeListener(ConfigChangeListener listener) throws Exception {
-    _connection.addConfigChangeListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
-      throws Exception {
-    _connection.addInstanceConfigChangeListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
-      throws Exception {
-    _connection.addConfigChangeListener(_role, listener, _clusterId, scope);
-  }
-
-  @Override
-  public void addMessageListener(MessageListener listener, String instanceName) throws Exception {
-    _connection.addMessageListener(_role, listener, _clusterId, ParticipantId.from(instanceName));
-  }
-
-  @Override
-  public void addCurrentStateChangeListener(CurrentStateChangeListener listener,
-      String instanceName, String sessionId) throws Exception {
-    _connection.addCurrentStateChangeListener(_role, listener, _clusterId,
-        ParticipantId.from(instanceName), SessionId.from(sessionId));
-  }
-
-  @Override
-  public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
-    _connection.addExternalViewChangeListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public void addControllerListener(ControllerChangeListener listener) {
-    _connection.addControllerListener(_role, listener, _clusterId);
-  }
-
-  @Override
-  public boolean removeListener(PropertyKey key, Object listener) {
-    return _connection.removeListener(_role, listener, key);
-  }
-
-  @Override
-  public HelixDataAccessor getHelixDataAccessor() {
-    return _accessor;
-  }
-
-  @Override
-  public ConfigAccessor getConfigAccessor() {
-    return _connection.getConfigAccessor();
-  }
-
-  @Override
-  public String getClusterName() {
-    return _clusterId.stringify();
-  }
-
-  @Override
-  public String getInstanceName() {
-    return _instanceId.stringify();
-  }
-
-  @Override
-  public String getSessionId() {
-    return _connection.getSessionId().stringify();
-  }
-
-  @Override
-  public long getLastNotificationTime() {
-    return 0;
-  }
-
-  @Override
-  public HelixAdmin getClusterManagmentTool() {
-    return _connection.createClusterManagementTool();
-  }
-
-  @Override
-  public ZkHelixPropertyStore<ZNRecord> getHelixPropertyStore() {
-    return (ZkHelixPropertyStore<ZNRecord>) _connection.createPropertyStore(_clusterId);
-  }
-
-  @Override
-  public ClusterMessagingService getMessagingService() {
-    return _messagingService;
-  }
-
-  @Override
-  public InstanceType getInstanceType() {
-    return _instanceType;
-  }
-
-  @Override
-  public String getVersion() {
-    return _connection.getHelixVersion();
-  }
-
-  @Override
-  public HelixManagerProperties getProperties() {
-    return _connection.getHelixProperties();
-  }
-
-  @Override
-  public StateMachineEngine getStateMachineEngine() {
-    StateMachineEngine engine = null;
-    switch (_role.getType()) {
-    case PARTICIPANT:
-      HelixParticipant participant = (HelixParticipant) _role;
-      engine = participant.getStateMachineEngine();
-      break;
-    case CONTROLLER_PARTICIPANT:
-      HelixAutoController autoController = (HelixAutoController) _role;
-      engine = autoController.getStateMachineEngine();
-      break;
-    default:
-      LOG.info("helix manager type: " + _role.getType() + " does NOT have state-machine-engine");
-      break;
-    }
-
-    return engine;
-  }
-
-  @Override
-  public boolean isLeader() {
-    boolean isLeader = false;
-    switch (_role.getType()) {
-    case CONTROLLER:
-      HelixController controller = (HelixController) _role;
-      isLeader = controller.isLeader();
-      break;
-    case CONTROLLER_PARTICIPANT:
-      HelixAutoController autoController = (HelixAutoController) _role;
-      isLeader = autoController.isLeader();
-      break;
-    default:
-      LOG.info("helix manager type: " + _role.getType() + " does NOT support leadership");
-      break;
-    }
-    return isLeader;
-  }
-
-  @Override
-  public void startTimerTasks() {
-    throw new UnsupportedOperationException(
-        "HelixConnectionAdaptor does NOT support start timer tasks");
-  }
-
-  @Override
-  public void stopTimerTasks() {
-    throw new UnsupportedOperationException(
-        "HelixConnectionAdaptor does NOT support stop timer tasks");
-  }
-
-  @Override
-  public void addPreConnectCallback(PreConnectCallback callback) {
-    switch (_role.getType()) {
-    case PARTICIPANT:
-      HelixParticipant participant = (HelixParticipant) _role;
-      participant.addPreConnectCallback(callback);
-      break;
-    case CONTROLLER_PARTICIPANT:
-      HelixAutoController autoController = (HelixAutoController) _role;
-      autoController.addPreConnectCallback(callback);
-      break;
-    default:
-      LOG.info("helix manager type: " + _role.getType()
-          + " does NOT support add pre-connect callback");
-      break;
-    }
-  }
-
-  @Override
-  public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
-    switch (_role.getType()) {
-    case PARTICIPANT:
-      HelixParticipant participant = (HelixParticipant) _role;
-      participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
-      break;
-    case CONTROLLER_PARTICIPANT:
-      HelixAutoController autoController = (HelixAutoController) _role;
-      autoController.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
-      break;
-    default:
-      LOG.info("helix manager type: " + _role.getType()
-          + " does NOT support set additional live instance information");
-      break;
-    }
-  }
-
-  @Override
-  public void addControllerMessageListener(MessageListener listener) {
-    // TODO Auto-generated method stub
-
-  }
-
-}


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

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
index f95f6ee..12a3fe6 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -21,27 +21,22 @@ package org.apache.helix.manager.zk;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Timer;
-import java.util.concurrent.TimeUnit;
-
-import org.I0Itec.zkclient.IZkStateListener;
-import org.I0Itec.zkclient.ZkConnection;
-import org.apache.helix.BaseDataAccessor;
+
 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;
 import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixConstants.ChangeType;
+import org.apache.helix.HelixMultiClusterController;
+import org.apache.helix.HelixConnection;
+import org.apache.helix.HelixController;
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerProperties;
-import org.apache.helix.HelixTimerTask;
+import org.apache.helix.HelixParticipant;
+import org.apache.helix.HelixRole;
+import org.apache.helix.HelixService;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
@@ -50,26 +45,20 @@ import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.PropertyPathConfig;
-import org.apache.helix.PropertyType;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.messaging.DefaultMessagingService;
+import org.apache.helix.api.id.AdministratorId;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.ControllerId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.SpectatorId;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.monitoring.ZKPathDataDumpTask;
-import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.helix.participant.StateMachineEngine;
-import org.apache.helix.store.zk.AutoFallbackPropertyStore;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.log4j.Logger;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper.States;
 
-public class ZKHelixManager implements HelixManager, IZkStateListener {
+public class ZKHelixManager implements HelixManager {
   private static Logger LOG = Logger.getLogger(ZKHelixManager.class);
 
   public static final int FLAPPING_TIME_WINDIOW = 300000; // Default to 300 sec
@@ -77,89 +66,11 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "allowParticipantAutoJoin";
 
   protected final String _zkAddress;
-  private final String _clusterName;
-  private final String _instanceName;
-  private final InstanceType _instanceType;
-  private final int _sessionTimeout;
-  private final List<PreConnectCallback> _preConnectCallbacks;
-  protected final List<CallbackHandler> _handlers;
-  private final HelixManagerProperties _properties;
-
-  /**
-   * helix version#
-   */
-  private final String _version;
-
-  protected ZkClient _zkclient = null;
-  private final DefaultMessagingService _messagingService;
-
-  private BaseDataAccessor<ZNRecord> _baseDataAccessor;
-  private ZKHelixDataAccessor _dataAccessor;
-  private final Builder _keyBuilder;
-  private ConfigAccessor _configAccessor;
-  private ZkHelixPropertyStore<ZNRecord> _helixPropertyStore;
-  protected LiveInstanceInfoProvider _liveInstanceInfoProvider = null;
-
-  private volatile String _sessionId;
-
-  /**
-   * Keep track of timestamps that zk State has become Disconnected
-   * If in a _timeWindowLengthMs window zk State has become Disconnected
-   * for more than_maxDisconnectThreshold times disconnect the zkHelixManager
-   */
-  private final List<Long> _disconnectTimeHistory = new ArrayList<Long>();
-  private final int _flappingTimeWindowMs;
-  private final int _maxDisconnectThreshold;
-
-  /**
-   * participant fields
-   */
-  private final StateMachineEngine _stateMachineEngine;
-  private final List<HelixTimerTask> _timerTasks = new ArrayList<HelixTimerTask>();
-
-  /**
-   * controller fields
-   */
-  private final GenericHelixController _controller = new GenericHelixController();
-  private CallbackHandler _leaderElectionHandler = null;
-  protected final List<HelixTimerTask> _controllerTimerTasks = new ArrayList<HelixTimerTask>();
-
-  /**
-   * status dump timer-task
-   */
-  static class StatusDumpTask extends HelixTimerTask {
-    Timer _timer = null;
-    final HelixManager helixController;
-
-    public StatusDumpTask(HelixManager helixController) {
-      this.helixController = helixController;
-    }
-
-    @Override
-    public void start() {
-      long initialDelay = 0;
-      long period = 15 * 60 * 1000;
-      long timeThresholdNoChangeForStatusUpdates = 15 * 60 * 1000; // 15 minutes
-      long timeThresholdNoChangeForErrors = 24 * 60 * 60 * 1000; // 1 day
-      int maximumNumberOfLeafNodesAllowed = 10000;
-
-      if (_timer == null) {
-        LOG.info("Start StatusDumpTask");
-        _timer = new Timer("StatusDumpTimerTask", true);
-        _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController,
-            timeThresholdNoChangeForStatusUpdates, timeThresholdNoChangeForErrors,
-            maximumNumberOfLeafNodesAllowed), initialDelay, period);
-      }
-    }
+  protected final HelixRole _role; // keep it protected for test purpose
 
-    @Override
-    public void stop() {
-      if (_timer != null) {
-        LOG.info("Stop StatusDumpTask");
-        _timer.cancel();
-        _timer = null;
-      }
-    }
+  public ZKHelixManager(HelixRole role) {
+    _role = role;
+    _zkAddress = null;
   }
 
   public ZKHelixManager(String clusterName, String instanceName, InstanceType instanceType,
@@ -169,9 +80,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
         + clusterName + ", instanceName: " + instanceName + ", type: " + instanceType);
 
     _zkAddress = zkAddress;
-    _clusterName = clusterName;
-    _instanceType = instanceType;
-
+    ClusterId clusterId = ClusterId.from(clusterName);
     if (instanceName == null) {
       try {
         instanceName =
@@ -183,678 +92,281 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
       }
     }
 
-    _instanceName = instanceName;
-    _preConnectCallbacks = new ArrayList<PreConnectCallback>();
-    _handlers = new ArrayList<CallbackHandler>();
-    _properties = new HelixManagerProperties("cluster-manager-version.properties");
-    _version = _properties.getVersion();
-
-    _keyBuilder = new Builder(clusterName);
-    _messagingService = new DefaultMessagingService(this);
-
-    /**
-     * use system property if available
-     */
-    _flappingTimeWindowMs =
-        getSystemPropertyAsInt("helixmanager.flappingTimeWindow",
-            ZKHelixManager.FLAPPING_TIME_WINDIOW);
-
-    _maxDisconnectThreshold =
-        getSystemPropertyAsInt("helixmanager.maxDisconnectThreshold",
-            ZKHelixManager.MAX_DISCONNECT_THRESHOLD);
+    ZkHelixConnection conn = new ZkHelixConnection(zkAddress);
+    conn.connect();
 
-    _sessionTimeout =
-        getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT);
-
-    /**
-     * instance type specific init
-     */
     switch (instanceType) {
-    case PARTICIPANT:
-      _stateMachineEngine = new HelixStateMachineEngine(this);
+    case PARTICIPANT: {
+      _role = conn.createParticipant(clusterId, ParticipantId.from(instanceName));
       break;
-    case CONTROLLER:
-      _stateMachineEngine = null;
-      _controllerTimerTasks.add(new StatusDumpTask(this));
-
+    }
+    case CONTROLLER: {
+      _role = conn.createController(clusterId, ControllerId.from(instanceName));
       break;
-    case CONTROLLER_PARTICIPANT:
-      _stateMachineEngine = new HelixStateMachineEngine(this);
-      _controllerTimerTasks.add(new StatusDumpTask(this));
+    }
+    case CONTROLLER_PARTICIPANT: {
+      _role = conn.createAutoController(clusterId, ControllerId.from(instanceName));
       break;
-    case ADMINISTRATOR:
-    case SPECTATOR:
-      _stateMachineEngine = null;
+    }
+    case ADMINISTRATOR: {
+      _role = new ZkHelixRoleDefaultImpl(conn, clusterId, AdministratorId.from(instanceName));
       break;
-    default:
-      throw new IllegalArgumentException("unrecognized type: " + instanceType);
     }
-  }
-
-  private int getSystemPropertyAsInt(String propertyKey, int propertyDefaultValue) {
-    String valueString = System.getProperty(propertyKey, "" + propertyDefaultValue);
-
-    try {
-      int value = Integer.parseInt(valueString);
-      if (value > 0) {
-        return value;
-      }
-    } catch (NumberFormatException e) {
-      LOG.warn("Exception while parsing property: " + propertyKey + ", string: " + valueString
-          + ", using default value: " + propertyDefaultValue);
+    case SPECTATOR: {
+      _role = new ZkHelixRoleDefaultImpl(conn, clusterId, SpectatorId.from(instanceName));
+      break;
+    }
+    default:
+      throw new IllegalArgumentException("Unrecognized type: " + instanceType);
     }
-
-    return propertyDefaultValue;
   }
 
   @Override
-  public boolean removeListener(PropertyKey key, Object listener) {
-    LOG.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: "
-        + _clusterName + " by instance: " + _instanceName);
-
-    synchronized (this) {
-      List<CallbackHandler> toRemove = new ArrayList<CallbackHandler>();
-      for (CallbackHandler handler : _handlers) {
-        // compare property-key path and listener reference
-        if (handler.getPath().equals(key.getPath()) && handler.getListener().equals(listener)) {
-          toRemove.add(handler);
-        }
-      }
-
-      _handlers.removeAll(toRemove);
+  public void connect() throws Exception {
+    HelixConnection conn = (ZkHelixConnection) _role.getConnection();
 
-      // handler.reset() may modify the handlers list, so do it outside the iteration
-      for (CallbackHandler handler : toRemove) {
-        handler.reset();
-      }
+    if (!conn.isConnected()) {
+      conn.connect();
     }
 
-    return true;
+    HelixService service = (HelixService) _role;
+    service.start();
   }
 
-  void checkConnected() {
-    if (!isConnected()) {
-      throw new HelixException("HelixManager is not connected. Call HelixManager#connect()");
-    }
+  @Override
+  public boolean isConnected() {
+    return ((HelixService) _role).isStarted();
   }
 
-  void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType,
-      EventType[] eventType) {
-    checkConnected();
-
-    PropertyType type = propertyKey.getType();
-
-    synchronized (this) {
-      for (CallbackHandler handler : _handlers) {
-        // compare property-key path and listener reference
-        if (handler.getPath().equals(propertyKey.getPath())
-            && handler.getListener().equals(listener)) {
-          LOG.info("Listener: " + listener + " on path: " + propertyKey.getPath()
-              + " already exists. skip add");
-
-          return;
-        }
-      }
+  @Override
+  public void disconnect() {
+    HelixService service = (HelixService) _role;
+    HelixConnection conn = _role.getConnection();
 
-      CallbackHandler newHandler =
-          new CallbackHandler(this, _zkclient, propertyKey, listener, eventType, changeType);
+    service.stop();
 
-      _handlers.add(newHandler);
-      LOG.info("Added listener: " + listener + " for type: " + type + " to path: "
-          + newHandler.getPath());
+    if (conn.isConnected()) {
+      conn.disconnect();
     }
   }
 
   @Override
-  public void addIdealStateChangeListener(final IdealStateChangeListener listener) throws Exception {
-    addListener(listener, new Builder(_clusterName).idealStates(), ChangeType.IDEAL_STATE,
-        new EventType[] {
-            EventType.NodeDataChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+  public void addIdealStateChangeListener(IdealStateChangeListener listener) throws Exception {
+    _role.getConnection().addIdealStateChangeListener(_role, listener, _role.getClusterId());
   }
 
   @Override
   public void addLiveInstanceChangeListener(LiveInstanceChangeListener listener) throws Exception {
-    addListener(listener, new Builder(_clusterName).liveInstances(), ChangeType.LIVE_INSTANCE,
-        new EventType[] {
-            EventType.NodeDataChanged, EventType.NodeChildrenChanged, EventType.NodeDeleted,
-            EventType.NodeCreated
-        });
-  }
-
-  @Override
-  public void addConfigChangeListener(ConfigChangeListener listener) throws Exception {
-    addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG,
-        new EventType[] {
-          EventType.NodeChildrenChanged
-        });
+    _role.getConnection().addLiveInstanceChangeListener(_role, listener, _role.getClusterId());
   }
 
   @Override
   public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
       throws Exception {
-    addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG,
-        new EventType[] {
-          EventType.NodeChildrenChanged
-        });
+    _role.getConnection().addInstanceConfigChangeListener(_role, listener, _role.getClusterId());
   }
 
   @Override
   public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
       throws Exception {
-    Builder keyBuilder = new Builder(_clusterName);
-
-    PropertyKey propertyKey = null;
-    switch (scope) {
-    case CLUSTER:
-      propertyKey = keyBuilder.clusterConfigs();
-      break;
-    case PARTICIPANT:
-      propertyKey = keyBuilder.instanceConfigs();
-      break;
-    case RESOURCE:
-      propertyKey = keyBuilder.resourceConfigs();
-      break;
-    default:
-      break;
-    }
-
-    if (propertyKey != null) {
-      addListener(listener, propertyKey, ChangeType.CONFIG, new EventType[] {
-        EventType.NodeChildrenChanged
-      });
-    } else {
-      LOG.error("Can't add listener to config scope: " + scope);
-    }
-  }
-
-  // TODO: Decide if do we still need this since we are exposing
-  // ClusterMessagingService
-  @Override
-  public void addMessageListener(MessageListener listener, String instanceName) {
-    addListener(listener, new Builder(_clusterName).messages(instanceName), ChangeType.MESSAGE,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+    _role.getConnection().addConfigChangeListener(_role, listener, _role.getClusterId(), scope);
   }
 
   @Override
-  public void addControllerMessageListener(MessageListener listener) {
-    addListener(listener, new Builder(_clusterName).controllerMessages(),
-        ChangeType.MESSAGES_CONTROLLER, new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+  public void addMessageListener(MessageListener listener, String instanceName) throws Exception {
+    _role.getConnection().addMessageListener(_role, listener, _role.getClusterId(),
+        ParticipantId.from(instanceName));
   }
 
   @Override
   public void addCurrentStateChangeListener(CurrentStateChangeListener listener,
       String instanceName, String sessionId) throws Exception {
-    addListener(listener, new Builder(_clusterName).currentStates(instanceName, sessionId),
-        ChangeType.CURRENT_STATE, new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+    _role.getConnection().addCurrentStateChangeListener(_role, listener, _role.getClusterId(),
+        ParticipantId.from(instanceName), SessionId.from(sessionId));
   }
 
   @Override
   public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
-    addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+    _role.getConnection().addExternalViewChangeListener(_role, listener, _role.getClusterId());
   }
 
   @Override
   public void addControllerListener(ControllerChangeListener listener) {
-    addListener(listener, new Builder(_clusterName).controller(), ChangeType.CONTROLLER,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
+    _role.getConnection().addControllerListener(_role, listener, _role.getClusterId());
   }
 
   @Override
-  public HelixDataAccessor getHelixDataAccessor() {
-    checkConnected();
-    return _dataAccessor;
+  public void addControllerMessageListener(MessageListener listener) {
+    _role.getConnection().addControllerMessageListener(_role, listener, _role.getClusterId());
   }
 
   @Override
-  public ConfigAccessor getConfigAccessor() {
-    checkConnected();
-    return _configAccessor;
+  public boolean removeListener(PropertyKey key, Object listener) {
+    return _role.getConnection().removeListener(_role, listener, key);
   }
 
   @Override
-  public String getClusterName() {
-    return _clusterName;
+  public HelixDataAccessor getHelixDataAccessor() {
+    return _role.getAccessor();
   }
 
   @Override
-  public String getInstanceName() {
-    return _instanceName;
-  }
-
-  BaseDataAccessor<ZNRecord> createBaseDataAccessor() {
-    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkclient);
-
-    return baseDataAccessor;
-  }
-
-  void createClient() throws Exception {
-    PathBasedZkSerializer zkSerializer =
-        ChainedPathZkSerializer.builder(new ZNRecordStreamingSerializer()).build();
-
-    _zkclient =
-        new ZkClient(_zkAddress, _sessionTimeout, ZkClient.DEFAULT_CONNECTION_TIMEOUT, zkSerializer);
-
-    _baseDataAccessor = createBaseDataAccessor();
-
-    _dataAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor);
-    _configAccessor = new ConfigAccessor(_zkclient);
-
-    int retryCount = 0;
-
-    _zkclient.subscribeStateChanges(this);
-    while (retryCount < 3) {
-      try {
-        _zkclient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS);
-        handleStateChanged(KeeperState.SyncConnected);
-        handleNewSession();
-        break;
-      } catch (HelixException e) {
-        LOG.error("fail to createClient.", e);
-        throw e;
-      } catch (Exception e) {
-        retryCount++;
-
-        LOG.error("fail to createClient. retry " + retryCount, e);
-        if (retryCount == 3) {
-          throw e;
-        }
-      }
-    }
+  public ConfigAccessor getConfigAccessor() {
+    return _role.getConnection().getConfigAccessor();
   }
 
   @Override
-  public void connect() throws Exception {
-    LOG.info("ClusterManager.connect()");
-    if (isConnected()) {
-      LOG.warn("Cluster manager: " + _instanceName + " for cluster: " + _clusterName
-          + " already connected. skip connect");
-      return;
-    }
-
-    try {
-      createClient();
-      _messagingService.onConnected();
-    } catch (Exception e) {
-      LOG.error("fail to connect " + _instanceName, e);
-      disconnect();
-      throw e;
-    }
+  public String getClusterName() {
+    return _role.getClusterId().stringify();
   }
 
   @Override
-  public void disconnect() {
-    if (_zkclient == null) {
-      LOG.info("instanceName: " + _instanceName + " already disconnected");
-      return;
-    }
-
-    LOG.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName);
-
-    try {
-      /**
-       * stop all timer tasks
-       */
-      stopTimerTasks();
-
-      /**
-       * shutdown thread pool first to avoid reset() being invoked in the middle of state
-       * transition
-       */
-      _messagingService.getExecutor().shutdown();
-
-      // TODO reset user defined handlers only
-      resetHandlers();
-
-      if (_leaderElectionHandler != null) {
-        _leaderElectionHandler.reset();
-      }
-
-    } finally {
-      _zkclient.close();
-      _zkclient = null;
-      LOG.info("Cluster manager: " + _instanceName + " disconnected");
-    }
+  public String getInstanceName() {
+    return _role.getId().stringify();
   }
 
   @Override
   public String getSessionId() {
-    checkConnected();
-    return _sessionId;
-  }
-
-  @Override
-  public boolean isConnected() {
-    if (_zkclient == null) {
-      return false;
-    }
-    ZkConnection zkconnection = (ZkConnection) _zkclient.getConnection();
-    if (zkconnection != null) {
-      States state = zkconnection.getZookeeperState();
-      return state == States.CONNECTED;
-    }
-    return false;
+    return _role.getConnection().getSessionId().stringify();
   }
 
   @Override
   public long getLastNotificationTime() {
+    // TODO implement this
     return 0;
   }
 
   @Override
-  public void addPreConnectCallback(PreConnectCallback callback) {
-    LOG.info("Adding preconnect callback: " + callback);
-    _preConnectCallbacks.add(callback);
+  public HelixAdmin getClusterManagmentTool() {
+    return _role.getConnection().createClusterManagementTool();
   }
 
   @Override
-  public boolean isLeader() {
-    if (_instanceType != InstanceType.CONTROLLER
-        && _instanceType != InstanceType.CONTROLLER_PARTICIPANT) {
-      return false;
-    }
-
-    if (!isConnected()) {
-      return false;
-    }
-
-    try {
-      LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader());
-      if (leader != null) {
-        String leaderName = leader.getInstanceName();
-        String sessionId = leader.getSessionId();
-        if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null
-            && sessionId.equals(_sessionId)) {
-          return true;
-        }
-      }
-    } catch (Exception e) {
-      // log
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized ZkHelixPropertyStore<ZNRecord> getHelixPropertyStore() {
-    checkConnected();
-
-    if (_helixPropertyStore == null) {
-      String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName);
-      String fallbackPath = String.format("/%s/%s", _clusterName, "HELIX_PROPERTYSTORE");
-      _helixPropertyStore =
-          new AutoFallbackPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient),
-              path, fallbackPath);
-    }
-
-    return _helixPropertyStore;
-  }
-
-  @Override
-  public synchronized HelixAdmin getClusterManagmentTool() {
-    checkConnected();
-    if (_zkclient != null) {
-      return new ZKHelixAdmin(_zkclient);
-    }
-
-    LOG.error("Couldn't get ZKClusterManagementTool because zkclient is null");
-    return null;
+  public ZkHelixPropertyStore<ZNRecord> getHelixPropertyStore() {
+    return (ZkHelixPropertyStore<ZNRecord>) _role.getConnection().createPropertyStore(
+        _role.getClusterId());
   }
 
   @Override
   public ClusterMessagingService getMessagingService() {
-    // The caller can register message handler factories on messaging service before the
-    // helix manager is connected. Thus we do not do connected check here.
-    return _messagingService;
+    return _role.getMessagingService();
   }
 
   @Override
   public InstanceType getInstanceType() {
-    return _instanceType;
+    return _role.getType();
   }
 
   @Override
   public String getVersion() {
-    return _version;
+    return _role.getConnection().getHelixVersion();
   }
 
   @Override
   public HelixManagerProperties getProperties() {
-    return _properties;
+    return _role.getConnection().getHelixProperties();
   }
 
   @Override
   public StateMachineEngine getStateMachineEngine() {
-    return _stateMachineEngine;
-  }
-
-  // TODO: rename this and not expose this function as part of interface
-  @Override
-  public void startTimerTasks() {
-    for (HelixTimerTask task : _timerTasks) {
-      task.start();
+    StateMachineEngine engine = null;
+    switch (_role.getType()) {
+    case PARTICIPANT: {
+      HelixParticipant participant = (HelixParticipant) _role;
+      engine = participant.getStateMachineEngine();
+      break;
+    }
+    case CONTROLLER_PARTICIPANT: {
+      HelixMultiClusterController autoController = (HelixMultiClusterController) _role;
+      engine = autoController.getStateMachineEngine();
+      break;
+    }
+    default:
+      LOG.info("helix manager type: " + _role.getType() + " does NOT have state-machine-engine");
     }
+
+    return engine;
   }
 
   @Override
-  public void stopTimerTasks() {
-    for (HelixTimerTask task : _timerTasks) {
-      task.stop();
+  public boolean isLeader() {
+    boolean isLeader = false;
+    switch (_role.getType()) {
+    case CONTROLLER: {
+      HelixController controller = (HelixController) _role;
+      isLeader = controller.isLeader();
+      break;
     }
+    case CONTROLLER_PARTICIPANT: {
+      HelixMultiClusterController autoController = (HelixMultiClusterController) _role;
+      isLeader = autoController.isLeader();
+      break;
+    }
+    default:
+      LOG.info("helix manager type: " + _role.getType() + " does NOT support leadership");
+    }
+    return isLeader;
   }
 
   @Override
-  public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
-    _liveInstanceInfoProvider = liveInstanceInfoProvider;
-  }
-
-  /**
-   * wait until we get a non-zero session-id. note that we might lose zkconnection
-   * right after we read session-id. but it's ok to get stale session-id and we will have
-   * another handle-new-session callback to correct this.
-   */
-  void waitUntilConnected() {
-    boolean isConnected;
-    do {
-      isConnected =
-          _zkclient.waitUntilConnected(ZkClient.DEFAULT_CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS);
-      if (!isConnected) {
-        LOG.error("fail to connect zkserver: " + _zkAddress + " in "
-            + ZkClient.DEFAULT_CONNECTION_TIMEOUT + "ms. expiredSessionId: " + _sessionId
-            + ", clusterName: " + _clusterName);
-        continue;
-      }
-
-      ZkConnection zkConnection = ((ZkConnection) _zkclient.getConnection());
-      _sessionId = Long.toHexString(zkConnection.getZookeeper().getSessionId());
-
-      /**
-       * at the time we read session-id, zkconnection might be lost again
-       * wait until we get a non-zero session-id
-       */
-    } while ("0".equals(_sessionId));
-
-    LOG.info("Handling new session, session id: " + _sessionId + ", instance: " + _instanceName
-        + ", instanceTye: " + _instanceType + ", cluster: " + _clusterName + ", zkconnection: "
-        + ((ZkConnection) _zkclient.getConnection()).getZookeeper());
-  }
-
-  void initHandlers(List<CallbackHandler> handlers) {
-    synchronized (this) {
-      if (handlers != null) {
-        for (CallbackHandler handler : handlers) {
-          handler.init();
-          LOG.info("init handler: " + handler.getPath() + ", " + handler.getListener());
-        }
-      }
+  public void startTimerTasks() {
+    switch (getInstanceType()) {
+    case CONTROLLER: {
+      ((ZkHelixController) _role).startTimerTasks();
+      break;
     }
-  }
-
-  void resetHandlers() {
-    synchronized (this) {
-      if (_handlers != null) {
-        // get a copy of the list and iterate over the copy list
-        // in case handler.reset() modify the original handler list
-        List<CallbackHandler> tmpHandlers = new ArrayList<CallbackHandler>();
-        tmpHandlers.addAll(_handlers);
-
-        for (CallbackHandler handler : tmpHandlers) {
-          handler.reset();
-          LOG.info("reset handler: " + handler.getPath() + ", " + handler.getListener());
-        }
-      }
+    default:
+      throw new IllegalStateException("Cann't start timer tasks for type: " + getInstanceType());
     }
   }
 
-  /**
-   * If zk state has changed into Disconnected for _maxDisconnectThreshold times during previous
-   * _timeWindowLengthMs Ms
-   * time window, we think that there are something wrong going on and disconnect the zkHelixManager
-   * from zk.
-   */
-  boolean isFlapping() {
-    if (_disconnectTimeHistory.size() == 0) {
-      return false;
+  @Override
+  public void stopTimerTasks() {
+    switch (getInstanceType()) {
+    case CONTROLLER: {
+      ((ZkHelixController) _role).stopTimerTasks();
+      break;
     }
-    long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1);
-
-    // Remove disconnect history timestamp that are older than _flappingTimeWindowMs ago
-    while ((_disconnectTimeHistory.get(0) + _flappingTimeWindowMs) < mostRecentTimestamp) {
-      _disconnectTimeHistory.remove(0);
+    default:
+      throw new IllegalStateException("Cann't stop timer tasks for type: " + getInstanceType());
     }
-    return _disconnectTimeHistory.size() > _maxDisconnectThreshold;
   }
 
   @Override
-  public void handleStateChanged(KeeperState state) throws Exception {
-    switch (state) {
-    case SyncConnected:
-      ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection();
-      LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper());
-      break;
-    case Disconnected:
-      LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId + ", instance: "
-          + _instanceName + ", type: " + _instanceType);
-
-      /**
-       * Track the time stamp that the disconnected happens, then check history and see if
-       * we should disconnect the helix-manager
-       */
-      _disconnectTimeHistory.add(System.currentTimeMillis());
-      if (isFlapping()) {
-        LOG.error("instanceName: " + _instanceName + " is flapping. diconnect it. "
-            + " maxDisconnectThreshold: " + _maxDisconnectThreshold + " disconnects in "
-            + _flappingTimeWindowMs + "ms.");
-        disconnect();
-      }
+  public void addPreConnectCallback(PreConnectCallback callback) {
+    switch (_role.getType()) {
+    case PARTICIPANT: {
+      HelixParticipant participant = (HelixParticipant) _role;
+      participant.addPreConnectCallback(callback);
       break;
-    case Expired:
-      LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId + ", instance: "
-          + _instanceName + ", type: " + _instanceType);
+    }
+    case CONTROLLER_PARTICIPANT: {
+      HelixMultiClusterController autoController = (HelixMultiClusterController) _role;
+      autoController.addPreConnectCallback(callback);
       break;
+    }
     default:
-      break;
+      LOG.info("helix manager type: " + _role.getType()
+          + " does NOT support add pre-connect callback");
     }
   }
 
   @Override
-  public void handleNewSession() throws Exception {
-    waitUntilConnected();
-
-    /**
-     * stop all timer tasks, reset all handlers, make sure cleanup completed for previous session
-     * disconnect if fail to cleanup
-     */
-    stopTimerTasks();
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.reset();
-    }
-    resetHandlers();
-
-    /**
-     * clean up write-through cache
-     */
-    _baseDataAccessor.reset();
-
-    /**
-     * from here on, we are dealing with new session
-     */
-    if (!ZKUtil.isClusterSetup(_clusterName, _zkclient)) {
-      throw new HelixException("Cluster structure is not set up for cluster: " + _clusterName);
-    }
-
-    switch (_instanceType) {
-    case PARTICIPANT:
-      handleNewSessionAsParticipant();
-      break;
-    case CONTROLLER:
-      handleNewSessionAsController();
-      break;
-    case CONTROLLER_PARTICIPANT:
-      handleNewSessionAsParticipant();
-      handleNewSessionAsController();
-      break;
-    case ADMINISTRATOR:
-    case SPECTATOR:
-    default:
+  public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
+    switch (_role.getType()) {
+    case PARTICIPANT: {
+      HelixParticipant participant = (HelixParticipant) _role;
+      participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
       break;
     }
-
-    startTimerTasks();
-
-    /**
-     * init handlers
-     * ok to init message handler and data-accessor twice
-     * the second init will be skipped (see CallbackHandler)
-     */
-    initHandlers(_handlers);
-  }
-
-  void handleNewSessionAsParticipant() throws Exception {
-    /**
-     * auto-join
-     */
-    ParticipantManagerHelper participantHelper =
-        new ParticipantManagerHelper(this, _zkclient, _sessionTimeout, _liveInstanceInfoProvider);
-    participantHelper.joinCluster();
-
-    /**
-     * Invoke PreConnectCallbacks
-     */
-    for (PreConnectCallback callback : _preConnectCallbacks) {
-      callback.onPreConnect();
+    case CONTROLLER_PARTICIPANT: {
+      HelixMultiClusterController autoController = (HelixMultiClusterController) _role;
+      autoController.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
+      break;
     }
-
-    participantHelper.createLiveInstance();
-    participantHelper.carryOverPreviousCurrentState();
-
-    /**
-     * setup message listener
-     */
-    participantHelper.setupMsgHandler();
-  }
-
-  void handleNewSessionAsController() {
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.init();
-    } else {
-      _leaderElectionHandler =
-          new CallbackHandler(this, _zkclient, _keyBuilder.controller(),
-              new DistributedLeaderElection(this, _controller, _controllerTimerTasks),
-              new EventType[] {
-                  EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-              }, ChangeType.CONTROLLER);
+    default:
+      LOG.info("helix manager type: " + _role.getType()
+          + " does NOT support set additional live instance information");
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
index 5961fe3..374a30b 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackHandler.java
@@ -37,7 +37,6 @@ import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -121,7 +120,7 @@ public class ZkCallbackHandler implements IZkChildListener, IZkDataListener
     }
 
     _role = role;
-    _manager = new HelixConnectionAdaptor(role);
+    _manager = new ZKHelixManager(role);
     _instanceName = role.getId().stringify();
     _connection = role.getConnection();
     _accessor = _connection.createDataAccessor(role.getClusterId());
@@ -171,15 +170,10 @@ public class ZkCallbackHandler implements IZkChildListener, IZkDataListener
 
       } else if (_changeType == ChangeType.INSTANCE_CONFIG) {
         subscribeForChanges(changeContext, _path, true, true);
-        if (_listener instanceof ConfigChangeListener) {
-          ConfigChangeListener configChangeListener = (ConfigChangeListener) _listener;
-          List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
-          configChangeListener.onConfigChange(configs, changeContext);
-        } else if (_listener instanceof InstanceConfigChangeListener) {
-          InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener;
-          List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
-          listener.onInstanceConfigChange(configs, changeContext);
-        }
+        InstanceConfigChangeListener listener = (InstanceConfigChangeListener) _listener;
+        List<InstanceConfig> configs = _accessor.getChildValues(_propertyKey);
+        listener.onInstanceConfigChange(configs, changeContext);
+
       } else if (_changeType == CONFIG) {
         subscribeForChanges(changeContext, _path, true, true);
         ScopedConfigChangeListener listener = (ScopedConfigChangeListener) _listener;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java
deleted file mode 100644
index 1d4b225..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixAutoController.java
+++ /dev/null
@@ -1,133 +0,0 @@
-package org.apache.helix.manager.zk;
-
-/*
- * 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 org.apache.helix.ClusterMessagingService;
-import org.apache.helix.HelixAutoController;
-import org.apache.helix.HelixConnection;
-import org.apache.helix.InstanceType;
-import org.apache.helix.LiveInstanceInfoProvider;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.api.id.ClusterId;
-import org.apache.helix.api.id.ControllerId;
-import org.apache.helix.api.id.Id;
-import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-
-public class ZkHelixAutoController implements HelixAutoController {
-  private static Logger LOG = Logger.getLogger(ZkHelixAutoController.class);
-
-  final ZkHelixConnection _connection;
-  final ClusterId _clusterId;
-  final ControllerId _controllerId;
-  final ZkHelixParticipant _participant;
-  final ZkHelixController _controller;
-
-  public ZkHelixAutoController(ZkHelixConnection connection, ClusterId clusterId,
-      ControllerId controllerId) {
-    _connection = connection;
-    _clusterId = clusterId;
-    _controllerId = controllerId;
-
-    _participant =
-        new ZkHelixParticipant(connection, clusterId, ParticipantId.from(controllerId.stringify()));
-    _controller = new ZkHelixController(connection, clusterId, controllerId);
-  }
-
-  @Override
-  public HelixConnection getConnection() {
-    return _connection;
-  }
-
-  @Override
-  public ClusterId getClusterId() {
-    return _clusterId;
-  }
-
-  @Override
-  public Id getId() {
-    return getControllerId();
-  }
-
-  @Override
-  public InstanceType getType() {
-    return InstanceType.CONTROLLER_PARTICIPANT;
-  }
-
-  @Override
-  public ClusterMessagingService getMessagingService() {
-    return _participant.getMessagingService();
-  }
-
-  @Override
-  public void start() {
-    _connection.addConnectionStateListener(this);
-    onConnected();
-  }
-
-  @Override
-  public void stop() {
-    _connection.removeConnectionStateListener(this);
-    onDisconnecting();
-  }
-
-  @Override
-  public void onConnected() {
-    _controller.reset();
-    _participant.reset();
-
-    _participant.init();
-    _controller.init();
-  }
-
-  @Override
-  public void onDisconnecting() {
-    LOG.info("disconnecting " + _controllerId + "(" + getType() + ") from " + _clusterId);
-    _controller.onDisconnecting();
-    _participant.onDisconnecting();
-  }
-
-  @Override
-  public ControllerId getControllerId() {
-    return _controllerId;
-  }
-
-  @Override
-  public StateMachineEngine getStateMachineEngine() {
-    return _participant.getStateMachineEngine();
-  }
-
-  @Override
-  public void addPreConnectCallback(PreConnectCallback callback) {
-    _participant.addPreConnectCallback(callback);
-  }
-
-  @Override
-  public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
-    _participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
-  }
-
-  @Override
-  public boolean isLeader() {
-    return _controller.isLeader();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
index bec6f5c..0a9dc94 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
@@ -27,19 +27,17 @@ import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantLock;
 
 import org.I0Itec.zkclient.IZkStateListener;
 import org.I0Itec.zkclient.ZkConnection;
 import org.apache.helix.BaseDataAccessor;
 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;
 import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixAutoController;
+import org.apache.helix.HelixMultiClusterController;
 import org.apache.helix.HelixConnection;
 import org.apache.helix.HelixConnectionStateListener;
 import org.apache.helix.HelixConstants.ChangeType;
@@ -73,37 +71,35 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
 public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   private static Logger LOG = Logger.getLogger(ZkHelixConnection.class);
 
-  final String _zkAddr;
-  final int _sessionTimeout;
-  SessionId _sessionId;
-  ZkClient _zkclient;
-  BaseDataAccessor<ZNRecord> _baseAccessor;
-  ConfigAccessor _configAccessor;
-  final Set<HelixConnectionStateListener> _connectionListener =
-      new CopyOnWriteArraySet<HelixConnectionStateListener>();
+  private final String _zkAddr;
+  private final int _sessionTimeout;
+  private SessionId _sessionId;
+  ZkClient _zkclient; // keep it package level for test purpose
+  private BaseDataAccessor<ZNRecord> _baseAccessor;
+  private ConfigAccessor _configAccessor;
+  private final Set<HelixConnectionStateListener> _connectionListener;
 
-  final Map<HelixRole, List<ZkCallbackHandler>> _handlers;
-  final HelixManagerProperties _properties;
+  final Map<HelixRole, List<ZkCallbackHandler>> _handlers; // keep it package level for test purpose
+  private final HelixManagerProperties _properties;
 
   /**
    * Keep track of timestamps that zk State has become Disconnected
    * If in a _timeWindowLengthMs window zk State has become Disconnected
    * for more than_maxDisconnectThreshold times disconnect the zkHelixManager
    */
-  final List<Long> _disconnectTimeHistory = new ArrayList<Long>();
-  final int _flappingTimeWindowMs;
-  final int _maxDisconnectThreshold;
-
-  final ReentrantLock _lock = new ReentrantLock();
+  private final List<Long> _disconnectTimeHistory = new ArrayList<Long>();
+  private final int _flappingTimeWindowMs;
+  private final int _maxDisconnectThreshold;
 
   /**
    * helix version#
    */
-  final String _version;
+  private final String _version;
 
   public ZkHelixConnection(String zkAddr) {
     _zkAddr = zkAddr;
     _handlers = new HashMap<HelixRole, List<ZkCallbackHandler>>();
+    _connectionListener = new CopyOnWriteArraySet<HelixConnectionStateListener>();
 
     /**
      * use system property if available
@@ -141,14 +137,16 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public void connect() {
+  public synchronized void connect() {
+    if (isConnected()) {
+      return;
+    }
+
     boolean isStarted = false;
     try {
-      _lock.lock();
       _zkclient =
           new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
               ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
-      // waitUntilConnected();
 
       _baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkclient);
       _configAccessor = new ConfigAccessor(_zkclient);
@@ -160,7 +158,6 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
     } catch (Exception e) {
       LOG.error("Exception connect", e);
     } finally {
-      _lock.unlock();
       if (!isStarted) {
         disconnect();
       }
@@ -168,10 +165,9 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public void disconnect() {
+  public synchronized void disconnect() {
     try {
-      _lock.lock();
-      if (_zkclient == null) {
+      if (!isConnected()) {
         return;
       }
 
@@ -195,19 +191,12 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
       }
     } catch (Exception e) {
       LOG.error("Exception disconnect", e);
-    } finally {
-      _lock.unlock();
     }
   }
 
   @Override
-  public boolean isConnected() {
-    try {
-      _lock.lock();
-      return _zkclient != null;
-    } finally {
-      _lock.unlock();
-    }
+  public synchronized boolean isConnected() {
+    return _zkclient != null;
   }
 
   @Override
@@ -221,8 +210,8 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public HelixAutoController createAutoController(ClusterId clusterId, ControllerId controllerId) {
-    return new ZkHelixAutoController(this, clusterId, controllerId);
+  public HelixMultiClusterController createAutoController(ClusterId clusterId, ControllerId controllerId) {
+    return new ZkHelixMultiClusterController(this, clusterId, controllerId);
   }
 
   @Override
@@ -243,8 +232,15 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
         key.getPath(), fallbackPath);
   }
 
+  private void checkConnected() {
+    if (!isConnected()) {
+      throw new IllegalStateException("Connection not connected. Please call #connect()");
+    }
+  }
+
   @Override
   public HelixDataAccessor createDataAccessor(ClusterId clusterId) {
+    checkConnected();
     return new ZKHelixDataAccessor(clusterId.stringify(), _baseAccessor);
   }
 
@@ -307,16 +303,6 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public void addConfigChangeListener(HelixRole role, ConfigChangeListener listener,
-      ClusterId clusterId) {
-
-    addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).instanceConfigs(),
-        ChangeType.INSTANCE_CONFIG, new EventType[] {
-          EventType.NodeChildrenChanged
-        });
-  }
-
-  @Override
   public void addInstanceConfigChangeListener(HelixRole role,
       InstanceConfigChangeListener listener, ClusterId clusterId) {
     addListener(role, listener, new PropertyKey.Builder(clusterId.stringify()).instanceConfigs(),
@@ -422,57 +408,45 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
   }
 
   @Override
-  public void handleStateChanged(KeeperState state) throws Exception {
-    try {
-      _lock.lock();
-
-      switch (state) {
-      case SyncConnected:
-        ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection();
-        LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper());
-        break;
-      case Disconnected:
-        LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId);
-
-        /**
-         * Track the time stamp that the disconnected happens, then check history and see if
-         * we should disconnect the helix-manager
-         */
-        _disconnectTimeHistory.add(System.currentTimeMillis());
-        if (isFlapping()) {
-          LOG.error("helix-connection: " + this + ", sessionId: " + _sessionId
-              + " is flapping. diconnect it. " + " maxDisconnectThreshold: "
-              + _maxDisconnectThreshold + " disconnects in " + _flappingTimeWindowMs + "ms");
-          disconnect();
-        }
-        break;
-      case Expired:
-        LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId);
-        break;
-      default:
-        break;
+  public synchronized void handleStateChanged(KeeperState state) throws Exception {
+    switch (state) {
+    case SyncConnected:
+      ZkConnection zkConnection = (ZkConnection) _zkclient.getConnection();
+      LOG.info("KeeperState: " + state + ", zookeeper:" + zkConnection.getZookeeper());
+      break;
+    case Disconnected:
+      LOG.info("KeeperState:" + state + ", disconnectedSessionId: " + _sessionId);
+
+      /**
+       * Track the time stamp that the disconnected happens, then check history and see if
+       * we should disconnect the helix-manager
+       */
+      _disconnectTimeHistory.add(System.currentTimeMillis());
+      if (isFlapping()) {
+        LOG.error("helix-connection: " + this + ", sessionId: " + _sessionId
+            + " is flapping. diconnect it. " + " maxDisconnectThreshold: "
+            + _maxDisconnectThreshold + " disconnects in " + _flappingTimeWindowMs + "ms");
+        disconnect();
       }
-    } finally {
-      _lock.unlock();
+      break;
+    case Expired:
+      LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId);
+      break;
+    default:
+      break;
     }
   }
 
   @Override
-  public void handleNewSession() throws Exception {
+  public synchronized void handleNewSession() throws Exception {
     waitUntilConnected();
 
-    try {
-      _lock.lock();
-
-      for (final HelixConnectionStateListener listener : _connectionListener) {
-        try {
-          listener.onConnected();
-        } catch (Exception e) {
-          LOG.error("Exception invoking connect on listener: " + listener, e);
-        }
+    for (final HelixConnectionStateListener listener : _connectionListener) {
+      try {
+        listener.onConnected();
+      } catch (Exception e) {
+        LOG.error("Exception invoking connect on listener: " + listener, e);
       }
-    } finally {
-      _lock.unlock();
     }
   }
 
@@ -514,7 +488,7 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
        * at the time we read session-id, zkconnection might be lost again
        * wait until we get a non-zero session-id
        */
-    } while ("0".equals(_sessionId));
+    } while (!isConnected || "0".equals(_sessionId));
 
     LOG.info("Handling new session, session id: " + _sessionId + ", zkconnection: "
         + ((ZkConnection) _zkclient.getConnection()).getZookeeper());
@@ -527,8 +501,8 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
 
   @Override
   public ClusterMessagingService createMessagingService(HelixRole role) {
-    HelixManager manager = new HelixConnectionAdaptor(role);
-    return new DefaultMessagingService(manager);
+    HelixManager manager = new ZKHelixManager(role);
+     return new DefaultMessagingService(manager);
   }
 
   void addListener(HelixRole role, Object listener, PropertyKey propertyKey, ChangeType changeType,

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
index f9529b7..0698945 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixController.java
@@ -56,6 +56,7 @@ public class ZkHelixController implements HelixController {
   final HelixDataAccessor _accessor;
   final HelixManager _manager;
   final ZkHelixLeaderElection _leaderElection;
+  boolean _isStarted;
 
   public ZkHelixController(ZkHelixConnection connection, ClusterId clusterId,
       ControllerId controllerId) {
@@ -69,7 +70,7 @@ public class ZkHelixController implements HelixController {
     _messagingService = (DefaultMessagingService) connection.createMessagingService(this);
     _timerTasks = new ArrayList<HelixTimerTask>();
 
-    _manager = new HelixConnectionAdaptor(this);
+    _manager = new ZKHelixManager(this);
     _leaderElection = new ZkHelixLeaderElection(this, _pipeline);
 
     _timerTasks.add(new StatusDumpTask(clusterId, _manager.getHelixDataAccessor()));
@@ -104,6 +105,11 @@ public class ZkHelixController implements HelixController {
     onDisconnecting();
   }
 
+  @Override
+  public boolean isStarted() {
+    return _isStarted;
+  }
+
   void reset() {
     /**
      * reset all handlers, make sure cleanup completed for previous session
@@ -140,6 +146,7 @@ public class ZkHelixController implements HelixController {
   public void onConnected() {
     reset();
     init();
+    _isStarted = true;
   }
 
   @Override
@@ -147,6 +154,8 @@ public class ZkHelixController implements HelixController {
     LOG.info("disconnecting " + _controllerId + "(" + getType() + ") from " + _clusterId);
 
     reset();
+
+    _isStarted = false;
   }
 
   @Override
@@ -216,7 +225,7 @@ public class ZkHelixController implements HelixController {
       /**
        * setup generic-controller
        */
-      _connection.addConfigChangeListener(this, pipeline, _clusterId);
+      _connection.addInstanceConfigChangeListener(this, pipeline, _clusterId);
       _connection.addLiveInstanceChangeListener(this, pipeline, _clusterId);
       _connection.addIdealStateChangeListener(this, pipeline, _clusterId);
       _connection.addControllerListener(this, pipeline, _clusterId);
@@ -228,7 +237,7 @@ public class ZkHelixController implements HelixController {
   }
 
   void removeListenersFromController(GenericHelixController pipeline) {
-    PropertyKey.Builder keyBuilder = new PropertyKey.Builder(_manager.getClusterName());
+    PropertyKey.Builder keyBuilder = new PropertyKey.Builder(getClusterId().stringify());
     /**
      * reset generic-controller
      */
@@ -248,4 +257,9 @@ public class ZkHelixController implements HelixController {
     return _manager;
   }
 
+  @Override
+  public HelixDataAccessor getAccessor() {
+    return _accessor;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixMultiClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixMultiClusterController.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixMultiClusterController.java
new file mode 100644
index 0000000..05ad223
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixMultiClusterController.java
@@ -0,0 +1,145 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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 org.apache.helix.ClusterMessagingService;
+import org.apache.helix.HelixMultiClusterController;
+import org.apache.helix.HelixConnection;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.InstanceType;
+import org.apache.helix.LiveInstanceInfoProvider;
+import org.apache.helix.PreConnectCallback;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.ControllerId;
+import org.apache.helix.api.id.Id;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.participant.StateMachineEngine;
+import org.apache.log4j.Logger;
+
+public class ZkHelixMultiClusterController implements HelixMultiClusterController {
+  private static Logger LOG = Logger.getLogger(ZkHelixMultiClusterController.class);
+
+  final ZkHelixConnection _connection;
+  final ClusterId _clusterId;
+  final ControllerId _controllerId;
+  final ZkHelixParticipant _participant;
+  final ZkHelixController _controller;
+
+  public ZkHelixMultiClusterController(ZkHelixConnection connection, ClusterId clusterId,
+      ControllerId controllerId) {
+    _connection = connection;
+    _clusterId = clusterId;
+    _controllerId = controllerId;
+
+    _participant =
+        new ZkHelixParticipant(connection, clusterId, ParticipantId.from(controllerId.stringify()));
+    _controller = new ZkHelixController(connection, clusterId, controllerId);
+  }
+
+  @Override
+  public HelixConnection getConnection() {
+    return _connection;
+  }
+
+  @Override
+  public ClusterId getClusterId() {
+    return _clusterId;
+  }
+
+  @Override
+  public Id getId() {
+    return getControllerId();
+  }
+
+  @Override
+  public InstanceType getType() {
+    return InstanceType.CONTROLLER_PARTICIPANT;
+  }
+
+  @Override
+  public ClusterMessagingService getMessagingService() {
+    return _participant.getMessagingService();
+  }
+
+  @Override
+  public void start() {
+    _connection.addConnectionStateListener(this);
+    onConnected();
+  }
+
+  @Override
+  public void stop() {
+    _connection.removeConnectionStateListener(this);
+    onDisconnecting();
+  }
+
+  @Override
+  public boolean isStarted() {
+    return _participant.isStarted() && _controller.isStarted();
+  }
+
+  @Override
+  public void onConnected() {
+    _controller.reset();
+    _participant.reset();
+
+    _participant.init();
+    _controller.init();
+
+  }
+
+  @Override
+  public void onDisconnecting() {
+    LOG.info("disconnecting " + _controllerId + "(" + getType() + ") from " + _clusterId);
+    _controller.onDisconnecting();
+    _participant.onDisconnecting();
+  }
+
+  @Override
+  public ControllerId getControllerId() {
+    return _controllerId;
+  }
+
+  @Override
+  public StateMachineEngine getStateMachineEngine() {
+    return _participant.getStateMachineEngine();
+  }
+
+  @Override
+  public void addPreConnectCallback(PreConnectCallback callback) {
+    _participant.addPreConnectCallback(callback);
+  }
+
+  @Override
+  public void setLiveInstanceInfoProvider(LiveInstanceInfoProvider liveInstanceInfoProvider) {
+    _participant.setLiveInstanceInfoProvider(liveInstanceInfoProvider);
+  }
+
+  @Override
+  public boolean isLeader() {
+    return _controller.isLeader();
+  }
+
+  @Override
+  public HelixDataAccessor getAccessor() {
+    return _participant.getAccessor();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
index d3ee8d1..af50eb7 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixParticipant.java
@@ -72,6 +72,7 @@ public class ZkHelixParticipant implements HelixParticipant {
   final DefaultMessagingService _messagingService;
   final List<PreConnectCallback> _preConnectCallbacks;
   final List<HelixTimerTask> _timerTasks;
+  boolean _isStarted;
 
   /**
    * state-transition message handler factory for helix-participant
@@ -93,7 +94,7 @@ public class ZkHelixParticipant implements HelixParticipant {
     _participantId = participantId;
 
     _messagingService = (DefaultMessagingService) connection.createMessagingService(this);
-    HelixManager manager = new HelixConnectionAdaptor(this);
+    HelixManager manager = new ZKHelixManager(this);
     _stateMachineEngine = new HelixStateMachineEngine(manager);
     _preConnectCallbacks = new ArrayList<PreConnectCallback>();
     _timerTasks = new ArrayList<HelixTimerTask>();
@@ -397,6 +398,7 @@ public class ZkHelixParticipant implements HelixParticipant {
   public void onConnected() {
     reset();
     init();
+    _isStarted = true;
   }
 
   @Override
@@ -415,12 +417,15 @@ public class ZkHelixParticipant implements HelixParticipant {
      * remove live instance ephemeral znode
      */
     _accessor.removeProperty(_keyBuilder.liveInstance(_participantId.stringify()));
+    _isStarted = false;
   }
 
   @Override
   public void start() {
     _connection.addConnectionStateListener(this);
-    onConnected();
+    if (_connection.isConnected()) {
+      onConnected();
+    }
   }
 
   @Override
@@ -430,6 +435,11 @@ public class ZkHelixParticipant implements HelixParticipant {
   }
 
   @Override
+  public boolean isStarted() {
+    return _isStarted;
+  }
+
+  @Override
   public ClusterMessagingService getMessagingService() {
     return _messagingService;
   }
@@ -460,6 +470,7 @@ public class ZkHelixParticipant implements HelixParticipant {
     _liveInstanceInfoProvider = liveInstanceInfoProvider;
   }
 
+  @Override
   public HelixDataAccessor getAccessor() {
     return _accessor;
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixRoleDefaultImpl.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixRoleDefaultImpl.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixRoleDefaultImpl.java
new file mode 100644
index 0000000..9866c7b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixRoleDefaultImpl.java
@@ -0,0 +1,126 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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 org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ClusterMessagingService;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixConnection;
+import org.apache.helix.HelixConnectionStateListener;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixRole;
+import org.apache.helix.HelixService;
+import org.apache.helix.InstanceType;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.api.accessor.ClusterAccessor;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.Id;
+
+public class ZkHelixRoleDefaultImpl implements HelixRole, HelixService, HelixConnectionStateListener {
+  final ZkHelixConnection _connection;
+  final HelixDataAccessor _accessor;
+  final BaseDataAccessor<ZNRecord> _baseAccessor;
+  final PropertyKey.Builder _keyBuilder;
+  final ClusterAccessor _clusterAccessor;
+  final ConfigAccessor _configAccessor;
+  final ClusterId _clusterId;
+  final Id _instanceId;
+  final ClusterMessagingService _messagingService;
+  boolean _isStarted;
+
+  public ZkHelixRoleDefaultImpl(ZkHelixConnection connection, ClusterId clusterId,
+      Id instanceId) {
+    _connection = connection;
+    _accessor = connection.createDataAccessor(clusterId);
+    _baseAccessor = _accessor.getBaseDataAccessor();
+    _keyBuilder = _accessor.keyBuilder();
+    _clusterAccessor = connection.createClusterAccessor(clusterId);
+    _configAccessor = connection.getConfigAccessor();
+
+    _clusterId = clusterId;
+    _instanceId = instanceId;
+    _messagingService = connection.createMessagingService(this);
+
+  }
+
+  @Override
+  public HelixConnection getConnection() {
+    return _connection;
+  }
+
+  @Override
+  public ClusterId getClusterId() {
+    return _clusterId;
+  }
+
+  @Override
+  public Id getId() {
+    return _instanceId;
+  }
+
+  @Override
+  public InstanceType getType() {
+    return InstanceType.SPECTATOR;
+  }
+
+  @Override
+  public ClusterMessagingService getMessagingService() {
+    return _messagingService;
+  }
+
+  @Override
+  public HelixDataAccessor getAccessor() {
+    return _accessor;
+  }
+
+  @Override
+  public void start() {
+    _connection.addConnectionStateListener(this);
+    if (_connection.isConnected()) {
+      onConnected();
+    }
+  }
+
+  @Override
+  public void stop() {
+    _connection.removeConnectionStateListener(this);
+    onDisconnecting();
+  }
+
+  @Override
+  public boolean isStarted() {
+    return _isStarted;
+  }
+
+  @Override
+  public void onConnected() {
+    _connection.resetHandlers(this);
+    _connection.initHandlers(this);
+    _isStarted = true;
+  }
+
+  @Override
+  public void onDisconnecting() {
+    _connection.resetHandlers(this);
+    _isStarted = false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
index e799e38..38aaf98 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
@@ -40,11 +40,12 @@ import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.handling.AsyncCallbackService;
 import org.apache.helix.messaging.handling.HelixTaskExecutor;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
-import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.log4j.Logger;
 
 public class DefaultMessagingService implements ClusterMessagingService {
@@ -223,7 +224,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
 
     ConfigAccessor configAccessor = _manager.getConfigAccessor();
     if (configAccessor != null) {
-      ConfigScope scope = null;
+      HelixConfigScope scope = null;
 
       // Read the participant config and cluster config for the per-message type thread pool size.
       // participant config will override the cluster config.
@@ -231,13 +232,16 @@ public class DefaultMessagingService implements ClusterMessagingService {
       if (_manager.getInstanceType() == InstanceType.PARTICIPANT
           || _manager.getInstanceType() == InstanceType.CONTROLLER_PARTICIPANT) {
         scope =
-            new ConfigScopeBuilder().forCluster(_manager.getClusterName())
-                .forParticipant(_manager.getInstanceName()).build();
+            new HelixConfigScopeBuilder(ConfigScopeProperty.PARTICIPANT)
+                .forCluster(_manager.getClusterName()).forParticipant(_manager.getInstanceName())
+                .build();
         threadpoolSizeStr = configAccessor.get(scope, key);
       }
 
       if (threadpoolSizeStr == null) {
-        scope = new ConfigScopeBuilder().forCluster(_manager.getClusterName()).build();
+        scope =
+            new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(
+                _manager.getClusterName()).build();
         threadpoolSizeStr = configAccessor.get(scope, key);
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index 0ac3b58..8a380c4 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -493,7 +493,6 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   @Override
   public void onMessage(String instanceName, List<Message> messages,
       NotificationContext changeContext) {
-
     HelixManager manager = changeContext.getManager();
     if (_messageQueueMonitor == null) {
       _messageQueueMonitor =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/model/ConfigScope.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ConfigScope.java b/helix-core/src/main/java/org/apache/helix/model/ConfigScope.java
deleted file mode 100644
index 292ef0f..0000000
--- a/helix-core/src/main/java/org/apache/helix/model/ConfigScope.java
+++ /dev/null
@@ -1,137 +0,0 @@
-package org.apache.helix.model;
-
-/*
- * 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.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.util.StringTemplate;
-import org.apache.log4j.Logger;
-
-/**
- * @deprecated replaced by {@link HelixConfigScope}
- */
-@Deprecated
-public class ConfigScope {
-  private static Logger LOG = Logger.getLogger(ConfigScope.class);
-
-  private static final List<ConfigScopeProperty> scopePriority =
-      new ArrayList<ConfigScopeProperty>();
-  private static final Map<ConfigScopeProperty, Map<ConfigScopeProperty, ConfigScopeProperty>> scopeTransition =
-      new HashMap<ConfigScopeProperty, Map<ConfigScopeProperty, ConfigScopeProperty>>();
-  private static final StringTemplate template = new StringTemplate();
-  static {
-    // scope priority: CLUSTER > PARTICIPANT > RESOURCE > PARTITION
-    scopePriority.add(ConfigScopeProperty.CLUSTER);
-    scopePriority.add(ConfigScopeProperty.PARTICIPANT);
-    scopePriority.add(ConfigScopeProperty.RESOURCE);
-    scopePriority.add(ConfigScopeProperty.PARTITION);
-
-    // scope transition table to check valid inputs
-    scopeTransition.put(ConfigScopeProperty.CLUSTER,
-        new HashMap<ConfigScopeProperty, ConfigScopeProperty>());
-    scopeTransition.get(ConfigScopeProperty.CLUSTER).put(ConfigScopeProperty.PARTICIPANT,
-        ConfigScopeProperty.PARTICIPANT);
-    scopeTransition.get(ConfigScopeProperty.CLUSTER).put(ConfigScopeProperty.RESOURCE,
-        ConfigScopeProperty.RESOURCE);
-    scopeTransition.put(ConfigScopeProperty.RESOURCE,
-        new HashMap<ConfigScopeProperty, ConfigScopeProperty>());
-    scopeTransition.get(ConfigScopeProperty.RESOURCE).put(ConfigScopeProperty.PARTITION,
-        ConfigScopeProperty.PARTITION);
-
-    // string templates to generate znode path/index
-    // @formatter:off
-    template.addEntry(ConfigScopeProperty.CLUSTER, 2,
-        "/{clusterName}/CONFIGS/CLUSTER/{clusterName}");
-    template.addEntry(ConfigScopeProperty.PARTICIPANT, 2,
-        "/{clusterName}/CONFIGS/PARTICIPANT/{participantName}");
-    template.addEntry(ConfigScopeProperty.RESOURCE, 2,
-        "/{clusterName}/CONFIGS/RESOURCE/{resourceName}");
-    template.addEntry(ConfigScopeProperty.PARTITION, 3,
-        "/{clusterName}/CONFIGS/RESOURCE/{resourceName}|{partitionName}");
-    // @formatter:on
-  }
-
-  private final String _clusterName;
-  private final ConfigScopeProperty _scope;
-
-  /**
-   * _scopeStr is like:
-   * "/ClusterName/CONFIGS/{CLUSTER|PARTICIPANT|RESOURCE|PARTITION}/{clusterName|instanceName|resourceName}{|partitionName}"
-   */
-  private final String _scopeStr;
-
-  public ConfigScope(ConfigScopeBuilder configScopeBuilder) {
-    Map<ConfigScopeProperty, String> scopeMap = configScopeBuilder.getScopeMap();
-    List<String> keys = new ArrayList<String>();
-
-    ConfigScopeProperty curScope = null;
-    for (ConfigScopeProperty scope : scopePriority) {
-      if (scopeMap.containsKey(scope)) {
-        if (curScope == null && scope == ConfigScopeProperty.CLUSTER) {
-          keys.add(scopeMap.get(scope));
-          curScope = ConfigScopeProperty.CLUSTER;
-        } else if (curScope == null) {
-          throw new IllegalArgumentException("Missing CLUSTER scope. Can't build scope using "
-              + configScopeBuilder);
-        } else {
-          if (!scopeTransition.containsKey(curScope)
-              || !scopeTransition.get(curScope).containsKey(scope)) {
-            throw new IllegalArgumentException("Can't build scope using " + configScopeBuilder);
-          }
-          keys.add(scopeMap.get(scope));
-          curScope = scopeTransition.get(curScope).get(scope);
-        }
-      }
-    }
-
-    if (curScope == ConfigScopeProperty.CLUSTER) {
-      // append one more {clusterName}
-      keys.add(scopeMap.get(ConfigScopeProperty.CLUSTER));
-    }
-
-    String scopeStr = template.instantiate(curScope, keys.toArray(new String[0]));
-
-    _clusterName = keys.get(0);
-    _scopeStr = scopeStr;
-    _scope = curScope;
-  }
-
-  public ConfigScopeProperty getScope() {
-    return _scope;
-  }
-
-  public String getClusterName() {
-    return _clusterName;
-  }
-
-  public String getScopeStr() {
-    return _scopeStr;
-  }
-
-  @Override
-  public String toString() {
-    return super.toString() + ": " + _scopeStr;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/model/builder/ConfigScopeBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ConfigScopeBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ConfigScopeBuilder.java
deleted file mode 100644
index a8ce835..0000000
--- a/helix-core/src/main/java/org/apache/helix/model/builder/ConfigScopeBuilder.java
+++ /dev/null
@@ -1,127 +0,0 @@
-package org.apache.helix.model.builder;
-
-/*
- * 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.util.HashMap;
-import java.util.Map;
-
-import org.apache.helix.model.ConfigScope;
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.util.StringTemplate;
-import org.apache.log4j.Logger;
-
-/**
- * @deprecated replaced by {@link HelixConfigScopeBuilder}
- */
-@Deprecated
-public class ConfigScopeBuilder {
-  private static Logger LOG = Logger.getLogger(ConfigScopeBuilder.class);
-
-  private static StringTemplate template = new StringTemplate();
-  static {
-    // @formatter:off
-    template.addEntry(ConfigScopeProperty.CLUSTER, 1, "CLUSTER={clusterName}");
-    template.addEntry(ConfigScopeProperty.RESOURCE, 2,
-        "CLUSTER={clusterName},RESOURCE={resourceName}");
-    template.addEntry(ConfigScopeProperty.PARTITION, 3,
-        "CLUSTER={clusterName},RESOURCE={resourceName},PARTITION={partitionName}");
-    template.addEntry(ConfigScopeProperty.PARTICIPANT, 2,
-        "CLUSTER={clusterName},PARTICIPANT={participantName}");
-    // @formatter:on
-  }
-
-  private final Map<ConfigScopeProperty, String> _scopeMap;
-
-  public Map<ConfigScopeProperty, String> getScopeMap() {
-    return _scopeMap;
-  }
-
-  public ConfigScopeBuilder() {
-    _scopeMap = new HashMap<ConfigScopeProperty, String>();
-  }
-
-  public ConfigScopeBuilder forCluster(String clusterName) {
-    _scopeMap.put(ConfigScopeProperty.CLUSTER, clusterName);
-    return this;
-  }
-
-  public ConfigScopeBuilder forParticipant(String participantName) {
-    _scopeMap.put(ConfigScopeProperty.PARTICIPANT, participantName);
-    return this;
-  }
-
-  public ConfigScopeBuilder forResource(String resourceName) {
-    _scopeMap.put(ConfigScopeProperty.RESOURCE, resourceName);
-    return this;
-
-  }
-
-  public ConfigScopeBuilder forPartition(String partitionName) {
-    _scopeMap.put(ConfigScopeProperty.PARTITION, partitionName);
-    return this;
-
-  }
-
-  public ConfigScope build() {
-    // TODO: validate the scopes map
-    return new ConfigScope(this);
-  }
-
-  public ConfigScope build(ConfigScopeProperty scope, String clusterName, String... scopeKeys) {
-    if (scopeKeys == null) {
-      scopeKeys = new String[] {};
-    }
-
-    String[] args = new String[1 + scopeKeys.length];
-    args[0] = clusterName;
-    System.arraycopy(scopeKeys, 0, args, 1, scopeKeys.length);
-    String scopePairs = template.instantiate(scope, args);
-
-    return build(scopePairs);
-  }
-
-  public ConfigScope build(String scopePairs) {
-    String[] scopes = scopePairs.split("[\\s,]+");
-    for (String scope : scopes) {
-      try {
-        int idx = scope.indexOf('=');
-        if (idx == -1) {
-          LOG.error("Invalid scope string: " + scope);
-          continue;
-        }
-
-        String scopeStr = scope.substring(0, idx);
-        String value = scope.substring(idx + 1);
-        ConfigScopeProperty scopeProperty = ConfigScopeProperty.valueOf(scopeStr);
-        _scopeMap.put(scopeProperty, value);
-      } catch (Exception e) {
-        LOG.error("Invalid scope string: " + scope);
-        continue;
-      }
-    }
-
-    return build();
-  }
-
-  @Override
-  public String toString() {
-    return _scopeMap.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/participant/CustomCodeInvoker.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/CustomCodeInvoker.java b/helix-core/src/main/java/org/apache/helix/participant/CustomCodeInvoker.java
index a736d71..6c96629 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/CustomCodeInvoker.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/CustomCodeInvoker.java
@@ -21,10 +21,10 @@ package org.apache.helix.participant;
 
 import java.util.List;
 
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.Type;
@@ -35,7 +35,7 @@ import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.log4j.Logger;
 
-public class CustomCodeInvoker implements LiveInstanceChangeListener, ConfigChangeListener,
+public class CustomCodeInvoker implements LiveInstanceChangeListener, InstanceConfigChangeListener,
     ExternalViewChangeListener {
   private static Logger LOG = Logger.getLogger(CustomCodeInvoker.class);
   private final CustomCodeCallbackHandler _callback;
@@ -89,7 +89,7 @@ public class CustomCodeInvoker implements LiveInstanceChangeListener, ConfigChan
   }
 
   @Override
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
+  public void onInstanceConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
     LOG.info("onConfigChange() invoked");
     callParticipantCode(changeContext);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java b/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java
index aa21ee3..3866cf5 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/GenericLeaderStandbyModel.java
@@ -63,7 +63,7 @@ public class GenericLeaderStandbyModel extends StateModel {
       if (notificationType == ChangeType.LIVE_INSTANCE) {
         manager.addLiveInstanceChangeListener(_particHolder);
       } else if (notificationType == ChangeType.CONFIG) {
-        manager.addConfigChangeListener(_particHolder);
+        manager.addInstanceConfigChangeListener(_particHolder);
       } else if (notificationType == ChangeType.EXTERNAL_VIEW) {
         manager.addExternalViewChangeListener(_particHolder);
       } else {

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
index 9bba660..7799ca1 100644
--- a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
@@ -29,16 +29,16 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.helix.ConfigChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.log4j.Logger;
 
-public class RoutingTableProvider implements ExternalViewChangeListener, ConfigChangeListener {
+public class RoutingTableProvider implements ExternalViewChangeListener, InstanceConfigChangeListener {
   private static final Logger logger = Logger.getLogger(RoutingTableProvider.class);
   private final AtomicReference<RoutingTable> _routingTableRef;
 
@@ -105,7 +105,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
   }
 
   @Override
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
+  public void onInstanceConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
     // session has expired clean up the routing table
     if (changeContext.getType() == NotificationContext.Type.FINALIZE) {
       logger.info("Resetting the routing table. ");

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/Mocks.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/Mocks.java b/helix-core/src/test/java/org/apache/helix/Mocks.java
index a3a6044..0303f12 100644
--- a/helix-core/src/test/java/org/apache/helix/Mocks.java
+++ b/helix-core/src/test/java/org/apache/helix/Mocks.java
@@ -275,12 +275,6 @@ public class Mocks {
     }
 
     @Override
-    public void addConfigChangeListener(ConfigChangeListener listener) {
-      // TODO Auto-generated method stub
-
-    }
-
-    @Override
     public void addMessageListener(MessageListener listener, String instanceName) {
       // TODO Auto-generated method stub
 


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

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 ac20652..63b5666 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
@@ -30,10 +30,10 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.HelixDefinedState;
+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.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
@@ -56,12 +56,12 @@ public class TestDrop extends ZkTestBase {
    * @param participants
    */
   private void assertEmptyCSandEV(String clusterName, String db,
-      MockParticipantManager[] participants) {
+      MockParticipant[] participants) {
     HelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, _baseAccessor);
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     Assert.assertNull(accessor.getProperty(keyBuilder.externalView(db)));
 
-    for (MockParticipantManager participant : participants) {
+    for (MockParticipant participant : participants) {
       String instanceName = participant.getInstanceName();
       String sessionId = participant.getSessionId();
       Assert.assertNull(accessor.getProperty(keyBuilder.currentState(instanceName, sessionId, db)));
@@ -77,7 +77,7 @@ public class TestDrop 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
@@ -89,15 +89,15 @@ public class TestDrop extends ZkTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     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();
     }
 
@@ -132,7 +132,7 @@ public class TestDrop 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
@@ -144,8 +144,8 @@ public class TestDrop 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
@@ -157,10 +157,10 @@ public class TestDrop extends ZkTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -206,7 +206,7 @@ public class TestDrop 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
@@ -218,8 +218,8 @@ public class TestDrop 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
@@ -231,10 +231,10 @@ public class TestDrop extends ZkTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -314,7 +314,7 @@ public class TestDrop 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
@@ -340,8 +340,8 @@ public class TestDrop extends ZkTestBase {
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), isBuilder.build());
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
@@ -352,10 +352,10 @@ public class TestDrop extends ZkTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -400,7 +400,7 @@ public class TestDrop 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
@@ -412,15 +412,15 @@ public class TestDrop 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();
     }
 
@@ -486,16 +486,16 @@ public class TestDrop extends ZkTestBase {
     ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, _baseAccessor);
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
 
-    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/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 d4faf84..d661601 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
@@ -20,7 +20,7 @@ package org.apache.helix.integration;
  */
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -69,7 +69,7 @@ public class TestDropResource extends ZkStandAloneCMTestBase {
         TestHelper.<String> setOf("localhost_12918", "localhost_12919",
         /* "localhost_12920", */"localhost_12921", "localhost_12922"), _zkaddr);
 
-    _participants[2] = new MockParticipantManager(_zkaddr, CLUSTER_NAME, hostToKill);
+    _participants[2] = new MockParticipant(_zkaddr, CLUSTER_NAME, hostToKill);
     _participants[2].syncStart();
 
     TestHelper.verifyWithTimeout("verifyEmptyCurStateAndExtView", 30 * 1000, CLUSTER_NAME, "MyDB2",

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 83e3001..21a1be3 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
@@ -26,8 +26,8 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.State;
 import org.apache.helix.api.id.PartitionId;
-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.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.testutil.ZkTestBase;
@@ -99,21 +99,21 @@ public class TestEnablePartitionDuringDisable 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
     EnablePartitionTransition transition = new EnablePartitionTransition();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (instanceName.equals("localhost_12919")) {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(transition);
       } else {
-        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/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 4119fc6..09fe1e4 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
@@ -28,11 +28,10 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 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.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.testutil.ZkTestBase;
@@ -78,8 +77,8 @@ public class TestEntropyFreeNodeBounce extends ZkTestBase {
     }
 
     // Start the controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // get an admin and accessor
@@ -97,14 +96,14 @@ public class TestEntropyFreeNodeBounce extends ZkTestBase {
       Assert.assertTrue(result);
       ExternalView stableExternalView =
           accessor.getProperty(keyBuilder.externalView(RESOURCE_NAME));
-      for (HelixManager participant : participants) {
+      for (int i = 0; i < NUM_PARTICIPANTS; i++) {
         // disable the controller, bounce the node, re-enable the controller, verify assignments
         // remained the same
         helixAdmin.enableCluster(clusterName, false);
-        participant.disconnect();
+        participants[i].disconnect();
         Thread.sleep(1000);
-        participant = createParticipant(clusterName, participant.getInstanceName());
-        participant.connect();
+        participants[i] = createParticipant(clusterName, participants[i].getInstanceName());
+        participants[i].connect();
         Thread.sleep(1000);
         helixAdmin.enableCluster(clusterName, true);
         Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 edc2965..19af9a7 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,8 +26,8 @@ import java.util.Map;
 import java.util.Set;
 
 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.ZKHelixAdmin;
 import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.testutil.TestUtil;
@@ -40,7 +40,7 @@ public class TestErrorPartition extends ZkTestBase {
   @Test()
   public void testErrorPartition() throws Exception {
     String clusterName = TestUtil.getTestName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     System.out.println("START testErrorPartition() at " + new Date(System.currentTimeMillis()));
     ZKHelixAdmin tool = new ZKHelixAdmin(_zkclient);
@@ -48,8 +48,8 @@ public class TestErrorPartition extends ZkTestBase {
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, "localhost", "TestDB", 1, 10, 5, 3,
         "MasterSlave", true);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     for (int i = 0; i < 5; i++) {
@@ -61,10 +61,10 @@ public class TestErrorPartition extends ZkTestBase {
             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();
     }
@@ -115,7 +115,7 @@ public class TestErrorPartition extends ZkTestBase {
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             _zkaddr, clusterName));
     Assert.assertTrue(result);
-    participants[0] = new MockParticipantManager(_zkaddr, clusterName, "localhost_12918");
+    participants[0] = new MockParticipant(_zkaddr, clusterName, "localhost_12918");
     new Thread(participants[0]).start();
 
     result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 835f81e..8791ff8 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
@@ -25,8 +25,8 @@ import java.util.List;
 
 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.testutil.TestUtil;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -42,7 +42,7 @@ public class TestExternalViewUpdates extends ZkTestBase {
     System.out.println("START testExternalViewUpdates at " + new Date(System.currentTimeMillis()));
 
     String clusterName = TestUtil.getTestName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
     int resourceNb = 10;
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -53,15 +53,15 @@ public class TestExternalViewUpdates extends ZkTestBase {
         1, // 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 < 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/TestFullAutoNodeTagging.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
index 6704fa9..20ea617 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
@@ -28,8 +28,8 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.Verifier;
-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;
@@ -95,16 +95,16 @@ public class TestFullAutoNodeTagging extends ZkTestBase {
     }
 
     // Start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // Start participants
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -193,16 +193,16 @@ public class TestFullAutoNodeTagging extends ZkTestBase {
     helixAdmin.setResourceIdealState(clusterName, RESOURCE_NAME, idealState);
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -264,16 +264,16 @@ public class TestFullAutoNodeTagging extends ZkTestBase {
     helixAdmin.setResourceIdealState(clusterName, RESOURCE_NAME, idealState);
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       final String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
 
       // ensure that everything is valid if this is a tagged node that is starting

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 2d95811..1acc142 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
@@ -27,8 +27,8 @@ import org.apache.helix.HelixManager;
 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.model.LiveInstance;
 import org.apache.helix.participant.CustomCodeCallbackHandler;
@@ -87,15 +87,15 @@ public class TestHelixCustomCodeRunner extends ZkTestBase {
         _nodeNb, // replica
         "MasterSlave", true);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, _clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, _clusterName, "controller_0");
     controller.syncStart();
 
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
     for (int i = 0; i < _nodeNb; i++) {
       String instanceName = "localhost_" + (_startPort + i);
 
-      participants[i] = new MockParticipantManager(_zkaddr, _clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, _clusterName, instanceName);
 
       registerCustomCodeRunner(participants[i]);
       participants[i].syncStart();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java b/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java
index c6d963d..0acd107 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInstanceAutoJoin.java
@@ -2,11 +2,12 @@ package org.apache.helix.integration;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.model.ConfigScope;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -43,8 +44,7 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 1);
     String instance2 = "localhost_279699";
     // StartCMResult result = TestHelper.startDummyProcess(zkaddr, CLUSTER_NAME, instance2);
-    MockParticipantManager newParticipant =
-        new MockParticipantManager(_zkaddr, CLUSTER_NAME, instance2);
+    MockParticipant newParticipant = new MockParticipant(_zkaddr, CLUSTER_NAME, instance2);
     newParticipant.syncStart();
 
     Thread.sleep(500);
@@ -52,11 +52,12 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
     Assert.assertTrue(null == manager.getHelixDataAccessor().getProperty(
         accessor.keyBuilder().liveInstance(instance2)));
 
-    ConfigScope scope = new ConfigScopeBuilder().forCluster(CLUSTER_NAME).build();
+    HelixConfigScope scope =
+        new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(CLUSTER_NAME).build();
 
     manager.getConfigAccessor().set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "true");
 
-    newParticipant = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instance2);
+    newParticipant = new MockParticipant(_zkaddr, CLUSTER_NAME, instance2);
     newParticipant.syncStart();
 
     Thread.sleep(500);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
index 1f1af0e..b89d184 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
@@ -28,8 +28,8 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.id.StateModelDefId;
-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.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -91,16 +91,16 @@ public class TestInvalidAutoIdealState extends ZkTestBase {
     admin.setResourceIdealState(clusterName, "TestDB", idealState);
 
     // 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();
     }
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     boolean result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 2cedb83..9202f4f 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
@@ -24,8 +24,8 @@ import java.util.Map;
 
 import org.apache.helix.HelixAdmin;
 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.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
@@ -66,8 +66,8 @@ public class TestInvalidResourceRebalance extends ZkTestBase {
         true); // do rebalance
 
     // start controller
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // add the ideal state spec (prevents non-CUSTOMIZED MasterSlave ideal states)
@@ -80,11 +80,11 @@ public class TestInvalidResourceRebalance extends ZkTestBase {
         properties);
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       final 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/TestMessageThrottle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
index 1442979..6201dca 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
@@ -29,13 +29,12 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 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.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.Message;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
 import org.apache.helix.testutil.TestUtil;
 import org.apache.helix.testutil.ZkTestBase;
@@ -51,7 +50,7 @@ public class TestMessageThrottle extends ZkTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
 
     String clusterName = TestUtil.getTestName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -115,15 +114,15 @@ public class TestMessageThrottle extends ZkTestBase {
       });
     }
 
-    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/TestMessageThrottle2.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
index 2731b79..496a16f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
@@ -34,7 +34,6 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
-import org.apache.helix.InstanceType;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
@@ -42,9 +41,10 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZKHelixManager;
+import org.apache.helix.manager.zk.ZkHelixConnection;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -237,32 +237,34 @@ public class TestMessageThrottle2 extends ZkTestBase {
 
   static final class MyProcess {
     private final String instanceName;
-    private HelixManager helixManager;
+    // private HelixManager helixManager;
+    private MockParticipant participant;
 
     public MyProcess(String instanceName) {
       this.instanceName = instanceName;
     }
 
     public void start() throws Exception {
-      helixManager =
-          new ZKHelixManager(clusterName, instanceName, InstanceType.PARTICIPANT, _zkaddr);
+//      helixManager =
+//          new ZKHelixManager(clusterName, instanceName, InstanceType.PARTICIPANT, _zkaddr);
+      participant = new MockParticipant(_zkaddr, clusterName, instanceName);
       {
         // hack to set sessionTimeout
-        Field sessionTimeout = ZKHelixManager.class.getDeclaredField("_sessionTimeout");
+        Field sessionTimeout = ZkHelixConnection.class.getDeclaredField("_sessionTimeout");
         sessionTimeout.setAccessible(true);
-        sessionTimeout.setInt(helixManager, 1000);
+        sessionTimeout.setInt(participant.getConn(), 1000);
       }
 
-      StateMachineEngine stateMach = helixManager.getStateMachineEngine();
-      stateMach.registerStateModelFactory("MasterSlave", new MyStateModelFactory(helixManager));
-      helixManager.connect();
+      StateMachineEngine stateMach = participant.getStateMachineEngine();
+      stateMach.registerStateModelFactory("MasterSlave", new MyStateModelFactory(participant));
+      participant.connect();
 
       StatusPrinter statusPrinter = new StatusPrinter();
-      statusPrinter.registerWith(helixManager);
+      statusPrinter.registerWith(participant);
     }
 
     public void stop() {
-      helixManager.disconnect();
+      participant.disconnect();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
index 7df9e8b..08954e5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
@@ -84,7 +84,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBase {
   }
 
   @Test()
-  public void TestMessageSimpleSend() throws Exception {
+  public void testMessageSimpleSend() throws Exception {
     String hostSrc = "localhost_" + START_PORT;
     String hostDest = "localhost_" + (START_PORT + 1);
 
@@ -176,7 +176,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBase {
   }
 
   @Test()
-  public void TestMessageSimpleSendReceiveAsync() throws Exception {
+  public void testMessageSimpleSendReceiveAsync() throws Exception {
     String hostSrc = "localhost_" + START_PORT;
     String hostDest = "localhost_" + (START_PORT + 1);
 
@@ -243,7 +243,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBase {
   }
 
   @Test()
-  public void TestBlockingSendReceive() throws Exception {
+  public void testBlockingSendReceive() throws Exception {
     String hostSrc = "localhost_" + START_PORT;
     String hostDest = "localhost_" + (START_PORT + 1);
 
@@ -282,7 +282,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBase {
   }
 
   @Test()
-  public void TestMultiMessageCriteria() throws Exception {
+  public void testMultiMessageCriteria() throws Exception {
     String hostSrc = "localhost_" + START_PORT;
 
     for (int i = 0; i < NODE_NR; i++) {
@@ -382,7 +382,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBase {
   }
 
   @Test()
-  public void TestControllerMessage() throws Exception {
+  public void testControllerMessage() throws Exception {
     String hostSrc = "localhost_" + START_PORT;
 
     for (int i = 0; i < NODE_NR; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
index 1329cbe..105633a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
@@ -22,13 +22,12 @@ package org.apache.helix.integration;
 import java.util.Date;
 
 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.mock.participant.MockBootstrapModelFactory;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.testutil.TestUtil;
 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.apache.log4j.Logger;
@@ -52,16 +51,16 @@ public class TestNonOfflineInitState extends ZkTestBase {
         1, // replicas
         "Bootstrap", 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[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);
 
       // add a state model with non-OFFLINE initial state
       StateMachineEngine stateMach = participants[i].getStateMachineEngine();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 544fbd5..461d8bb 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
@@ -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.model.IdealState;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -45,7 +45,7 @@ public class TestNullReplica extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -62,15 +62,15 @@ public class TestNullReplica extends ZkTestBase {
     idealState.getSimpleFields().remove(IdealState.IdealStateProperty.REPLICAS.toString());
     _zkclient.writeData(idealStatePath, 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/TestParticipantNameCollision.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java
index d4d56df..bc30754 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantNameCollision.java
@@ -22,7 +22,7 @@ package org.apache.helix.integration;
 import java.util.Date;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.MockParticipant;
 import org.apache.log4j.Logger;
 import org.testng.annotations.Test;
 
@@ -33,13 +33,13 @@ public class TestParticipantNameCollision extends ZkStandAloneCMTestBase {
   public void testParticiptantNameCollision() throws Exception {
     logger.info("RUN TestParticipantNameCollision() at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager newParticipant = null;
+    MockParticipant newParticipant = null;
     for (int i = 0; i < 1; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
       try {
         // the call fails on getClusterManagerForParticipant()
         // no threads start
-        newParticipant = new MockParticipantManager(_zkaddr, CLUSTER_NAME, instanceName);
+        newParticipant = new MockParticipant(_zkaddr, CLUSTER_NAME, instanceName);
         newParticipant.syncStart();
       } catch (Exception e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 2f27fd2..823a9ce 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
@@ -24,17 +24,14 @@ import java.util.Date;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
-import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixAdmin;
 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.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.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.IdealState;
@@ -139,15 +136,15 @@ public class TestPartitionLevelTransitionConstraint extends ZkTestBase {
     admin.setConstraint(clusterName, ConstraintType.MESSAGE_CONSTRAINT, "constraint1",
         constraintItemBuilder.build());
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start 1st participant
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     String instanceName1 = "localhost_12918";
 
-    participants[0] = new MockParticipantManager(_zkaddr, clusterName, instanceName1);
+    participants[0] = new MockParticipant(_zkaddr, clusterName, instanceName1);
     participants[0].getStateMachineEngine().registerStateModelFactory("Bootstrap",
         new BootstrapStateModelFactory());
     participants[0].syncStart();
@@ -160,7 +157,7 @@ public class TestPartitionLevelTransitionConstraint extends ZkTestBase {
 
     // start 2nd participant which will be the master for Test0_0
     String instanceName2 = "localhost_12919";
-    participants[1] = new MockParticipantManager(_zkaddr, clusterName, instanceName2);
+    participants[1] = new MockParticipant(_zkaddr, clusterName, instanceName2);
     participants[1].getStateMachineEngine().registerStateModelFactory("Bootstrap",
         new BootstrapStateModelFactory());
     participants[1].syncStart();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 2d87e61..5c59348 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,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.HelixDataAccessor;
 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.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -48,7 +48,7 @@ public class TestPauseSignal extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -60,15 +60,15 @@ public class TestPauseSignal 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 < 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/TestPreferenceListAsQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java b/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
index 6de604b..06a2b56 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPreferenceListAsQueue.java
@@ -44,7 +44,6 @@ import org.apache.helix.model.builder.ConstraintItemBuilder;
 import org.apache.helix.participant.statemachine.StateModel;
 import org.apache.helix.participant.statemachine.StateModelFactory;
 import org.apache.helix.testutil.ZkTestBase;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.BeforeMethod;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java b/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java
index 1ce31f4..b7f61c6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRedefineStateModelDef.java
@@ -26,8 +26,8 @@ import java.util.TreeMap;
 
 import org.apache.helix.PropertyKey;
 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.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
@@ -66,16 +66,16 @@ public class TestRedefineStateModelDef extends ZkTestBase {
     autoRebalance(clusterName);
 
     // 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[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();
     }
 
@@ -88,7 +88,7 @@ public class TestRedefineStateModelDef extends ZkTestBase {
     // stop controller, redefine state model definition, and re-start controller
     controller.syncStop();
     redefineStateModelDef(clusterName);
-    controller = new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    controller = new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     result =

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 5595d0c..e522008 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
@@ -24,13 +24,12 @@ import java.util.Date;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 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.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 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;
@@ -76,18 +75,18 @@ public class TestReelectedPipelineCorrectness extends ZkTestBase {
     _setupTool.activateCluster(clusterName, controllerCluster, true);
 
     // start participants
-    MockParticipantManager[] participants = new MockParticipantManager[NUM_PARTICIPANTS];
+    MockParticipant[] participants = new MockParticipant[NUM_PARTICIPANTS];
     for (int i = 0; i < NUM_PARTICIPANTS; i++) {
       final String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       participants[i].syncStart();
     }
 
     // start controllers
-    ClusterDistributedController[] controllers = new ClusterDistributedController[NUM_CONTROLLERS];
+    MockMultiClusterController[] controllers = new MockMultiClusterController[NUM_CONTROLLERS];
     for (int i = 0; i < NUM_CONTROLLERS; i++) {
       controllers[i] =
-          new ClusterDistributedController(_zkaddr, controllerCluster, "controller_" + i);
+          new MockMultiClusterController(_zkaddr, controllerCluster, "controller_" + i);
       controllers[i].syncStart();
     }
     Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 c692ed5..f619503 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
@@ -30,8 +30,8 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.id.StateModelDefId;
 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.ZKHelixDataAccessor;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
@@ -43,12 +43,12 @@ import org.testng.annotations.Test;
 
 public class TestRenamePartition extends ZkTestBase {
   // map from clusterName to participants
-  final Map<String, MockParticipantManager[]> _participantMap =
-      new ConcurrentHashMap<String, MockParticipantManager[]>();
+  final Map<String, MockParticipant[]> _participantMap =
+      new ConcurrentHashMap<String, MockParticipant[]>();
 
   // map from clusterName to controllers
-  final Map<String, ClusterControllerManager> _controllerMap =
-      new ConcurrentHashMap<String, ClusterControllerManager>();
+  final Map<String, MockController> _controllerMap =
+      new ConcurrentHashMap<String, MockController>();
 
   @Test()
   public void testRenamePartitionAutoIS() throws Exception {
@@ -136,17 +136,17 @@ public class TestRenamePartition extends ZkTestBase {
   }
 
   private void startAndVerify(String clusterName) throws Exception {
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
-    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();
     }
 
@@ -160,14 +160,14 @@ public class TestRenamePartition extends ZkTestBase {
   }
 
   private void stop(String clusterName) {
-    ClusterControllerManager controller = _controllerMap.get(clusterName);
+    MockController controller = _controllerMap.get(clusterName);
     if (controller != null) {
       controller.syncStop();
     }
 
-    MockParticipantManager[] participants = _participantMap.get(clusterName);
+    MockParticipant[] participants = _participantMap.get(clusterName);
     if (participants != null) {
-      for (MockParticipantManager participant : participants) {
+      for (MockParticipant participant : participants) {
         participant.syncStop();
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 e6b9c2d..5804744 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,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 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.mock.participant.ErrTransition;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterSetup;
@@ -55,8 +55,8 @@ public class TestResetInstance 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();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -67,15 +67,15 @@ public class TestResetInstance extends ZkTestBase {
     };
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 85ab192..4855b3d 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,9 +28,9 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.State;
+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.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
@@ -80,8 +80,8 @@ public class TestResetPartitionState 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();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -92,15 +92,15 @@ public class TestResetPartitionState extends ZkTestBase {
     };
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 60983af..7d28931 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,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 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.mock.participant.ErrTransition;
 import org.apache.helix.testutil.ZkTestBase;
 import org.apache.helix.tools.ClusterSetup;
@@ -54,8 +54,8 @@ public class TestResetResource 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();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -66,15 +66,15 @@ public class TestResetResource extends ZkTestBase {
     };
 
     // start mock participants
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        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();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java b/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java
index e9d2b45..6057583 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRestartParticipant.java
@@ -24,8 +24,8 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.helix.NotificationContext;
 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.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.testutil.TestUtil;
@@ -37,15 +37,15 @@ import org.testng.annotations.Test;
 
 public class TestRestartParticipant extends ZkTestBase {
   public class KillOtherTransition extends MockTransition {
-    final AtomicReference<MockParticipantManager> _other;
+    final AtomicReference<MockParticipant> _other;
 
-    public KillOtherTransition(MockParticipantManager other) {
-      _other = new AtomicReference<MockParticipantManager>(other);
+    public KillOtherTransition(MockParticipant other) {
+      _other = new AtomicReference<MockParticipant>(other);
     }
 
     @Override
     public void doTransition(Message message, NotificationContext context) {
-      MockParticipantManager other = _other.getAndSet(null);
+      MockParticipant other = _other.getAndSet(null);
       if (other != null) {
         System.err.println("Kill " + other.getInstanceName()
             + ". Interrupted exceptions are IGNORABLE");
@@ -60,7 +60,7 @@ public class TestRestartParticipant extends ZkTestBase {
     System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis()));
 
     String clusterName = TestUtil.getTestName();
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -71,8 +71,8 @@ public class TestRestartParticipant 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
@@ -80,10 +80,10 @@ public class TestRestartParticipant extends ZkTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 4) {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
         participants[i].setTransition(new KillOtherTransition(participants[0]));
       } else {
-        participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
+        participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
       }
 
       participants[i].syncStart();
@@ -96,8 +96,8 @@ public class TestRestartParticipant extends ZkTestBase {
 
     // restart
     Thread.sleep(500);
-    MockParticipantManager participant =
-        new MockParticipantManager(_zkaddr, participants[0].getClusterName(),
+    MockParticipant participant =
+        new MockParticipant(_zkaddr, participants[0].getClusterName(),
             participants[0].getInstanceName());
     System.err.println("Restart " + participant.getInstanceName());
     participant.syncStart();

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 9adf374..67d5e32 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,8 +26,8 @@ import java.util.Map;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.PropertyKey;
 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.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -45,7 +45,7 @@ public class TestSchemataSM extends ZkTestBase {
     String clusterName = className + "_" + methodName;
     int n = 5;
 
-    MockParticipantManager[] participants = new MockParticipantManager[n];
+    MockParticipant[] participants = new MockParticipant[n];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -69,15 +69,15 @@ public class TestSchemataSM extends ZkTestBase {
         Arrays.asList(HelixConstants.StateModelToken.ANY_LIVEINSTANCE.toString()));
     accessor.setProperty(key, idealState);
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller");
     controller.syncStart();
 
     // start n-1 participants
     for (int i = 1; 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();
     }
 
@@ -87,7 +87,7 @@ public class TestSchemataSM extends ZkTestBase {
     Assert.assertTrue(result);
 
     // start the remaining 1 participant
-    participants[0] = new MockParticipantManager(_zkaddr, clusterName, "localhost_12918");
+    participants[0] = new MockParticipant(_zkaddr, clusterName, "localhost_12918");
     participants[0].syncStart();
 
     // make sure we have all participants in MASTER state

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 bbca923..9e80bb9 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
@@ -26,8 +26,8 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.api.id.PartitionId;
-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.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.testutil.ZkTestBase;
@@ -42,10 +42,14 @@ public class TestSessionExpiryInTransition extends ZkTestBase {
 
   public class SessionExpiryTransition extends MockTransition {
     private final AtomicBoolean _done = new AtomicBoolean();
+    private final MockParticipant _participant;
+
+    public SessionExpiryTransition(MockParticipant participant) {
+      _participant = participant;
+    }
 
     @Override
     public void doTransition(Message message, NotificationContext context) {
-      MockParticipantManager manager = (MockParticipantManager) context.getManager();
 
       String instance = message.getTgtName();
       PartitionId partition = message.getPartitionId();
@@ -54,7 +58,7 @@ public class TestSessionExpiryInTransition extends ZkTestBase {
           // on localhost_12918
           && _done.getAndSet(true) == false) {
         try {
-          ZkTestHelper.expireSession(manager.getZkClient());
+          ZkTestHelper.expireSession(_participant.getZkClient());
         } catch (Exception e) {
           LOG.error("Exception expire zk-session", e);
         }
@@ -72,7 +76,7 @@ public class TestSessionExpiryInTransition extends ZkTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipantManager[] participants = new MockParticipantManager[5];
+    MockParticipant[] participants = new MockParticipant[5];
 
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, // participant port
         "localhost", // participant name prefix
@@ -84,15 +88,15 @@ public class TestSessionExpiryInTransition 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 < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipantManager(_zkaddr, clusterName, instanceName);
-      participants[i].setTransition(new SessionExpiryTransition());
+      participants[i] = new MockParticipant(_zkaddr, clusterName, instanceName);
+      participants[i].setTransition(new SessionExpiryTransition(participants[i]));
       participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestSharedConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSharedConnection.java b/helix-core/src/test/java/org/apache/helix/integration/TestSharedConnection.java
index 665db05..8b77ce7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSharedConnection.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSharedConnection.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixConnection;
 import org.apache.helix.HelixController;
+import org.apache.helix.HelixManager;
 import org.apache.helix.HelixParticipant;
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.State;
@@ -33,7 +34,7 @@ import org.apache.helix.api.id.ControllerId;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.manager.zk.HelixConnectionAdaptor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkHelixConnection;
 import org.apache.helix.manager.zk.ZkHelixLeaderElection;
 import org.apache.helix.model.IdealState;
@@ -180,7 +181,7 @@ public class TestSharedConnection extends ZkTestBase {
     // Now verify that exactly one is leader
     int leaderCount = 0;
     for (HelixController controller : controllers) {
-      HelixConnectionAdaptor adaptor = new HelixConnectionAdaptor(controller);
+      HelixManager adaptor = new ZKHelixManager(controller);
       boolean result = ZkHelixLeaderElection.tryUpdateController(adaptor);
       if (result) {
         leaderCount++;

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java
index c1854c8..9c012a7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMMain.java
@@ -25,7 +25,7 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.Verifier;
-import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.MockController;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -39,10 +39,10 @@ public class TestStandAloneCMMain extends ZkStandAloneCMTestBase {
   @Test()
   public void testStandAloneCMMain() throws Exception {
     logger.info("RUN testStandAloneCMMain() at " + new Date(System.currentTimeMillis()));
-    ClusterControllerManager newController = null;
+    MockController newController = null;
     for (int i = 1; i <= 2; i++) {
       String controllerName = "controller_" + i;
-      newController = new ClusterControllerManager(_zkaddr, CLUSTER_NAME, controllerName);
+      newController = new MockController(_zkaddr, CLUSTER_NAME, controllerName);
       newController.syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/02165c52/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 da93e12..30c67c8 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,10 +23,9 @@ import java.util.Date;
 
 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.testutil.ZkTestBase;
-import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
@@ -47,15 +46,15 @@ public class TestStandAloneCMSessionExpiry extends ZkTestBase {
     TestHelper.setupCluster(clusterName, _zkaddr, 12918, "localhost", "TestDB", 1, 20, 5, 3,
         "MasterSlave", true);
 
-    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();
     }
 
-    ClusterControllerManager controller =
-        new ClusterControllerManager(_zkaddr, clusterName, "controller_0");
+    MockController controller =
+        new MockController(_zkaddr, clusterName, "controller_0");
     controller.syncStart();
 
     boolean result;
@@ -65,7 +64,7 @@ public class TestStandAloneCMSessionExpiry extends ZkTestBase {
     Assert.assertTrue(result);
 
     // participant session expiry
-    MockParticipantManager participantToExpire = participants[1];
+    MockParticipant participantToExpire = participants[1];
 
     System.out.println("Expire participant session");
     String oldSessionId = participantToExpire.getSessionId();