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 2013/11/11 22:09:55 UTC

[01/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Updated Branches:
  refs/heads/master ea8c61d7b -> ebd37fed4


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
index 32b9ec9..9036cf3 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
@@ -26,10 +26,14 @@ import org.apache.helix.api.id.MessageId;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.participant.DistClusterControllerStateModel;
+import org.apache.log4j.Logger;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public class TestDistControllerStateModel extends ZkUnitTestBase {
+  private static Logger LOG = Logger.getLogger(TestDistControllerStateModel.class);
+
   final String clusterName = CLUSTER_PREFIX + "_" + getShortClassName();
   DistClusterControllerStateModel stateModel = null;
 
@@ -55,8 +59,7 @@ public class TestDistControllerStateModel extends ZkUnitTestBase {
     try {
       stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null));
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Exception becoming leader from standby", e);
     }
     stateModel.onBecomeStandbyFromLeader(message, new NotificationContext(null));
   }
@@ -96,8 +99,7 @@ public class TestDistControllerStateModel extends ZkUnitTestBase {
     try {
       stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null));
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Exception becoming leader from standby", e);
     }
     stateModel.rollbackOnError(message, new NotificationContext(null), null);
   }
@@ -110,8 +112,7 @@ public class TestDistControllerStateModel extends ZkUnitTestBase {
     try {
       stateModel.onBecomeLeaderFromStandby(message, new NotificationContext(null));
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Exception becoming leader from standby", e);
     }
     stateModel.reset();
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 6d2b732..4d7f93c 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,13 +31,12 @@ 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.controller.HelixControllerMain;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterDistributedController;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
@@ -240,8 +239,8 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     final int n = 6;
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
-    MockParticipant[] participants = new MockParticipant[n];
-    ClusterController[] controllers = new ClusterController[2];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
 
     // activate clusters
@@ -311,14 +310,12 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     Assert.assertTrue(verifyResult);
 
     // clean up
-    // for (int i = 0; i < 2; i++) {
-    // controllers[i].syncStop();
-    // Thread.sleep(1000); // wait for all zk callbacks done
-    // }
-    // Thread.sleep(5000);
-    // for (int i = 0; i < n; i++) {
-    // participants[i].syncStop();
-    // }
+    for (ClusterDistributedController controller : controllers) {
+      controller.syncStop();
+    }
+    for (int i = 0; i < participants.length; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -333,11 +330,11 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
-    ClusterController[] controllers = new ClusterController[2];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
-        "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true";
+        "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     Thread.sleep(500);
 
@@ -357,7 +354,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     pw.write(new String(serializer.serialize(idealState.getRecord())));
     pw.close();
 
-    command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 ";
+    command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 ";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     boolean verifyResult =
@@ -365,7 +362,8 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(verifyResult);
 
-    command = "-zkSvr localhost:2183 -addIdealState " + clusterName + " db_11 " + tmpIdealStateFile;
+    command =
+        "-zkSvr " + ZK_ADDR + " -addIdealState " + clusterName + " db_11 " + tmpIdealStateFile;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     verifyResult =
@@ -377,61 +375,59 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     Assert.assertTrue(idealState2.getRecord().equals(idealState.getRecord()));
 
     // clean up
-    // for (int i = 0; i < 2; i++) {
-    // controllers[i].syncStop();
-    // Thread.sleep(1000); // wait for all zk callbacks done
-    // }
-    // Thread.sleep(5000);
-    // for (int i = 0; i < n; i++) {
-    // participants[i].syncStop();
-    // }
+    for (int i = 0; i < controllers.length; i++) {
+      controllers[i].syncStop();
+    }
+    for (int i = 0; i < participants.length; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
   }
 
   private void setupCluster(String clusterName, String grandClusterName, final int n,
-      MockParticipant[] participants, ClusterController[] controllers) throws Exception,
+      MockParticipantManager[] participants, ClusterDistributedController[] controllers)
+      throws Exception,
       InterruptedException {
     // add cluster
-    String command = "-zkSvr localhost:2183 -addCluster " + clusterName;
+    String command = "-zkSvr " + ZK_ADDR + " -addCluster " + clusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // add grand cluster
-    command = "-zkSvr localhost:2183 -addCluster " + grandClusterName;
+    command = "-zkSvr " + ZK_ADDR + " -addCluster " + grandClusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // add nodes
     for (int i = 0; i < n; i++) {
-      command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:123" + i;
+      command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:123" + i;
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     }
 
     // add resource
-    command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave";
+    command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // rebalance with key prefix
-    command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -key alias";
+    command = "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -key alias";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // add nodes to grand cluster
     command =
-        "-zkSvr localhost:2183 -addNode " + grandClusterName + " controller:9000;controller:9001";
+        "-zkSvr " + ZK_ADDR + " -addNode " + grandClusterName + " controller:9000;controller:9001";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // start mock nodes
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_123" + i;
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
     // start controller nodes
     for (int i = 0; i < 2; i++) {
       controllers[i] =
-          new ClusterController(grandClusterName, "controller_900" + i, ZK_ADDR,
-              HelixControllerMain.DISTRIBUTED);
+          new ClusterDistributedController(ZK_ADDR, grandClusterName, "controller_900" + i);
       controllers[i].syncStart();
     }
 
@@ -448,16 +444,16 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
-    ClusterController[] controllers = new ClusterController[2];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
-        "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true";
+        "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     Thread.sleep(500);
 
     // drop node should fail if the node is not disabled
-    command = "-zkSvr localhost:2183 -dropNode " + clusterName + " localhost:1232";
+    command = "-zkSvr " + ZK_ADDR + " -dropNode " + clusterName + " localhost:1232";
     try {
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
       Assert.fail("dropNode should fail since the node is not disabled");
@@ -466,11 +462,11 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     }
 
     // disabled node
-    command = "-zkSvr localhost:2183 -enableInstance " + clusterName + " localhost:1232 false";
+    command = "-zkSvr " + ZK_ADDR + " -enableInstance " + clusterName + " localhost:1232 false";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // Cannot dropNode if the node is not disconnected
-    command = "-zkSvr localhost:2183 -dropNode " + clusterName + " localhost:1232";
+    command = "-zkSvr " + ZK_ADDR + " -dropNode " + clusterName + " localhost:1232";
     try {
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
       Assert.fail("dropNode should fail since the node is not disconnected");
@@ -480,7 +476,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     // Cannot swapNode if the node is not disconnected
     command =
-        "-zkSvr localhost:2183 -swapInstance " + clusterName + " localhost_1232 localhost_12320";
+        "-zkSvr " + ZK_ADDR + " -swapInstance " + clusterName + " localhost_1232 localhost_12320";
     try {
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
       Assert.fail("swapInstance should fail since the node is not disconnected");
@@ -492,12 +488,12 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     participants[2].syncStop();
 
     // add new node then swap instance
-    command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:12320";
+    command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:12320";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // swap instance. The instance get swapped out should not exist anymore
     command =
-        "-zkSvr localhost:2183 -swapInstance " + clusterName + " localhost_1232 localhost_12320";
+        "-zkSvr " + ZK_ADDR + " -swapInstance " + clusterName + " localhost_1232 localhost_12320";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
@@ -506,6 +502,14 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     Assert.assertFalse(_gZkClient.exists(path), path
         + " should not exist since localhost_1232 has been swapped by localhost_12320");
 
+    // clean up
+    for (int i = 0; i < controllers.length; i++) {
+      controllers[i].syncStop();
+    }
+    for (int i = 0; i < participants.length; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -519,26 +523,26 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
-    ClusterController[] controllers = new ClusterController[2];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
-        "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true";
+        "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     Thread.sleep(500);
 
     command =
-        "-zkSvr localhost:2183 -addNode " + clusterName
+        "-zkSvr " + ZK_ADDR + " -addNode " + clusterName
             + " localhost:12331;localhost:12341;localhost:12351;localhost:12361";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     command = "-zkSvr localhost:2183 -expandCluster " + clusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
-    MockParticipant[] newParticipants = new MockParticipant[4];
+    MockParticipantManager[] newParticipants = new MockParticipantManager[4];
     for (int i = 3; i <= 6; i++) {
       String instanceName = "localhost_123" + i + "1";
-      newParticipants[i - 3] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+      newParticipants[i - 3] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       newParticipants[i - 3].syncStart();
     }
 
@@ -552,6 +556,17 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(verifyResult);
 
+    // clean up
+    for (int i = 0; i < controllers.length; i++) {
+      controllers[i].syncStop();
+    }
+    for (int i = 0; i < participants.length; i++) {
+      participants[i].syncStop();
+    }
+    for (int i = 0; i < newParticipants.length; i++) {
+      newParticipants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -565,17 +580,18 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
-    ClusterController[] controllers = new ClusterController[2];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[2];
     setupCluster(clusterName, grandClusterName, n, participants, controllers);
     String command =
-        "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " true";
+        "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName + " true";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     Thread.sleep(500);
 
     // deactivate cluster
     command =
-        "-zkSvr localhost:2183 -activateCluster " + clusterName + " " + grandClusterName + " false";
+        "-zkSvr " + ZK_ADDR + " -activateCluster " + clusterName + " " + grandClusterName
+            + " false";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
@@ -590,7 +606,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     Assert.assertFalse(_gZkClient.exists(path),
         "leader should be gone after deactivate the cluster");
 
-    command = "-zkSvr localhost:2183 -dropCluster " + clusterName;
+    command = "-zkSvr " + ZK_ADDR + " -dropCluster " + clusterName;
     try {
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
       Assert.fail("dropCluster should fail since there are still instances running");
@@ -598,16 +614,15 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
       // OK
     }
 
-    for (int i = 0; i < n; i++) {
+    for (int i = 0; i < participants.length; i++) {
       participants[i].syncStop();
     }
 
     command = "-zkSvr localhost:2183 -dropCluster " + clusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s"));
 
-    for (int i = 0; i < 2; i++) {
+    for (int i = 0; i < controllers.length; i++) {
       controllers[i].syncStop();
-      Thread.sleep(1000);
     }
     command = "-zkSvr localhost:2183 -dropCluster " + grandClusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
@@ -627,29 +642,30 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    String command = "-zkSvr localhost:2183 -addCluster " + clusterName;
+    String command = "-zkSvr " + ZK_ADDR + " -addCluster " + clusterName;
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 12 MasterSlave";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     for (int i = 0; i < 6; i++) {
-      command = "-zkSvr localhost:2183 -addNode " + clusterName + " localhost:123" + i;
+      command = "-zkSvr " + ZK_ADDR + " -addNode " + clusterName + " localhost:123" + i;
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     }
 
     for (int i = 0; i < 2; i++) {
       command =
-          "-zkSvr localhost:2183 -addInstanceTag " + clusterName + " localhost_123" + i + "  tag1";
+          "-zkSvr " + ZK_ADDR + " -addInstanceTag " + clusterName + " localhost_123" + i + "  tag1";
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     }
     for (int i = 2; i < 6; i++) {
       command =
-          "-zkSvr localhost:2183 -addInstanceTag " + clusterName + " localhost_123" + i + "  tag2";
+          "-zkSvr " + ZK_ADDR + " -addInstanceTag " + clusterName + " localhost_123" + i + "  tag2";
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     }
 
-    command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -instanceGroupTag tag1";
+    command =
+        "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -instanceGroupTag tag1";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     IdealState dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11"));
@@ -664,14 +680,15 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     }
     Assert.assertEquals(hosts.size(), 2);
 
-    command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 ";
+    command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 ";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     // re-add and rebalance
-    command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave";
+    command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
-    command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2";
+    command =
+        "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11"));
@@ -686,21 +703,22 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     }
     Assert.assertEquals(hosts.size(), 4);
 
-    command = "-zkSvr localhost:2183 -dropResource " + clusterName + " db_11 ";
+    command = "-zkSvr " + ZK_ADDR + " -dropResource " + clusterName + " db_11 ";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     for (int i = 3; i <= 3; i++) {
       command =
-          "-zkSvr localhost:2183 -removeInstanceTag " + clusterName + " localhost_123" + i
+          "-zkSvr " + ZK_ADDR + " -removeInstanceTag " + clusterName + " localhost_123" + i
               + " tag2";
       ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     }
 
     // re-add and rebalance
-    command = "-zkSvr localhost:2183 -addResource " + clusterName + " db_11 48 MasterSlave";
+    command = "-zkSvr " + ZK_ADDR + " -addResource " + clusterName + " db_11 48 MasterSlave";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
-    command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2";
+    command =
+        "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 3 -instanceGroupTag tag2";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     dbIs = accessor.getProperty(accessor.keyBuilder().idealStates("db_11"));
     hosts = new HashSet<ParticipantId>();
@@ -715,7 +733,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     Assert.assertEquals(hosts.size(), 3);
 
     // rebalance with key prefix
-    command = "-zkSvr localhost:2183 -rebalance " + clusterName + " db_11 2 -key alias";
+    command = "-zkSvr " + ZK_ADDR + " -rebalance " + clusterName + " db_11 2 -key alias";
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java
----------------------------------------------------------------------
diff --git a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java
index ceba1ed..308ae14 100644
--- a/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java
+++ b/recipes/user-defined-rebalancer/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java
@@ -45,4 +45,9 @@ public class Lock extends StateModel {
     System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName);
   }
 
+  @Transition(from = "*", to = "DROPPED")
+  public void drop(Message m, NotificationContext context) {
+    System.out.println(context.getManager().getInstanceName() + " dropping lock:" + lockName);
+  }
+
 }


[03/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
index 8558b18..30f5807 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
@@ -19,6 +19,7 @@ package org.apache.helix.integration;
  * under the License.
  */
 
+import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -28,20 +29,19 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.CountDownLatch;
 
 import org.apache.helix.Criteria;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.HelixProperty;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.id.MessageId;
-import org.apache.helix.api.id.SessionId;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;
@@ -52,8 +52,13 @@ import org.apache.helix.model.ConstraintItem;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.StatusUpdate;
 import org.apache.helix.monitoring.ZKPathDataDumpTask;
 import org.apache.helix.util.HelixUtil;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.JsonGenerationException;
+import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
 import org.testng.Assert;
@@ -70,6 +75,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     @Override
     public void onTimeOut() {
       // TODO Auto-generated method stub
+
     }
 
     @Override
@@ -78,20 +84,13 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     }
   }
 
-  TestMessagingHandlerFactory _factory = new TestMessagingHandlerFactory();
+  final TestMessagingHandlerFactory _factory = new TestMessagingHandlerFactory();
 
   public static class TestMessagingHandlerFactory implements MessageHandlerFactory {
-    int cnt;
     public Map<String, Set<String>> _results = new ConcurrentHashMap<String, Set<String>>();
 
-    public TestMessagingHandlerFactory() {
-      super();
-      cnt = 0;
-    }
-
     @Override
     public MessageHandler createHandler(Message message, NotificationContext context) {
-      // System.out.println("\t create-hdlr: " + message.getId());
       return new TestMessagingHandler(message, context);
     }
 
@@ -116,20 +115,73 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
       public HelixTaskResult handleMessage() throws InterruptedException {
         HelixTaskResult result = new HelixTaskResult();
         result.setSuccess(true);
-        // String tgtName = _message.getTgtName();
-        String messageId = _message.getMessageId().stringify();
-        String partitionId = _message.getPartitionId().stringify();
+        String destName = _message.getTgtName();
+        String partitionName = _message.getPartitionName();
+        result.getTaskResultMap().put("Message", _message.getMsgId());
+        synchronized (_results) {
+          if (!_results.containsKey(partitionName)) {
+            _results.put(partitionName, new HashSet<String>());
+          }
+          _results.get(partitionName).add(_message.getMsgId());
+        }
+        // System.err.println("handle msg: " + _message.getPartitionName() + ", from: "
+        // + _message.getFromState() + ", to: " + _message.getToState());
+        return result;
+      }
 
-        result.getTaskResultMap().put("Message", messageId);
+      @Override
+      public void onError(Exception e, ErrorCode code, ErrorType type) {
+        // TODO Auto-generated method stub
+      }
+    }
+  }
+
+  public static class TestMessagingHandlerFactoryLatch implements MessageHandlerFactory {
+    public volatile CountDownLatch _latch = new CountDownLatch(1);
+    public int _messageCount = 0;
+    public Map<String, Set<String>> _results = new ConcurrentHashMap<String, Set<String>>();
+
+    @Override
+    public synchronized MessageHandler createHandler(Message message, NotificationContext context) {
+      _messageCount++;
+      return new TestMessagingHandlerLatch(message, context);
+    }
+
+    public synchronized void signal() {
+      _latch.countDown();
+      _latch = new CountDownLatch(1);
+    }
+
+    @Override
+    public String getMessageType() {
+      return "TestMessagingHandlerLatch";
+    }
+
+    @Override
+    public void reset() {
+      // TODO Auto-generated method stub
+    }
+
+    public class TestMessagingHandlerLatch extends MessageHandler {
+      public TestMessagingHandlerLatch(Message message, NotificationContext context) {
+        super(message, context);
+        // TODO Auto-generated constructor stub
+      }
+
+      @Override
+      public HelixTaskResult handleMessage() throws InterruptedException {
+        _latch.await();
+        HelixTaskResult result = new HelixTaskResult();
+        result.setSuccess(true);
+        result.getTaskResultMap().put("Message", _message.getMsgId());
+        String destName = _message.getTgtName();
         synchronized (_results) {
-          if (!_results.containsKey(partitionId)) {
-            _results.put(partitionId, new HashSet<String>());
+          if (!_results.containsKey(_message.getPartitionName())) {
+            _results.put(_message.getPartitionName(), new ConcurrentSkipListSet<String>());
           }
-          _results.get(partitionId).add(messageId);
         }
-        cnt++;
-        // System.err.println(cnt + ": message " + messageId + ", tgtName: " + tgtName
-        // + ", partitionId: " + partitionId);
+        _results.get(_message.getPartitionName()).add(destName);
+        // System.err.println("Message " + _message.getMsgId() + " executed");
         return result;
       }
 
@@ -141,28 +193,116 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
   }
 
   @Test()
+  public void testSchedulerMsgUsingQueue() throws Exception {
+    Logger.getRootLogger().setLevel(Level.INFO);
+    _factory._results.clear();
+    HelixManager manager = null;
+    for (int i = 0; i < NODE_NR; i++) {
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
+          _factory.getMessageType(), _factory);
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
+    }
+
+    Message schedulerMessage =
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
+    schedulerMessage.setTgtName("CONTROLLER");
+    // TODO: change it to "ADMIN" ?
+    schedulerMessage.setSrcName("CONTROLLER");
+    schedulerMessage.getRecord().setSimpleField(
+        DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, "TestSchedulerMsg");
+    // Template for the individual message sent to each participant
+    Message msg = new Message(_factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
+    msg.setMsgState(MessageState.NEW);
+
+    // Criteria to send individual messages
+    Criteria cr = new Criteria();
+    cr.setInstanceName("localhost_%");
+    cr.setRecipientInstanceType(InstanceType.PARTICIPANT);
+    cr.setSessionSpecific(false);
+    cr.setResource("%");
+    cr.setPartition("%");
+
+    ObjectMapper mapper = new ObjectMapper();
+    SerializationConfig serializationConfig = mapper.getSerializationConfig();
+    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
+
+    StringWriter sw = new StringWriter();
+    mapper.writeValue(sw, cr);
+
+    String crString = sw.toString();
+
+    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
+    schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields());
+    schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1");
+
+    HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
+    Builder keyBuilder = helixDataAccessor.keyBuilder();
+    helixDataAccessor.createProperty(keyBuilder.controllerMessage(schedulerMessage.getMsgId()),
+        schedulerMessage);
+
+    for (int i = 0; i < 30; i++) {
+      Thread.sleep(2000);
+      if (_PARTITIONS == _factory._results.size()) {
+        break;
+      }
+    }
+
+    Assert.assertEquals(_PARTITIONS, _factory._results.size());
+    PropertyKey controllerTaskStatus =
+        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
+            schedulerMessage.getMsgId());
+
+    int messageResultCount = 0;
+    for (int i = 0; i < 10; i++) {
+      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+      Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount")
+          .equals("" + (_PARTITIONS * 3)));
+      for (String key : statusUpdate.getMapFields().keySet()) {
+        if (key.startsWith("MessageResult ")) {
+          messageResultCount++;
+        }
+      }
+      if (messageResultCount == _PARTITIONS * 3) {
+        break;
+      } else {
+        Thread.sleep(2000);
+      }
+    }
+    Assert.assertEquals(messageResultCount, _PARTITIONS * 3);
+    int count = 0;
+    for (Set<String> val : _factory._results.values()) {
+      count += val.size();
+    }
+    Assert.assertEquals(count, _PARTITIONS * 3);
+
+  }
+
+  @Test()
   public void testSchedulerMsg() throws Exception {
-    // Logger.getRootLogger().setLevel(Level.INFO);
+    Logger.getRootLogger().setLevel(Level.INFO);
     _factory._results.clear();
     HelixManager manager = null;
     for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           _factory.getMessageType(), _factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
     }
 
     Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
     schedulerMessage.setTgtName("CONTROLLER");
     // TODO: change it to "ADMIN" ?
     schedulerMessage.setSrcName("CONTROLLER");
     // schedulerMessage.getRecord().setSimpleField(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE,
     // "TestSchedulerMsg");
     // Template for the individual message sent to each participant
-    Message msg = new Message(_factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
+    Message msg = new Message(_factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
     msg.setMsgState(MessageState.NEW);
 
     // Criteria to send individual messages
@@ -188,8 +328,8 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     Builder keyBuilder = helixDataAccessor.keyBuilder();
-    helixDataAccessor.createProperty(
-        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify()), schedulerMessage);
+    helixDataAccessor.createProperty(keyBuilder.controllerMessage(schedulerMessage.getMsgId()),
+        schedulerMessage);
 
     for (int i = 0; i < 30; i++) {
       Thread.sleep(2000);
@@ -200,8 +340,8 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
     Assert.assertEquals(_PARTITIONS, _factory._results.size());
     PropertyKey controllerTaskStatus =
-        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMessageId().stringify());
+        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
+            schedulerMessage.getMsgId());
 
     int messageResultCount = 0;
     for (int i = 0; i < 10; i++) {
@@ -231,11 +371,11 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     String controllerStatusPath =
         HelixUtil.getControllerPropertyPath(manager.getClusterName(),
             PropertyType.STATUSUPDATES_CONTROLLER);
-    List<String> subPaths = _zkClient.getChildren(controllerStatusPath);
+    List<String> subPaths = _gZkClient.getChildren(controllerStatusPath);
     Assert.assertTrue(subPaths.size() > 0);
     for (String subPath : subPaths) {
       String nextPath = controllerStatusPath + "/" + subPath;
-      List<String> subsubPaths = _zkClient.getChildren(nextPath);
+      List<String> subsubPaths = _gZkClient.getChildren(nextPath);
       Assert.assertTrue(subsubPaths.size() > 0);
     }
 
@@ -243,38 +383,38 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
         HelixUtil.getInstancePropertyPath(manager.getClusterName(), "localhost_" + (START_PORT),
             PropertyType.STATUSUPDATES);
 
-    subPaths = _zkClient.getChildren(instanceStatusPath);
+    subPaths = _gZkClient.getChildren(instanceStatusPath);
     Assert.assertTrue(subPaths.size() > 0);
     for (String subPath : subPaths) {
       String nextPath = instanceStatusPath + "/" + subPath;
-      List<String> subsubPaths = _zkClient.getChildren(nextPath);
+      List<String> subsubPaths = _gZkClient.getChildren(nextPath);
       Assert.assertTrue(subsubPaths.size() > 0);
       for (String subsubPath : subsubPaths) {
         String nextnextPath = nextPath + "/" + subsubPath;
-        Assert.assertTrue(_zkClient.getChildren(nextnextPath).size() > 0);
+        Assert.assertTrue(_gZkClient.getChildren(nextnextPath).size() > 0);
       }
     }
     Thread.sleep(3000);
-    ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, _zkClient, 0);
+    ZKPathDataDumpTask dumpTask = new ZKPathDataDumpTask(manager, _gZkClient, 0);
     dumpTask.run();
 
-    subPaths = _zkClient.getChildren(controllerStatusPath);
+    subPaths = _gZkClient.getChildren(controllerStatusPath);
     Assert.assertTrue(subPaths.size() > 0);
     for (String subPath : subPaths) {
       String nextPath = controllerStatusPath + "/" + subPath;
-      List<String> subsubPaths = _zkClient.getChildren(nextPath);
+      List<String> subsubPaths = _gZkClient.getChildren(nextPath);
       Assert.assertTrue(subsubPaths.size() == 0);
     }
 
-    subPaths = _zkClient.getChildren(instanceStatusPath);
+    subPaths = _gZkClient.getChildren(instanceStatusPath);
     Assert.assertTrue(subPaths.size() > 0);
     for (String subPath : subPaths) {
       String nextPath = instanceStatusPath + "/" + subPath;
-      List<String> subsubPaths = _zkClient.getChildren(nextPath);
+      List<String> subsubPaths = _gZkClient.getChildren(nextPath);
       Assert.assertTrue(subsubPaths.size() > 0);
       for (String subsubPath : subsubPaths) {
         String nextnextPath = nextPath + "/" + subsubPath;
-        Assert.assertTrue(_zkClient.getChildren(nextnextPath).size() == 0);
+        Assert.assertTrue(_gZkClient.getChildren(nextnextPath).size() == 0);
       }
     }
   }
@@ -284,22 +424,22 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     _factory._results.clear();
     HelixManager manager = null;
     for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           _factory.getMessageType(), _factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
     }
 
     Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
     schedulerMessage.setTgtName("CONTROLLER");
     // TODO: change it to "ADMIN" ?
     schedulerMessage.setSrcName("CONTROLLER");
 
     // Template for the individual message sent to each participant
-    Message msg = new Message(_factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
+    Message msg = new Message(_factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
     msg.setMsgState(MessageState.NEW);
 
     // Criteria to send individual messages
@@ -375,22 +515,22 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
     HelixManager manager = null;
     for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
     }
 
     Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
     schedulerMessage.setTgtName("CONTROLLER");
     // TODO: change it to "ADMIN" ?
     schedulerMessage.setSrcName("CONTROLLER");
 
     // Template for the individual message sent to each participant
-    Message msg = new Message(factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
+    Message msg = new Message(factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
     msg.setMsgState(MessageState.NEW);
 
     // Criteria to send individual messages
@@ -416,21 +556,21 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     Builder keyBuilder = helixDataAccessor.keyBuilder();
-    PropertyKey controllerMessageKey =
-        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify());
+    PropertyKey controllerMessageKey = keyBuilder.controllerMessage(schedulerMessage.getMsgId());
     helixDataAccessor.setProperty(controllerMessageKey, schedulerMessage);
 
     Thread.sleep(3000);
 
     Assert.assertEquals(0, factory._results.size());
-
-    waitMessageUpdate("SentMessageCount", schedulerMessage.getMessageId().stringify(),
-        helixDataAccessor);
     PropertyKey controllerTaskStatus =
-        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMessageId().stringify());
-    waitMessageUpdate("SentMessageCount", schedulerMessage.getMessageId().stringify(),
-        helixDataAccessor);
+        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
+            schedulerMessage.getMsgId());
+    for (int i = 0; i < 10; i++) {
+      StatusUpdate update = helixDataAccessor.getProperty(controllerTaskStatus);
+      if (update == null || update.getRecord().getMapField("SentMessageCount") == null) {
+        Thread.sleep(1000);
+      }
+    }
     ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
     Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount").equals("0"));
     int count = 0;
@@ -442,30 +582,28 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
   @Test()
   public void testSchedulerMsg3() throws Exception {
-    final int avgReplicas = _PARTITIONS * 3 / 5;
-
     _factory._results.clear();
     HelixManager manager = null;
     for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           _factory.getMessageType(), _factory);
 
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-          _factory.getMessageType(), _factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
+      // _participants[i].getMessagingService().registerMessageHandlerFactory(
+      // _factory.getMessageType(), _factory);
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
     }
 
     Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
     schedulerMessage.setTgtName("CONTROLLER");
     // TODO: change it to "ADMIN" ?
     schedulerMessage.setSrcName("CONTROLLER");
 
     // Template for the individual message sent to each participant
-    Message msg = new Message(_factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
+    Message msg = new Message(_factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
     msg.setMsgState(MessageState.NEW);
 
     // Criteria to send individual messages
@@ -518,7 +656,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
       sw = new StringWriter();
       mapper.writeValue(sw, cr);
-      schedulerMessage.setMessageId(MessageId.from(UUID.randomUUID().toString()));
+      schedulerMessage.setMsgId(UUID.randomUUID().toString());
       crString = sw.toString();
       schedulerMessage.getRecord().setSimpleField("Criteria", crString);
       manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1);
@@ -526,35 +664,59 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
           callback._message.getResultMap().get(
               DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID);
 
-      Thread.sleep(1000);
       HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
       Builder keyBuilder = helixDataAccessor.keyBuilder();
 
-      waitMessageUpdate("Summary", msgId, helixDataAccessor);
+      for (int j = 0; j < 100; j++) {
+        Thread.sleep(200);
+        PropertyKey controllerTaskStatus =
+            keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
+        ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+        if (statusUpdate.getMapFields().containsKey("Summary")) {
+          break;
+        }
+      }
 
       PropertyKey controllerTaskStatus =
           keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
       ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
       Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount")
-          .equals("" + avgReplicas));
+          .equals("" + (_PARTITIONS * 3 / 5)));
       int messageResultCount = 0;
       for (String key : statusUpdate.getMapFields().keySet()) {
         if (key.startsWith("MessageResult")) {
           messageResultCount++;
         }
       }
-      Assert.assertEquals(messageResultCount, avgReplicas);
+      Assert.assertEquals(messageResultCount, _PARTITIONS * 3 / 5);
 
-      int count = 0;
-      // System.out.println(i);
-      for (Set<String> val : _factory._results.values()) {
-        // System.out.println(val);
-        count += val.size();
+
+      boolean success = false;
+      for (int j = 0; j < 6; j++) {
+        int count = 0;
+        // System.out.println(i);
+        for (Set<String> val : _factory._results.values()) {
+          // System.out.println(val);
+          count += val.size();
+        }
+        // System.out.println(count);
+        // Assert.assertEquals(count, _PARTITIONS * 3 / 5 * (i + 1));
+        success = count == _PARTITIONS * 3 / 5 * (i + 1);
+        if (success) {
+          break;
+        }
+        Thread.sleep(500);
       }
-      // System.out.println(count);
+      Assert.assertTrue(success);
+    }
+  }
 
-      Assert.assertEquals(count, avgReplicas * (i + 1));
+  private int count(TestMessagingHandlerFactory factory) {
+    int cnt = 0;
+    for (Set<String> val : factory._results.values()) {
+      cnt += val.size();
     }
+    return cnt;
   }
 
   @Test()
@@ -562,25 +724,26 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     _factory._results.clear();
     HelixManager manager = null;
     for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           _factory.getMessageType(), _factory);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-          _factory.getMessageType(), _factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
+
+      // _participants[i].getMessagingService().registerMessageHandlerFactory(
+      // _factory.getMessageType(), _factory);
+
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
     }
 
     Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
     schedulerMessage.setTgtName("CONTROLLER");
     // TODO: change it to "ADMIN" ?
     schedulerMessage.setSrcName("CONTROLLER");
 
     // Template for the individual message sent to each participant
-    Message msg = new Message(_factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
-    msg.setMsgState(MessageState.NEW);
+    Message msgTemplate = new Message(_factory.getMessageType(), "Template");
+    msgTemplate.setTgtSessionId("*");
+    msgTemplate.setMsgState(MessageState.NEW);
 
     // Criteria to send individual messages
     Criteria cr = new Criteria();
@@ -600,7 +763,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     String crString = sw.toString();
 
     schedulerMessage.getRecord().setSimpleField("Criteria", crString);
-    schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields());
+    schedulerMessage.getRecord().setMapField("MessageTemplate", msgTemplate.getRecord().getSimpleFields());
     schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1");
     schedulerMessage.getRecord().setSimpleField("WAIT_ALL", "true");
 
@@ -635,8 +798,17 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
         callback._message.getResultMap()
             .get(DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID);
 
-    final HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
-    final Builder keyBuilder = helixDataAccessor.keyBuilder();
+    boolean success = TestHelper.verify(new TestHelper.Verifier() {
+
+      @Override
+      public boolean verify() throws Exception {
+        return count(_factory) == 60; // TestDB number_of_partitions x replicas
+      }
+    }, 10 * 1000);
+    Assert.assertTrue(success, "If not specifying participant, controller will send 60 messages");
+
+    HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
+    Builder keyBuilder = helixDataAccessor.keyBuilder();
     ArrayList<String> msgIds = new ArrayList<String>();
     for (int i = 0; i < NODE_NR; i++) {
       callback = new MockAsyncCallback();
@@ -647,25 +819,34 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
       sw = new StringWriter();
       mapper.writeValue(sw, cr);
-      schedulerMessage.setMessageId(MessageId.from(UUID.randomUUID().toString()));
-
-      // need to use a different name for scheduler_task_queue task resource
-      schedulerMessage.getRecord().setSimpleField(
-          DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, "TestSchedulerMsg4_" + i);
-
+      schedulerMessage.setMsgId(UUID.randomUUID().toString());
       crString = sw.toString();
       schedulerMessage.getRecord().setSimpleField("Criteria", crString);
       manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1);
+
+      Thread.sleep(5000);
+      System.err.println("count: " + count(_factory));
+
       String msgId =
           callback._message.getResultMap().get(
               DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID);
       msgIds.add(msgId);
     }
 
-    for (int i = 0; i < NODE_NR; i++) {
-      final String msgId = msgIds.get(i);
+    // System.err.println("count: " + count(_factory));
 
-      waitMessageUpdate("Summary", msgId, helixDataAccessor);
+    for (int i = 0; i < NODE_NR; i++) {
+      String msgId = msgIds.get(i);
+      for (int j = 0; j < 100; j++) {
+        Thread.sleep(200);
+        PropertyKey controllerTaskStatus =
+            keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
+        ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+        if (statusUpdate.getMapFields().containsKey("Summary")) {
+          // System.err.println(msgId+" done");
+          break;
+        }
+      }
 
       PropertyKey controllerTaskStatus =
           keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
@@ -678,11 +859,24 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
           messageResultCount++;
         }
       }
+      if (messageResultCount != _PARTITIONS * 3 / 5) {
+        int x = 10;
+        x = x + messageResultCount;
+      }
       Assert.assertEquals(messageResultCount, _PARTITIONS * 3 / 5);
     }
 
-    waitMessageUpdate("Summary", msgIdPrime, helixDataAccessor);
+    for (int j = 0; j < 100; j++) {
+      Thread.sleep(200);
+      PropertyKey controllerTaskStatus =
+          keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgIdPrime);
+      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+      if (statusUpdate.getMapFields().containsKey("Summary")) {
+        break;
+      }
+    }
 
+    // Thread.sleep(5000);
     int count = 0;
     for (Set<String> val : _factory._results.values()) {
       // System.out.println(val);
@@ -692,31 +886,147 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     Assert.assertEquals(count, _PARTITIONS * 3 * 2);
   }
 
-  /**
-   * wait message summary to appear in controller-message-status-update
-   * @param msgId
-   * @param accessor
-   * @return
-   * @throws Exception
-   */
-  private boolean waitMessageUpdate(final String mapKey, final String msgId,
-      final HelixDataAccessor accessor) throws Exception {
-    final PropertyKey.Builder keyBuilder = accessor.keyBuilder();
-    return TestHelper.verify(new TestHelper.Verifier() {
+  @Test
+  public void testSchedulerMsgContraints() throws JsonGenerationException, JsonMappingException,
+      IOException, InterruptedException {
+    TestMessagingHandlerFactoryLatch factory = new TestMessagingHandlerFactoryLatch();
+    HelixManager manager = null;
+    for (int i = 0; i < NODE_NR; i++) {
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
+          factory.getMessageType(), factory);
 
-      @Override
-      public boolean verify() throws Exception {
-        PropertyKey key = keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.name(), msgId);
-        HelixProperty statusUpdate = accessor.getProperty(key);
-        if (statusUpdate == null) {
-          return false;
-        }
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
+          factory.getMessageType(), factory);
 
-        if (statusUpdate.getRecord().getMapField(mapKey) == null) {
-          return false;
-        }
-        return true;
+      manager = _participants[i]; // _startCMResultMap.get(hostDest)._manager;
+    }
+
+    Message schedulerMessage =
+        new Message(MessageType.SCHEDULER_MSG + "", UUID.randomUUID().toString());
+    schedulerMessage.setTgtSessionId("*");
+    schedulerMessage.setTgtName("CONTROLLER");
+    // TODO: change it to "ADMIN" ?
+    schedulerMessage.setSrcName("CONTROLLER");
+
+    // Template for the individual message sent to each participant
+    Message msg = new Message(factory.getMessageType(), "Template");
+    msg.setTgtSessionId("*");
+    msg.setMsgState(MessageState.NEW);
+
+    // Criteria to send individual messages
+    Criteria cr = new Criteria();
+    cr.setInstanceName("localhost_%");
+    cr.setRecipientInstanceType(InstanceType.PARTICIPANT);
+    cr.setSessionSpecific(false);
+    cr.setResource("%");
+    cr.setPartition("%");
+
+    ObjectMapper mapper = new ObjectMapper();
+    SerializationConfig serializationConfig = mapper.getSerializationConfig();
+    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
+
+    StringWriter sw = new StringWriter();
+    mapper.writeValue(sw, cr);
+
+    String crString = sw.toString();
+
+    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
+    schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields());
+    schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1");
+    schedulerMessage.getRecord().setSimpleField("WAIT_ALL", "true");
+    schedulerMessage.getRecord().setSimpleField(
+        DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, "TestSchedulerMsgContraints");
+
+    Criteria cr2 = new Criteria();
+    cr2.setRecipientInstanceType(InstanceType.CONTROLLER);
+    cr2.setInstanceName("*");
+    cr2.setSessionSpecific(false);
+
+    MockAsyncCallback callback = new MockAsyncCallback();
+    mapper = new ObjectMapper();
+    serializationConfig = mapper.getSerializationConfig();
+    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
+
+    sw = new StringWriter();
+    mapper.writeValue(sw, cr);
+
+    HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
+    Builder keyBuilder = helixDataAccessor.keyBuilder();
+
+    // Set contraints that only 1 msg per participant
+    Map<String, String> constraints = new TreeMap<String, String>();
+    constraints.put("MESSAGE_TYPE", "STATE_TRANSITION");
+    constraints.put("TRANSITION", "OFFLINE-COMPLETED");
+    constraints.put("CONSTRAINT_VALUE", "1");
+    constraints.put("INSTANCE", ".*");
+    manager.getClusterManagmentTool().setConstraint(manager.getClusterName(),
+        ConstraintType.MESSAGE_CONSTRAINT, "constraint1", new ConstraintItem(constraints));
+
+    // Send scheduler message
+    crString = sw.toString();
+    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
+    manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1);
+    String msgId =
+        callback._message.getResultMap()
+            .get(DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID);
+
+    for (int j = 0; j < 10; j++) {
+      Thread.sleep(200);
+      PropertyKey controllerTaskStatus =
+          keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
+      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+      if (statusUpdate.getMapFields().containsKey("SentMessageCount")) {
+        Assert.assertEquals(
+            statusUpdate.getMapFields().get("SentMessageCount").get("MessageCount"), ""
+                + (_PARTITIONS * 3));
+        break;
       }
-    }, 20 * 1000);
+    }
+
+    for (int i = 0; i < _PARTITIONS * 3 / 5; i++) {
+      for (int j = 0; j < 10; j++) {
+        Thread.sleep(300);
+        if (factory._messageCount == 5 * (i + 1))
+          break;
+      }
+      Thread.sleep(300);
+      Assert.assertEquals(factory._messageCount, 5 * (i + 1));
+      factory.signal();
+      // System.err.println(i);
+    }
+
+    for (int j = 0; j < 10; j++) {
+      Thread.sleep(200);
+      PropertyKey controllerTaskStatus =
+          keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
+      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+      if (statusUpdate.getMapFields().containsKey("Summary")) {
+        break;
+      }
+    }
+
+    Assert.assertEquals(_PARTITIONS, factory._results.size());
+    PropertyKey controllerTaskStatus =
+        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
+    ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
+    Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount")
+        .equals("" + (_PARTITIONS * 3)));
+    int messageResultCount = 0;
+    for (String key : statusUpdate.getMapFields().keySet()) {
+      if (key.startsWith("MessageResult ")) {
+        messageResultCount++;
+      }
+    }
+    Assert.assertEquals(messageResultCount, _PARTITIONS * 3);
+
+    int count = 0;
+    for (Set<String> val : factory._results.values()) {
+      count += val.size();
+    }
+    Assert.assertEquals(count, _PARTITIONS * 3);
+
+    manager.getClusterManagmentTool().removeConstraint(manager.getClusterName(),
+        ConstraintType.MESSAGE_CONSTRAINT, "constraint1");
+
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
deleted file mode 100644
index b887fe7..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
+++ /dev/null
@@ -1,254 +0,0 @@
-package org.apache.helix.integration;
-
-import java.io.StringWriter;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-import java.util.concurrent.CountDownLatch;
-
-import org.apache.helix.Criteria;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.InstanceType;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.api.id.MessageId;
-import org.apache.helix.api.id.SessionId;
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
-import org.apache.helix.messaging.AsyncCallback;
-import org.apache.helix.messaging.handling.HelixTaskResult;
-import org.apache.helix.messaging.handling.MessageHandler;
-import org.apache.helix.messaging.handling.MessageHandlerFactory;
-import org.apache.helix.model.ClusterConstraints.ConstraintType;
-import org.apache.helix.model.ConstraintItem;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageState;
-import org.apache.helix.model.Message.MessageType;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-public class TestSchedulerMsgContraints extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
-
-  class MockAsyncCallback extends AsyncCallback {
-    Message _message;
-
-    public MockAsyncCallback() {
-    }
-
-    @Override
-    public void onTimeOut() {
-      // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void onReplyMessage(Message message) {
-      _message = message;
-    }
-  }
-
-  public static class TestMessagingHandlerFactoryLatch implements MessageHandlerFactory {
-    public volatile CountDownLatch _latch = new CountDownLatch(1);
-    public int _messageCount = 0;
-    public Map<String, Set<String>> _results = new ConcurrentHashMap<String, Set<String>>();
-
-    @Override
-    public synchronized MessageHandler createHandler(Message message, NotificationContext context) {
-      _messageCount++;
-      return new TestMessagingHandlerLatch(message, context);
-    }
-
-    public synchronized void signal() {
-      _latch.countDown();
-      _latch = new CountDownLatch(1);
-    }
-
-    @Override
-    public String getMessageType() {
-      return "TestMessagingHandlerLatch";
-    }
-
-    @Override
-    public void reset() {
-      // TODO Auto-generated method stub
-    }
-
-    public class TestMessagingHandlerLatch extends MessageHandler {
-      public TestMessagingHandlerLatch(Message message, NotificationContext context) {
-        super(message, context);
-        // TODO Auto-generated constructor stub
-      }
-
-      @Override
-      public HelixTaskResult handleMessage() throws InterruptedException {
-        _latch.await();
-        HelixTaskResult result = new HelixTaskResult();
-        result.setSuccess(true);
-        result.getTaskResultMap().put("Message", _message.getMessageId().stringify());
-        String destName = _message.getTgtName();
-        synchronized (_results) {
-          if (!_results.containsKey(_message.getPartitionId().stringify())) {
-            _results
-                .put(_message.getPartitionId().stringify(), new ConcurrentSkipListSet<String>());
-          }
-        }
-        _results.get(_message.getPartitionId().stringify()).add(destName);
-        // System.err.println("Message " + _message.getMsgId() + " executed");
-        return result;
-      }
-
-      @Override
-      public void onError(Exception e, ErrorCode code, ErrorType type) {
-        // TODO Auto-generated method stub
-      }
-    }
-  }
-
-  @Test
-  public void testSchedulerMsgContraints() throws Exception {
-    TestMessagingHandlerFactoryLatch factory = new TestMessagingHandlerFactoryLatch();
-    HelixManager manager = null;
-    for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-          factory.getMessageType(), factory);
-      //
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-          factory.getMessageType(), factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
-    }
-
-    Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
-    schedulerMessage.setTgtName("CONTROLLER");
-    // TODO: change it to "ADMIN" ?
-    schedulerMessage.setSrcName("CONTROLLER");
-
-    // Template for the individual message sent to each participant
-    Message msg = new Message(factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
-    msg.setMsgState(MessageState.NEW);
-
-    // Criteria to send individual messages
-    Criteria cr = new Criteria();
-    cr.setInstanceName("localhost_%");
-    cr.setRecipientInstanceType(InstanceType.PARTICIPANT);
-    cr.setSessionSpecific(false);
-    cr.setResource("%");
-    cr.setPartition("%");
-
-    ObjectMapper mapper = new ObjectMapper();
-    SerializationConfig serializationConfig = mapper.getSerializationConfig();
-    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-
-    StringWriter sw = new StringWriter();
-    mapper.writeValue(sw, cr);
-
-    String crString = sw.toString();
-
-    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
-    schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields());
-    schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1");
-    schedulerMessage.getRecord().setSimpleField("WAIT_ALL", "true");
-    schedulerMessage.getRecord().setSimpleField(
-        DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, "TestSchedulerMsgContraints");
-
-    Criteria cr2 = new Criteria();
-    cr2.setRecipientInstanceType(InstanceType.CONTROLLER);
-    cr2.setInstanceName("*");
-    cr2.setSessionSpecific(false);
-
-    MockAsyncCallback callback = new MockAsyncCallback();
-    mapper = new ObjectMapper();
-    serializationConfig = mapper.getSerializationConfig();
-    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-
-    sw = new StringWriter();
-    mapper.writeValue(sw, cr);
-
-    HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
-    PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder();
-
-    // Set contraints that only 1 msg per participant
-    Map<String, String> constraints = new TreeMap<String, String>();
-    constraints.put("MESSAGE_TYPE", "STATE_TRANSITION");
-    constraints.put("TRANSITION", "OFFLINE-COMPLETED");
-    constraints.put("CONSTRAINT_VALUE", "1");
-    constraints.put("INSTANCE", ".*");
-    manager.getClusterManagmentTool().setConstraint(manager.getClusterName(),
-        ConstraintType.MESSAGE_CONSTRAINT, "constraint1", new ConstraintItem(constraints));
-
-    // Send scheduler message
-    crString = sw.toString();
-    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
-    manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1);
-    String msgId =
-        callback._message.getResultMap()
-            .get(DefaultSchedulerMessageHandlerFactory.SCHEDULER_MSG_ID);
-
-    for (int j = 0; j < 10; j++) {
-      Thread.sleep(200);
-      PropertyKey controllerTaskStatus =
-          keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
-      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
-      if (statusUpdate.getMapFields().containsKey("SentMessageCount")) {
-        Assert.assertEquals(
-            statusUpdate.getMapFields().get("SentMessageCount").get("MessageCount"), ""
-                + (_PARTITIONS * 3));
-        break;
-      }
-    }
-
-    for (int i = 0; i < _PARTITIONS * 3 / 5; i++) {
-      for (int j = 0; j < 10; j++) {
-        Thread.sleep(300);
-        if (factory._messageCount == 5 * (i + 1))
-          break;
-      }
-      Thread.sleep(300);
-      Assert.assertEquals(factory._messageCount, 5 * (i + 1));
-      factory.signal();
-      // System.err.println(i);
-    }
-
-    for (int j = 0; j < 10; j++) {
-      Thread.sleep(200);
-      PropertyKey controllerTaskStatus =
-          keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
-      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
-      if (statusUpdate.getMapFields().containsKey("Summary")) {
-        break;
-      }
-    }
-
-    Assert.assertEquals(_PARTITIONS, factory._results.size());
-    PropertyKey controllerTaskStatus =
-        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), msgId);
-    ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
-    Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount")
-        .equals("" + (_PARTITIONS * 3)));
-    int messageResultCount = 0;
-    for (String key : statusUpdate.getMapFields().keySet()) {
-      if (key.startsWith("MessageResult ")) {
-        messageResultCount++;
-      }
-    }
-    Assert.assertEquals(messageResultCount, _PARTITIONS * 3);
-
-    int count = 0;
-    for (Set<String> val : factory._results.values()) {
-      count += val.size();
-    }
-    Assert.assertEquals(count, _PARTITIONS * 3);
-
-    manager.getClusterManagmentTool().removeConstraint(manager.getClusterName(),
-        ConstraintType.MESSAGE_CONSTRAINT, "constraint1");
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
deleted file mode 100644
index bbaa18d..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
+++ /dev/null
@@ -1,181 +0,0 @@
-package org.apache.helix.integration;
-
-import java.io.StringWriter;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.helix.Criteria;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.InstanceType;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.api.id.MessageId;
-import org.apache.helix.api.id.SessionId;
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
-import org.apache.helix.messaging.handling.HelixTaskResult;
-import org.apache.helix.messaging.handling.MessageHandler;
-import org.apache.helix.messaging.handling.MessageHandlerFactory;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageState;
-import org.apache.helix.model.Message.MessageType;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-public class TestSchedulerMsgUsingQueue extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
-  public static class TestMessagingHandlerFactory implements MessageHandlerFactory {
-    int cnt;
-    public Map<String, Set<String>> _results = new ConcurrentHashMap<String, Set<String>>();
-
-    public TestMessagingHandlerFactory() {
-      super();
-      cnt = 0;
-    }
-
-    @Override
-    public MessageHandler createHandler(Message message, NotificationContext context) {
-      // System.out.println("\t create-hdlr: " + message.getId());
-      return new TestMessagingHandler(message, context);
-    }
-
-    @Override
-    public String getMessageType() {
-      return "TestParticipant";
-    }
-
-    @Override
-    public void reset() {
-      // TODO Auto-generated method stub
-
-    }
-
-    public class TestMessagingHandler extends MessageHandler {
-      public TestMessagingHandler(Message message, NotificationContext context) {
-        super(message, context);
-        // TODO Auto-generated constructor stub
-      }
-
-      @Override
-      public HelixTaskResult handleMessage() throws InterruptedException {
-        HelixTaskResult result = new HelixTaskResult();
-        result.setSuccess(true);
-        // String tgtName = _message.getTgtName();
-        String messageId = _message.getMessageId().stringify();
-        String partitionId = _message.getPartitionId().stringify();
-
-        result.getTaskResultMap().put("Message", messageId);
-        synchronized (_results) {
-          if (!_results.containsKey(partitionId)) {
-            _results.put(partitionId, new HashSet<String>());
-          }
-          _results.get(partitionId).add(messageId);
-        }
-        cnt++;
-        // System.err.println(cnt + ": message " + messageId + ", tgtName: " + tgtName
-        // + ", partitionId: " + partitionId);
-        return result;
-      }
-
-      @Override
-      public void onError(Exception e, ErrorCode code, ErrorType type) {
-        // TODO Auto-generated method stub
-      }
-    }
-  }
-
-  @Test()
-  public void testSchedulerMsgUsingQueue() throws Exception {
-    // Logger.getRootLogger().setLevel(Level.INFO);
-    // _factory._results.clear();
-    TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
-
-    HelixManager manager = null;
-    for (int i = 0; i < NODE_NR; i++) {
-      String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-          factory.getMessageType(), factory);
-      manager = _startCMResultMap.get(hostDest)._manager;
-    }
-
-    Message schedulerMessage =
-        new Message(MessageType.SCHEDULER_MSG + "", MessageId.from(UUID.randomUUID().toString()));
-    schedulerMessage.setTgtSessionId(SessionId.from("*"));
-    schedulerMessage.setTgtName("CONTROLLER");
-    // TODO: change it to "ADMIN" ?
-    schedulerMessage.setSrcName("CONTROLLER");
-    schedulerMessage.getRecord().setSimpleField(
-        DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, "TestSchedulerMsgUsingQueue");
-    // Template for the individual message sent to each participant
-    Message msg = new Message(factory.getMessageType(), MessageId.from("Template"));
-    msg.setTgtSessionId(SessionId.from("*"));
-    msg.setMsgState(MessageState.NEW);
-
-    // Criteria to send individual messages
-    Criteria cr = new Criteria();
-    cr.setInstanceName("localhost_%");
-    cr.setRecipientInstanceType(InstanceType.PARTICIPANT);
-    cr.setSessionSpecific(false);
-    cr.setResource("%");
-    cr.setPartition("%");
-
-    ObjectMapper mapper = new ObjectMapper();
-    SerializationConfig serializationConfig = mapper.getSerializationConfig();
-    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-
-    StringWriter sw = new StringWriter();
-    mapper.writeValue(sw, cr);
-
-    String crString = sw.toString();
-
-    schedulerMessage.getRecord().setSimpleField("Criteria", crString);
-    schedulerMessage.getRecord().setMapField("MessageTemplate", msg.getRecord().getSimpleFields());
-    schedulerMessage.getRecord().setSimpleField("TIMEOUT", "-1");
-
-    HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
-    PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder();
-    helixDataAccessor.createProperty(
-        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify()), schedulerMessage);
-
-    for (int i = 0; i < 30; i++) {
-      Thread.sleep(2000);
-      if (_PARTITIONS == factory._results.size()) {
-        break;
-      }
-    }
-
-    Assert.assertEquals(_PARTITIONS, factory._results.size());
-    PropertyKey controllerTaskStatus =
-        keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMessageId().stringify());
-
-    int messageResultCount = 0;
-    for (int i = 0; i < 10; i++) {
-      ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
-      Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount")
-          .equals("" + (_PARTITIONS * 3)));
-      for (String key : statusUpdate.getMapFields().keySet()) {
-        if (key.startsWith("MessageResult ")) {
-          messageResultCount++;
-        }
-      }
-      if (messageResultCount == _PARTITIONS * 3) {
-        break;
-      } else {
-        Thread.sleep(2000);
-      }
-    }
-    Assert.assertEquals(messageResultCount, _PARTITIONS * 3);
-    int count = 0;
-    for (Set<String> val : factory._results.values()) {
-      count += val.size();
-    }
-    Assert.assertEquals(count, _PARTITIONS * 3);
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 3024f45..a927520 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
@@ -27,10 +27,10 @@ import org.apache.helix.HelixConstants;
 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.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -46,7 +46,7 @@ public class TestSchemataSM extends ZkIntegrationTestBase {
     String clusterName = className + "_" + methodName;
     int n = 5;
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -70,14 +70,15 @@ public class TestSchemataSM extends ZkIntegrationTestBase {
         Arrays.asList(HelixConstants.StateModelToken.ANY_LIVEINSTANCE.toString()));
     accessor.setProperty(key, idealState);
 
-    ClusterController controller = new ClusterController(clusterName, "controller", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller");
     controller.syncStart();
 
     // start n-1 participants
     for (int i = 1; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -87,7 +88,7 @@ public class TestSchemataSM extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // start the remaining 1 participant
-    participants[0] = new MockParticipant(clusterName, "localhost_12918", ZK_ADDR, null);
+    participants[0] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918");
     participants[0].syncStart();
 
     // make sure we have all participants in MASTER state
@@ -107,6 +108,7 @@ public class TestSchemataSM extends ZkIntegrationTestBase {
     }
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 1b69572..4abb519 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
@@ -22,31 +22,29 @@ package org.apache.helix.integration;
 import java.util.Date;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
+  private static Logger LOG = Logger.getLogger(TestSessionExpiryInTransition.class);
 
   public class SessionExpiryTransition extends MockTransition {
     private final AtomicBoolean _done = new AtomicBoolean();
 
     @Override
     public void doTransition(Message message, NotificationContext context) {
-      ZkHelixTestManager manager = (ZkHelixTestManager) context.getManager();
+      MockParticipantManager manager = (MockParticipantManager) context.getManager();
 
       String instance = message.getTgtName();
       PartitionId partition = message.getPartitionId();
@@ -57,8 +55,7 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
         try {
           ZkTestHelper.expireSession(manager.getZkClient());
         } catch (Exception e) {
-          // TODO Auto-generated catch block
-          e.printStackTrace();
+          LOG.error("Exception expire zk-session", e);
         }
       }
     }
@@ -66,7 +63,7 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
 
   @Test
   public void testSessionExpiryInTransition() throws Exception {
-    Logger.getRootLogger().setLevel(Level.WARN);
+    // Logger.getRootLogger().setLevel(Level.WARN);
 
     String className = TestHelper.getTestClassName();
     String methodName = TestHelper.getTestMethodName();
@@ -74,7 +71,7 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -86,15 +83,15 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      ZkHelixTestManager manager =
-          new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR);
-      participants[i] = new MockParticipant(manager, new SessionExpiryTransition());
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new SessionExpiryTransition());
       participants[i].syncStart();
     }
 
@@ -104,13 +101,11 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
 
-    Thread.sleep(2000);
-    controller.syncStop();
-
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 02a34d8..6eb7a8c 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
@@ -21,9 +21,15 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.TestHelper.Verifier;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
@@ -35,16 +41,34 @@ public class TestStandAloneCMMain extends ZkStandAloneCMTestBase {
   @Test()
   public void testStandAloneCMMain() throws Exception {
     logger.info("RUN testStandAloneCMMain() at " + new Date(System.currentTimeMillis()));
-
+    ClusterControllerManager newController = null;
     for (int i = 1; i <= 2; i++) {
       String controllerName = "controller_" + i;
-      StartCMResult startResult =
-          TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-              HelixControllerMain.STANDALONE);
-      _startCMResultMap.put(controllerName, startResult);
+      newController =
+          new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+      newController.syncStart();
     }
 
-    stopCurrentLeader(_zkClient, CLUSTER_NAME, _startCMResultMap);
+    // stopCurrentLeader(_zkClient, CLUSTER_NAME, _startCMResultMap);
+    _controller.syncStop();
+
+    final HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    final PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+    final String newControllerName = newController.getInstanceName();
+    TestHelper.verify(new Verifier() {
+
+      @Override
+      public boolean verify() throws Exception {
+        LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader());
+        if (leader == null) {
+          return false;
+        }
+        return leader.getInstanceName().equals(newControllerName);
+
+      }
+    }, 30 * 1000);
+
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             ZK_ADDR, CLUSTER_NAME));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 81c08ed..dad998d 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
@@ -21,11 +21,10 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
@@ -47,18 +46,16 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase {
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, PARTICIPANT_PREFIX, "TestDB", 1, 20, 5, 3,
         "MasterSlave", true);
 
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      ZkHelixTestManager manager =
-          new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR);
-      participants[i] = new MockParticipant(manager, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
-    ZkHelixTestManager controller =
-        new ZkHelixTestManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR);
-    controller.connect();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     boolean result;
     result =
@@ -67,7 +64,7 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // participant session expiry
-    ZkHelixTestManager participantToExpire = participants[1].getManager();
+    MockParticipantManager participantToExpire = participants[1];
 
     System.out.println("Expire participant session");
     String oldSessionId = participantToExpire.getSessionId();
@@ -107,8 +104,7 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase {
     // clean up
     System.out.println("Clean up ...");
     // Logger.getRootLogger().setLevel(Level.DEBUG);
-    controller.disconnect();
-    Thread.sleep(100);
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 dce3fd4..d191c18 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,7 +25,7 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.mock.controller.ClusterController;
+import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -54,10 +54,10 @@ public class TestStartMultipleControllersWithSameName extends ZkIntegrationTestB
     // rebalance
 
     // start controller
-    ClusterController[] controllers = new ClusterController[4];
+    ClusterControllerManager[] controllers = new ClusterControllerManager[4];
     for (int i = 0; i < 4; i++) {
-      controllers[i] = new ClusterController(clusterName, "controller_0", ZK_ADDR);
-      controllers[i].start();
+      controllers[i] = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+      controllers[i].syncStart();
     }
 
     Thread.sleep(500); // wait leader election finishes
@@ -69,7 +69,6 @@ public class TestStartMultipleControllersWithSameName extends ZkIntegrationTestB
     // clean up
     for (int i = 0; i < 4; i++) {
       controllers[i].syncStop();
-      Thread.sleep(1000); // wait for all zk callbacks done
     }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 aff40b3..318177a 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
@@ -29,17 +29,13 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.api.State;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.messaging.handling.MessageHandler.ErrorCode;
 import org.apache.helix.mock.participant.MockMSStateModel;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.mock.participant.SleepTransition;
 import org.apache.helix.model.ExternalView;
@@ -60,15 +56,14 @@ import org.testng.annotations.Test;
 public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
   private static Logger LOG = Logger.getLogger(TestStateTransitionTimeout.class);
 
+  @Override
   @BeforeClass
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
     String namespace = "/" + CLUSTER_NAME;
-    if (_zkClient.exists(namespace)) {
-      _zkClient.deleteRecursive(namespace);
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
     }
     _setupTool = new ClusterSetup(ZK_ADDR);
 
@@ -107,12 +102,14 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
       _sleep = sleep;
     }
 
+    @Override
     @Transition(to = "SLAVE", from = "OFFLINE")
     public void onBecomeSlaveFromOffline(Message message, NotificationContext context) {
       LOG.info("Become SLAVE from OFFLINE");
 
     }
 
+    @Override
     @Transition(to = "MASTER", from = "SLAVE")
     public void onBecomeMasterFromSlave(Message message, NotificationContext context)
         throws InterruptedException {
@@ -122,23 +119,27 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
       }
     }
 
+    @Override
     @Transition(to = "SLAVE", from = "MASTER")
     public void onBecomeSlaveFromMaster(Message message, NotificationContext context) {
       LOG.info("Become SLAVE from MASTER");
     }
 
+    @Override
     @Transition(to = "OFFLINE", from = "SLAVE")
     public void onBecomeOfflineFromSlave(Message message, NotificationContext context) {
       LOG.info("Become OFFLINE from SLAVE");
 
     }
 
+    @Override
     @Transition(to = "DROPPED", from = "OFFLINE")
     public void onBecomeDroppedFromOffline(Message message, NotificationContext context) {
       LOG.info("Become DROPPED from OFFLINE");
 
     }
 
+    @Override
     public void rollbackOnError(Message message, NotificationContext context,
         StateTransitionError error) {
       _error = error;
@@ -172,7 +173,7 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
   @Test
   public void testStateTransitionTimeOut() throws Exception {
     Map<String, SleepStateModelFactory> factories = new HashMap<String, SleepStateModelFactory>();
-    MockParticipant[] participants = new MockParticipant[NODE_NR];
+    // MockParticipantManager[] participants = new MockParticipantManager[NODE_NR];
     IdealState idealState =
         _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
     for (int i = 0; i < NODE_NR; i++) {
@@ -185,19 +186,20 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
         }
       }
 
-      participants[i] = new MockParticipant(factory, CLUSTER_NAME, instanceName, ZK_ADDR, null);
-      participants[i].syncStart();
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", factory);
+      _participants[i].syncStart();
     }
     String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult startResult =
-        TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    _startCMResultMap.put(controllerName, startResult);
+    _controller =
+        new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
     boolean result =
         ClusterStateVerifier
             .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, CLUSTER_NAME));
     Assert.assertTrue(result);
-    HelixDataAccessor accessor = participants[0].getManager().getHelixDataAccessor();
+    HelixDataAccessor accessor = _participants[0].getHelixDataAccessor();
 
     Builder kb = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(kb.externalView(TEST_DB));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 b2a5719..6cce716 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
@@ -21,9 +21,8 @@ package org.apache.helix.integration;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -33,8 +32,7 @@ import org.testng.annotations.Test;
 public class TestSwapInstance extends ZkStandAloneCMTestBase {
   @Test
   public void TestSwap() throws Exception {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller;
     HelixDataAccessor helixAccessor = manager.getHelixDataAccessor();
     _setupTool.addResourceToCluster(CLUSTER_NAME, "MyDB", 64, STATE_MODEL);
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", _replica);
@@ -49,7 +47,7 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
     idealStateOld2.merge(is2.getRecord());
 
     String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient);
+    ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
     _setupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, instanceName, false);
 
     boolean result =
@@ -68,8 +66,7 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
     }
     Assert.assertTrue(exception);
 
-    _startCMResultMap.get(instanceName)._manager.disconnect();
-    _startCMResultMap.get(instanceName)._thread.interrupt();
+    _participants[0].syncStop();
     Thread.sleep(1000);
 
     exception = false;
@@ -80,8 +77,9 @@ public class TestSwapInstance extends ZkStandAloneCMTestBase {
       exception = true;
     }
     Assert.assertFalse(exception);
-    StartCMResult result2 = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName2);
-    _startCMResultMap.put(instanceName2, result2);
+    MockParticipantManager newParticipant =
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName2);
+    newParticipant.syncStart();
 
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(


[04/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 8cbe55b..0c7c131 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,13 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
 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.mock.participant.MockParticipant;
 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.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -49,8 +49,7 @@ public class TestMessageThrottle extends ZkIntegrationTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
 
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
-    // ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -114,13 +113,15 @@ public class TestMessageThrottle extends ZkIntegrationTestBase {
       });
     }
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -137,6 +138,7 @@ public class TestMessageThrottle extends ZkIntegrationTestBase {
     Assert.assertTrue(success.get());
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 a947445..a182753 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
@@ -32,7 +32,6 @@ import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
@@ -246,8 +245,7 @@ public class TestMessageThrottle2 extends ZkIntegrationTestBase {
 
     public void start() throws Exception {
       helixManager =
-          HelixManagerFactory.getZKHelixManager(clusterName, instanceName,
-              InstanceType.PARTICIPANT, ZK_ADDR);
+          new ZKHelixManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR);
       {
         // hack to set sessionTimeout
         Field sessionTimeout = ZKHelixManager.class.getDeclaredField("_sessionTimeout");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9c6b4b7..86f1ce4 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
@@ -89,8 +89,8 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     String hostDest = "localhost_" + (START_PORT + 1);
 
     TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
-    _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-        factory.getMessageType(), factory);
+    _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(),
+        factory);
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
@@ -106,7 +106,8 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     cr.setRecipientInstanceType(InstanceType.PARTICIPANT);
     cr.setSessionSpecific(false);
 
-    int nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg);
+    // int nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg);
+    int nMsgs = _participants[0].getMessagingService().send(cr, msg);
     AssertJUnit.assertTrue(nMsgs == 1);
     Thread.sleep(2500);
     // Thread.currentThread().join();
@@ -118,7 +119,8 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     cr.setSessionSpecific(false);
     cr.setDataSource(DataSource.IDEALSTATES);
 
-    nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg);
+    // nMsgs = _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg);
+    nMsgs = _participants[0].getMessagingService().send(cr, msg);
     AssertJUnit.assertTrue(nMsgs == 1);
     Thread.sleep(2500);
     // Thread.currentThread().join();
@@ -181,11 +183,11 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     String hostDest = "localhost_" + (START_PORT + 1);
 
     TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
-    _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-        factory.getMessageType(), factory);
+    _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(),
+        factory);
 
-    _startCMResultMap.get(hostSrc)._manager.getMessagingService().registerMessageHandlerFactory(
-        factory.getMessageType(), factory);
+    _participants[0].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(),
+        factory);
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
@@ -204,7 +206,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     TestAsyncCallback callback = new TestAsyncCallback(60000);
 
-    _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback, 60000);
+    _participants[0].getMessagingService().send(cr, msg, callback, 60000);
 
     Thread.sleep(2000);
     // Thread.currentThread().join();
@@ -212,7 +214,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(callback.getMessageReplied().size() == 1);
 
     TestAsyncCallback callback2 = new TestAsyncCallback(500);
-    _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback2, 500);
+    _participants[0].getMessagingService().send(cr, msg, callback2, 500);
 
     Thread.sleep(3000);
     // Thread.currentThread().join();
@@ -226,7 +228,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     callback = new TestAsyncCallback(60000);
 
-    _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback, 60000);
+    _participants[0].getMessagingService().send(cr, msg, callback, 60000);
 
     Thread.sleep(2000);
     // Thread.currentThread().join();
@@ -234,7 +236,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(callback.getMessageReplied().size() == 1);
 
     callback2 = new TestAsyncCallback(500);
-    _startCMResultMap.get(hostSrc)._manager.getMessagingService().send(cr, msg, callback2, 500);
+    _participants[0].getMessagingService().send(cr, msg, callback2, 500);
 
     Thread.sleep(3000);
     // Thread.currentThread().join();
@@ -248,8 +250,8 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     String hostDest = "localhost_" + (START_PORT + 1);
 
     TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
-    _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
-        factory.getMessageType(), factory);
+    _participants[1].getMessagingService().registerMessageHandlerFactory(factory.getMessageType(),
+        factory);
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
@@ -268,8 +270,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     AsyncCallback asyncCallback = new MockAsyncCallback();
     int messagesSent =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            asyncCallback, 60000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, asyncCallback, 60000);
 
     AssertJUnit.assertTrue(asyncCallback.getMessageReplied().get(0).getRecord()
         .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ReplyMessage")
@@ -277,9 +278,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(asyncCallback.getMessageReplied().size() == 1);
 
     AsyncCallback asyncCallback2 = new MockAsyncCallback();
-    messagesSent =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            asyncCallback2, 500);
+    messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, asyncCallback2, 500);
     AssertJUnit.assertTrue(asyncCallback2.isTimedOut());
 
   }
@@ -291,8 +290,9 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     for (int i = 0; i < NODE_NR; i++) {
       TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
       String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[0].getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
+
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
@@ -310,8 +310,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     cr.setSessionSpecific(false);
     AsyncCallback callback1 = new MockAsyncCallback();
     int messageSent1 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback1, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000);
 
     AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord()
         .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ReplyMessage")
@@ -319,37 +318,32 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(callback1.getMessageReplied().size() == NODE_NR - 1);
 
     AsyncCallback callback2 = new MockAsyncCallback();
-    int messageSent2 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback2, 500);
+    int messageSent2 = _participants[0].getMessagingService().sendAndWait(cr, msg, callback2, 500);
+
     AssertJUnit.assertTrue(callback2.isTimedOut());
 
     cr.setPartition("TestDB_17");
     AsyncCallback callback3 = new MockAsyncCallback();
     int messageSent3 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback3, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback3, 10000);
     AssertJUnit.assertTrue(callback3.getMessageReplied().size() == _replica - 1);
 
     cr.setPartition("TestDB_15");
     AsyncCallback callback4 = new MockAsyncCallback();
     int messageSent4 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback4, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback4, 10000);
     AssertJUnit.assertTrue(callback4.getMessageReplied().size() == _replica);
 
     cr.setPartitionState("SLAVE");
     AsyncCallback callback5 = new MockAsyncCallback();
     int messageSent5 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback5, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback5, 10000);
     AssertJUnit.assertTrue(callback5.getMessageReplied().size() == _replica - 1);
 
     cr.setDataSource(DataSource.IDEALSTATES);
     AsyncCallback callback6 = new MockAsyncCallback();
     int messageSent6 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback6, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback6, 10000);
     AssertJUnit.assertTrue(callback6.getMessageReplied().size() == _replica - 1);
   }
 
@@ -360,8 +354,9 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     for (int i = 0; i < NODE_NR; i++) {
       TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
       String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
+
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
@@ -380,8 +375,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     cr.setSelfExcluded(false);
     AsyncCallback callback1 = new MockAsyncCallback();
     int messageSent1 =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback1, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000);
 
     AssertJUnit.assertTrue(callback1.getMessageReplied().size() == NODE_NR);
     AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord()
@@ -396,8 +390,9 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     for (int i = 0; i < NODE_NR; i++) {
       TestMessagingHandlerFactory factory = new TestMessagingHandlerFactory();
       String hostDest = "localhost_" + (START_PORT + i);
-      _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
+      _participants[i].getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
+
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(MessageType.CONTROLLER_MSG, msgId);
@@ -416,8 +411,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     AsyncCallback callback1 = new MockAsyncCallback();
     int messagesSent =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback1, 10000);
+        _participants[0].getMessagingService().sendAndWait(cr, msg, callback1, 10000);
 
     AssertJUnit.assertTrue(callback1.getMessageReplied().get(0).getRecord()
         .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult")
@@ -428,9 +422,8 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     msg.setMessageId(msgId);
     cr.setPartition("TestDB_17");
     AsyncCallback callback2 = new MockAsyncCallback();
-    messagesSent =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback2, 10000);
+    messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, callback2, 10000);
+
     AssertJUnit.assertTrue(callback2.getMessageReplied().get(0).getRecord()
         .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult")
         .indexOf(hostSrc) != -1);
@@ -441,9 +434,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     msg.setMessageId(msgId);
     cr.setPartitionState("SLAVE");
     AsyncCallback callback3 = new MockAsyncCallback();
-    messagesSent =
-        _startCMResultMap.get(hostSrc)._manager.getMessagingService().sendAndWait(cr, msg,
-            callback3, 10000);
+    messagesSent = _participants[0].getMessagingService().sendAndWait(cr, msg, callback3, 10000);
     AssertJUnit.assertTrue(callback3.getMessageReplied().get(0).getRecord()
         .getMapField(Message.Attributes.MESSAGE_RESULT.toString()).get("ControllerResult")
         .indexOf(hostSrc) != -1);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 c5558ca..aa48c90 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,10 +22,9 @@ package org.apache.helix.integration;
 import java.util.Date;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockBootstrapModelFactory;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -51,18 +50,19 @@ public class TestNonOfflineInitState extends ZkIntegrationTestBase {
         1, // replicas
         "Bootstrap", true); // do rebalance
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
 
       // add a state model with non-OFFLINE initial state
-      StateMachineEngine stateMach = participants[i].getManager().getStateMachineEngine();
+      StateMachineEngine stateMach = participants[i].getStateMachineEngine();
       MockBootstrapModelFactory bootstrapFactory = new MockBootstrapModelFactory();
       stateMach.registerStateModelFactory("Bootstrap", bootstrapFactory);
 
@@ -74,16 +74,21 @@ public class TestNonOfflineInitState extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(result);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END testNonOfflineInitState at " + new Date(System.currentTimeMillis()));
   }
 
   private static void setupCluster(String clusterName, String ZkAddr, int startPort,
       String participantNamePrefix, String resourceNamePrefix, int resourceNb, int partitionNb,
       int nodesNb, int replica, String stateModelDef, boolean doRebalance) throws Exception {
-    ZkClient zkClient = new ZkClient(ZkAddr);
-    if (zkClient.exists("/" + clusterName)) {
+    if (_gZkClient.exists("/" + clusterName)) {
       LOG.warn("Cluster already exists:" + clusterName + ". Deleting it");
-      zkClient.deleteRecursive("/" + clusterName);
+      _gZkClient.deleteRecursive("/" + clusterName);
     }
 
     ClusterSetup setupTool = new ClusterSetup(ZkAddr);
@@ -103,7 +108,6 @@ public class TestNonOfflineInitState extends ZkIntegrationTestBase {
         setupTool.rebalanceStorageCluster(clusterName, dbName, replica);
       }
     }
-    zkClient.close();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 496d1a6..0f5cc72 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.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -44,7 +44,7 @@ public class TestNullReplica extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -61,14 +61,15 @@ public class TestNullReplica extends ZkIntegrationTestBase {
     idealState.getSimpleFields().remove(IdealState.IdealStateProperty.REPLICAS.toString());
     _gZkClient.writeData(idealStatePath, idealState);
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -78,13 +79,11 @@ public class TestNullReplica extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
 
-    Thread.sleep(2000);
-    controller.syncStop();
-
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
index 07a2fc0..750e2b7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
@@ -35,10 +35,13 @@ import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase {
+  private static Logger LOG = Logger.getLogger(TestParticipantErrorMessage.class);
+
   @Test()
   public void TestParticipantErrorMessageSend() {
     String participant1 = "localhost_" + START_PORT;
@@ -54,7 +57,7 @@ public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase {
     Criteria recipientCriteria = new Criteria();
     recipientCriteria.setRecipientInstanceType(InstanceType.CONTROLLER);
     recipientCriteria.setSessionSpecific(false);
-    _startCMResultMap.get(participant1)._manager.getMessagingService().send(recipientCriteria,
+    _participants[0].getMessagingService().send(recipientCriteria,
         errorMessage1);
 
     Message errorMessage2 =
@@ -69,23 +72,22 @@ public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase {
     Criteria recipientCriteria2 = new Criteria();
     recipientCriteria2.setRecipientInstanceType(InstanceType.CONTROLLER);
     recipientCriteria2.setSessionSpecific(false);
-    _startCMResultMap.get(participant2)._manager.getMessagingService().send(recipientCriteria2,
+    _participants[1].getMessagingService().send(recipientCriteria2,
         errorMessage2);
 
     try {
       Thread.sleep(1500);
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted sleep", e);
     }
 
     boolean result =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
             CLUSTER_NAME));
     Assert.assertTrue(result);
-    Builder kb = _startCMResultMap.get(participant2)._manager.getHelixDataAccessor().keyBuilder();
+    Builder kb = _participants[1].getHelixDataAccessor().keyBuilder();
     ExternalView externalView =
-        _startCMResultMap.get(participant2)._manager.getHelixDataAccessor().getProperty(
+        _participants[1].getHelixDataAccessor().getProperty(
             kb.externalView("TestDB"));
 
     for (String partitionName : externalView.getRecord().getMapFields().keySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 4227688..aa67ac9 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.TestHelper.StartCMResult;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.log4j.Logger;
 import org.testng.annotations.Test;
 
@@ -33,20 +33,21 @@ public class TestParticipantNameCollision extends ZkStandAloneCMTestBase {
   public void testParticiptantNameCollision() throws Exception {
     logger.info("RUN TestParticipantNameCollision() at " + new Date(System.currentTimeMillis()));
 
-    StartCMResult result = null;
+    MockParticipantManager newParticipant = null;
     for (int i = 0; i < 1; i++) {
       String instanceName = "localhost_" + (START_PORT + i);
       try {
         // the call fails on getClusterManagerForParticipant()
         // no threads start
-        result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName);
+        newParticipant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+        newParticipant.syncStart();
       } catch (Exception e) {
         e.printStackTrace();
       }
     }
 
     Thread.sleep(30000);
-    TestHelper.verifyWithTimeout("verifyNotConnected", 30 * 1000, result._manager);
+    TestHelper.verifyWithTimeout("verifyNotConnected", 30 * 1000, newParticipant);
 
     logger.info("STOP TestParticipantNameCollision() at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 d900d98..a1c413b 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,12 +24,12 @@ 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.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.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -47,7 +47,7 @@ public class TestPauseSignal extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -59,14 +59,15 @@ public class TestPauseSignal extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -112,13 +113,11 @@ public class TestPauseSignal extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
 
-    Thread.sleep(2000);
-    controller.syncStop();
-
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 be2de65..f2af156 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
@@ -9,10 +9,10 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.StateModelDefinition;
@@ -49,15 +49,16 @@ public class TestRedefineStateModelDef extends ZkUnitTestBase {
     autoRebalance(clusterName);
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -70,7 +71,7 @@ public class TestRedefineStateModelDef extends ZkUnitTestBase {
     // stop controller, redefine state model definition, and re-start controller
     controller.syncStop();
     redefineStateModelDef(clusterName);
-    controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     result =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 e04cc79..d3a370d 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
@@ -23,16 +23,17 @@ import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.id.StateModelDefId;
-import org.apache.helix.controller.HelixControllerMain;
 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.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -40,6 +41,14 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestRenamePartition extends ZkIntegrationTestBase {
+  // map from clusterName to participants
+  final Map<String, MockParticipantManager[]> _participantMap =
+      new ConcurrentHashMap<String, MockParticipantManager[]>();
+
+  // map from clusterName to controllers
+  final Map<String, ClusterControllerManager> _controllerMap =
+      new ConcurrentHashMap<String, ClusterControllerManager>();
+
   @Test()
   public void testRenamePartitionAutoIS() throws Exception {
     String clusterName = "CLUSTER_" + getShortClassName() + "_auto";
@@ -72,8 +81,8 @@ public class TestRenamePartition extends ZkIntegrationTestBase {
             ZK_ADDR, clusterName));
     Assert.assertTrue(result);
 
+    stop(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 
   @Test()
@@ -104,7 +113,7 @@ public class TestRenamePartition extends ZkIntegrationTestBase {
     idealState.setStateModelDefId(StateModelDefId.from("MasterSlave"));
 
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_gZkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
@@ -119,23 +128,25 @@ public class TestRenamePartition extends ZkIntegrationTestBase {
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             ZK_ADDR, clusterName));
     Assert.assertTrue(result);
+
+    stop(clusterName);
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
   }
 
   private void startAndVerify(String clusterName) throws Exception {
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     boolean result =
@@ -143,5 +154,21 @@ public class TestRenamePartition extends ZkIntegrationTestBase {
             ZK_ADDR, clusterName));
     Assert.assertTrue(result);
 
+    _participantMap.put(clusterName, participants);
+    _controllerMap.put(clusterName, controller);
+  }
+
+  private void stop(String clusterName) {
+    ClusterControllerManager controller = _controllerMap.get(clusterName);
+    if (controller != null) {
+      controller.syncStop();
+    }
+
+    MockParticipantManager[] participants = _participantMap.get(clusterName);
+    if (participants != null) {
+      for (MockParticipantManager participant : participants) {
+        participant.syncStop();
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 72920a8..7159a17 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,9 +25,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.mock.controller.ClusterController;
+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.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -54,7 +54,8 @@ public class TestResetInstance extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -65,16 +66,15 @@ public class TestResetInstance extends ZkIntegrationTestBase {
     };
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -102,8 +102,6 @@ public class TestResetInstance extends ZkIntegrationTestBase {
     Assert.assertTrue(result, "Cluster verification fails");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9479cff..af1ef13 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
@@ -31,9 +31,9 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
+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.mock.participant.MockParticipant;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
@@ -81,7 +81,8 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -92,16 +93,16 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
     };
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+            new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -170,8 +171,6 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
     Assert.assertEquals(_errToOfflineInvoked, 2, "Should reset 2 partitions");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 f8b4dc9..46a05d8 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,9 +25,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.mock.controller.ClusterController;
+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.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -53,7 +53,8 @@ public class TestResetResource extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -64,16 +65,15 @@ public class TestResetResource extends ZkIntegrationTestBase {
     };
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -101,8 +101,6 @@ public class TestResetResource extends ZkIntegrationTestBase {
     Assert.assertTrue(result, "Cluster verification fails");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 008782c..64043ed 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.controller.HelixControllerMain;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -35,15 +35,15 @@ import org.testng.annotations.Test;
 
 public class TestRestartParticipant extends ZkIntegrationTestBase {
   public class KillOtherTransition extends MockTransition {
-    final AtomicReference<MockParticipant> _other;
+    final AtomicReference<MockParticipantManager> _other;
 
-    public KillOtherTransition(MockParticipant other) {
-      _other = new AtomicReference<MockParticipant>(other);
+    public KillOtherTransition(MockParticipantManager other) {
+      _other = new AtomicReference<MockParticipantManager>(other);
     }
 
     @Override
     public void doTransition(Message message, NotificationContext context) {
-      MockParticipant other = _other.getAndSet(null);
+      MockParticipantManager other = _other.getAndSet(null);
       if (other != null) {
         System.err.println("Kill " + other.getInstanceName()
             + ". Interrupted exceptions are IGNORABLE");
@@ -58,7 +58,7 @@ public class TestRestartParticipant extends ZkIntegrationTestBase {
     System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis()));
 
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -69,19 +69,19 @@ public class TestRestartParticipant extends ZkIntegrationTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 4) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR, new KillOtherTransition(
-                participants[0]));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new KillOtherTransition(participants[0]));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
-        // Thread.sleep(100);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
 
       participants[i].syncStart();
@@ -94,9 +94,9 @@ public class TestRestartParticipant extends ZkIntegrationTestBase {
 
     // restart
     Thread.sleep(500);
-    MockParticipant participant =
-        new MockParticipant(participants[0].getClusterName(), participants[0].getInstanceName(),
-            ZK_ADDR, null);
+    MockParticipantManager participant =
+        new MockParticipantManager(ZK_ADDR, participants[0].getClusterName(),
+            participants[0].getInstanceName());
     System.err.println("Restart " + participant.getInstanceName());
     participant.syncStart();
     result =
@@ -104,6 +104,13 @@ public class TestRestartParticipant extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(result);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
+    participant.syncStop();
+
     System.out.println("START testRestartParticipant at " + new Date(System.currentTimeMillis()));
 
   }


[09/10] git commit: [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
[HELIX-279] Apply gc handling fixes to ZKHelixManager


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

Branch: refs/heads/master
Commit: f8e3b1af5c94b779d4244137131dbabc593ca0e8
Parents: 18a8c7c
Author: zzhang <zz...@apache.org>
Authored: Mon Nov 11 13:09:24 2013 -0800
Committer: zzhang <zz...@apache.org>
Committed: Mon Nov 11 13:09:24 2013 -0800

----------------------------------------------------------------------
 .../tools/TestHelixAdminScenariosRest.java      |  889 +++++++++------
 .../apache/helix/tools/TestResetInstance.java   |   21 +-
 .../helix/tools/TestResetPartitionState.java    |   24 +-
 .../apache/helix/tools/TestResetResource.java   |   22 +-
 .../org/apache/helix/agent/TestHelixAgent.java  |   27 +-
 .../java/org/apache/helix/HelixManager.java     |    7 +
 .../java/org/apache/helix/HelixProperty.java    |    6 +-
 .../rebalancer/SemiAutoRebalancer.java          |    3 +
 .../util/ConstraintBasedAssignment.java         |   55 +-
 .../stages/BestPossibleStateCalcStage.java      |   64 +-
 .../stages/CurrentStateComputationStage.java    |   13 +-
 .../stages/MessageGenerationStage.java          |    2 +-
 .../stages/MessageSelectionStage.java           |    9 +-
 .../controller/stages/MessageThrottleStage.java |   11 +-
 .../controller/stages/ResourceCurrentState.java |   13 +-
 .../controller/stages/TaskAssignmentStage.java  |   12 +-
 .../helix/manager/zk/AbstractManager.java       |  691 ------------
 .../helix/manager/zk/ControllerManager.java     |  174 ---
 .../manager/zk/ControllerManagerHelper.java     |    7 +-
 .../DefaultSchedulerMessageHandlerFactory.java  |    1 +
 .../zk/DistributedControllerManager.java        |  190 ----
 .../manager/zk/DistributedLeaderElection.java   |   16 +-
 .../manager/zk/HelixConnectionAdaptor.java      |    6 +
 .../helix/manager/zk/ParticipantManager.java    |  153 ---
 .../manager/zk/ParticipantManagerHelper.java    |   27 +-
 .../apache/helix/manager/zk/ZKHelixManager.java | 1041 +++++++++---------
 .../helix/manager/zk/ZkAsyncCallbacks.java      |    3 +-
 .../helix/manager/zk/ZkStateChangeListener.java |  127 ---
 .../messaging/DefaultMessagingService.java      |    4 +-
 .../apache/helix/model/ResourceAssignment.java  |   19 +-
 .../src/test/java/org/apache/helix/Mocks.java   |    6 +
 .../test/java/org/apache/helix/TestHelper.java  |   97 +-
 .../org/apache/helix/TestZkClientWrapper.java   |   45 +-
 .../java/org/apache/helix/TestZnodeModify.java  |    3 +-
 .../org/apache/helix/ZkHelixTestManager.java    |   44 -
 .../org/apache/helix/api/TestNewStages.java     |   12 +-
 .../controller/stages/DummyClusterManager.java  |    6 +
 .../stages/TestMessageThrottleStage.java        |   30 +-
 .../stages/TestParseInfoFromAlert.java          |    5 +-
 .../stages/TestRebalancePipeline.java           |   19 +-
 .../strategy/TestShufflingTwoStateStrategy.java |   68 +-
 .../helix/healthcheck/TestAddDropAlert.java     |   35 +-
 .../healthcheck/TestAlertActionTriggering.java  |   21 +-
 .../helix/healthcheck/TestAlertFireHistory.java |   38 +-
 .../helix/healthcheck/TestDummyAlerts.java      |   14 +-
 .../helix/healthcheck/TestExpandAlert.java      |   40 +-
 .../helix/healthcheck/TestSimpleAlert.java      |   40 +-
 .../healthcheck/TestSimpleWildcardAlert.java    |   42 +-
 .../helix/healthcheck/TestStalenessAlert.java   |   36 +-
 .../helix/healthcheck/TestWildcardAlert.java    |   36 +-
 .../helix/integration/TestAddClusterV2.java     |   78 +-
 .../TestAddNodeAfterControllerStart.java        |   56 +-
 .../TestAddStateModelFactoryAfterConnect.java   |   14 +-
 .../integration/TestAutoIsWithEmptyMap.java     |   14 +-
 .../helix/integration/TestAutoRebalance.java    |   64 +-
 .../TestAutoRebalancePartitionLimit.java        |  102 +-
 .../helix/integration/TestBatchMessage.java     |   50 +-
 .../integration/TestBatchMessageWrapper.java    |   18 +-
 .../integration/TestBucketizedResource.java     |   15 +-
 .../integration/TestCarryOverBadCurState.java   |   19 +-
 .../integration/TestCleanupExternalView.java    |   20 +-
 .../helix/integration/TestClusterStartsup.java  |   15 +-
 .../helix/integration/TestCustomIdealState.java |   12 -
 .../TestCustomizedIdealStateRebalancer.java     |    4 +-
 .../apache/helix/integration/TestDisable.java   |   36 +-
 .../helix/integration/TestDisableNode.java      |    2 +-
 .../helix/integration/TestDisablePartition.java |    2 +-
 .../integration/TestDistributedCMMain.java      |   18 +-
 .../TestDistributedClusterController.java       |   18 +-
 .../apache/helix/integration/TestDriver.java    |   80 +-
 .../org/apache/helix/integration/TestDrop.java  |  150 ++-
 .../helix/integration/TestDropResource.java     |    9 +-
 .../TestEnablePartitionDuringDisable.java       |   22 +-
 .../helix/integration/TestErrorPartition.java   |   28 +-
 .../integration/TestExternalViewUpdates.java    |   21 +-
 .../integration/TestHelixCustomCodeRunner.java  |   40 +-
 .../helix/integration/TestHelixInstanceTag.java |    3 +-
 .../helix/integration/TestInstanceAutoJoin.java |   25 +-
 .../integration/TestInvalidAutoIdealState.java  |   14 +-
 .../TestMessagePartitionStateMismatch.java      |    4 +-
 .../helix/integration/TestMessageThrottle.java  |   16 +-
 .../helix/integration/TestMessageThrottle2.java |    4 +-
 .../helix/integration/TestMessagingService.java |   81 +-
 .../integration/TestNonOfflineInitState.java    |   28 +-
 .../helix/integration/TestNullReplica.java      |   15 +-
 .../TestParticipantErrorMessage.java            |   14 +-
 .../TestParticipantNameCollision.java           |    9 +-
 .../helix/integration/TestPauseSignal.java      |   15 +-
 .../integration/TestRedefineStateModelDef.java  |   13 +-
 .../helix/integration/TestRenamePartition.java  |   45 +-
 .../helix/integration/TestResetInstance.java    |   18 +-
 .../integration/TestResetPartitionState.java    |   17 +-
 .../helix/integration/TestResetResource.java    |   18 +-
 .../integration/TestRestartParticipant.java     |   41 +-
 .../helix/integration/TestSchedulerMessage.java |  586 +++++++---
 .../integration/TestSchedulerMsgContraints.java |  254 -----
 .../integration/TestSchedulerMsgUsingQueue.java |  181 ---
 .../helix/integration/TestSchemataSM.java       |   14 +-
 .../TestSessionExpiryInTransition.java          |   29 +-
 .../helix/integration/TestStandAloneCMMain.java |   40 +-
 .../TestStandAloneCMSessionExpiry.java          |   22 +-
 ...estStartMultipleControllersWithSameName.java |    9 +-
 .../integration/TestStateTransitionTimeout.java |   38 +-
 .../helix/integration/TestSwapInstance.java     |   16 +-
 .../integration/TestZkCallbackHandlerLeak.java  |   89 +-
 .../integration/ZkIntegrationTestBase.java      |   49 +-
 .../integration/ZkStandAloneCMTestBase.java     |   63 +-
 ...dAloneCMTestBaseWithPropertyServerCheck.java |   25 +-
 .../manager/ClusterControllerManager.java       |   17 +-
 .../manager/ClusterDistributedController.java   |   20 +-
 .../manager/MockParticipantManager.java         |    8 +-
 .../TestDistributedControllerManager.java       |    9 +-
 .../manager/TestParticipantManager.java         |   26 +-
 .../apache/helix/josql/TestJosqlProcessor.java  |    8 +-
 .../TestDefaultControllerMsgHandlerFactory.java |    9 +-
 .../helix/manager/zk/TestHandleNewSession.java  |   16 +-
 .../manager/zk/TestLiveInstanceBounce.java      |   20 +-
 .../zk/TestZKPropertyTransferServer.java        |   21 +-
 .../helix/manager/zk/TestZkClusterManager.java  |   16 +-
 .../apache/helix/manager/zk/TestZkFlapping.java |   14 +-
 .../zk/TestZkManagerFlappingDetection.java      |  117 +-
 .../manager/zk/TestZkStateChangeListener.java   |   54 +-
 .../handling/TestConfigThreadpoolSize.java      |   10 +-
 .../handling/TestResourceThreadpoolSize.java    |    8 +-
 .../mock/controller/ClusterController.java      |  127 ---
 .../MockHealthReportParticipant.java            |   16 +-
 .../helix/mock/participant/MockParticipant.java |  181 ---
 .../helix/participant/MockZKHelixManager.java   |    6 +
 .../TestDistControllerStateModel.java           |   13 +-
 .../apache/helix/tools/TestHelixAdminCli.java   |  162 +--
 .../helix/userdefinedrebalancer/Lock.java       |    5 +
 131 files changed, 3175 insertions(+), 4684 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java
index 04a70d2..6a0e331 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestHelixAdminScenariosRest.java
@@ -19,24 +19,19 @@ package org.apache.helix.tools;
  * under the License.
  */
 
-/*
- * Simulate all the admin tasks needed by using command line tool
- * 
- * */
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
+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.ZKUtil;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState.IdealStateProperty;
@@ -64,8 +59,10 @@ import org.restlet.representation.Representation;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Simulate all the admin tasks needed by using command line tool
+ */
 public class TestHelixAdminScenariosRest extends AdminTestBase {
-  Map<String, StartCMResult> _startCMResultMap = new HashMap<String, StartCMResult>();
   RestAdminApplication _adminApp;
   Component _component;
   String _tag1 = "tag1123";
@@ -90,55 +87,6 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     return mapper.readValue(sr, clazz);
   }
 
-  @Test
-  public void testAddDeleteClusterAndInstanceAndResource() throws Exception {
-    // Helix bug helix-102
-    // ZKPropertyTransferServer.PERIOD = 500;
-    // ZkPropertyTransferClient.SEND_PERIOD = 500;
-    // ZKPropertyTransferServer.getInstance().init(19999, ZK_ADDR);
-
-    /** ======================= Add clusters ============================== */
-
-    testAddCluster();
-
-    /** ================= Add / drop some resources =========================== */
-
-    testAddResource();
-
-    /** ====================== Add / delete instances =========================== */
-
-    testAddInstance();
-
-    /** ===================== Rebalance resource =========================== */
-
-    testRebalanceResource();
-
-    /** ==================== start the clusters ============================= */
-
-    testStartCluster();
-
-    /** ==================== drop add resource in live clusters =================== */
-    testDropAddResource();
-
-    /** ======================Operations with live node ============================ */
-
-    testInstanceOperations();
-
-    /** ======================Operations with partitions ============================ */
-
-    testEnablePartitions();
-
-    /** ============================ expand cluster =========================== */
-
-    testExpandCluster();
-
-    /** ============================ deactivate cluster =========================== */
-    testDeactivateCluster();
-
-    // wait all zk callbacks done
-    Thread.sleep(1000);
-  }
-
   static String assertSuccessPostOperation(String url, Map<String, String> jsonParameters,
       boolean hasException) throws IOException {
     Reference resourceRef = new Reference(url);
@@ -226,41 +174,46 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Assert.assertTrue(exceptionThrown);
   }
 
+  private Map<String, String> addClusterCmd(String clusterName) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.CLUSTER_NAME, clusterName);
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster);
+
+    return parameters;
+  }
+
+  private void addCluster(String clusterName) throws IOException {
+    String url = "http://localhost:" + ADMIN_PORT + "/clusters";
+    String response = assertSuccessPostOperation(url, addClusterCmd(clusterName), false);
+    Assert.assertTrue(response.contains(clusterName));
+  }
+
+  @Test
   public void testAddCluster() throws Exception {
     String url = "http://localhost:" + ADMIN_PORT + "/clusters";
-    Map<String, String> paraMap = new HashMap<String, String>();
 
     // Normal add
-    paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster);
-
-    String response = assertSuccessPostOperation(url, paraMap, false);
+    String response = assertSuccessPostOperation(url, addClusterCmd("clusterTest"), false);
     Assert.assertTrue(response.contains("clusterTest"));
 
     // malformed cluster name
-    paraMap.put(JsonParameters.CLUSTER_NAME, "/ClusterTest");
-    response = assertSuccessPostOperation(url, paraMap, true);
+    response = assertSuccessPostOperation(url, addClusterCmd("/ClusterTest"), true);
 
     // Add the grand cluster
-    paraMap.put(JsonParameters.CLUSTER_NAME, "Klazt3rz");
-    response = assertSuccessPostOperation(url, paraMap, false);
+    response = assertSuccessPostOperation(url, addClusterCmd("Klazt3rz"), false);
     Assert.assertTrue(response.contains("Klazt3rz"));
 
-    paraMap.put(JsonParameters.CLUSTER_NAME, "\\ClusterTest");
-    response = assertSuccessPostOperation(url, paraMap, false);
+    response = assertSuccessPostOperation(url, addClusterCmd("\\ClusterTest"), false);
     Assert.assertTrue(response.contains("\\ClusterTest"));
 
     // Add already exist cluster
-    paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest");
-    response = assertSuccessPostOperation(url, paraMap, true);
+    response = assertSuccessPostOperation(url, addClusterCmd("clusterTest"), true);
 
     // delete cluster without resource and instance
     Assert.assertTrue(ZKUtil.isClusterSetup("Klazt3rz", _gZkClient));
     Assert.assertTrue(ZKUtil.isClusterSetup("clusterTest", _gZkClient));
     Assert.assertTrue(ZKUtil.isClusterSetup("\\ClusterTest", _gZkClient));
 
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.dropCluster);
-
     String clusterUrl = getClusterUrl("\\ClusterTest");
     deleteUrl(clusterUrl, false);
 
@@ -284,94 +237,180 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Assert.assertFalse(_gZkClient.exists("/clusterTest1"));
     Assert.assertFalse(_gZkClient.exists("/clusterTestOK"));
 
-    paraMap.put(JsonParameters.CLUSTER_NAME, "clusterTest1");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addCluster);
-    response = assertSuccessPostOperation(url, paraMap, false);
+    response = assertSuccessPostOperation(url, addClusterCmd("clusterTest1"), false);
     response = getUrl(clustersUrl);
     Assert.assertTrue(response.contains("clusterTest1"));
   }
 
-  public void testAddResource() throws Exception {
-    String reourcesUrl = "http://localhost:" + ADMIN_PORT + "/clusters/clusterTest1/resourceGroups";
+  private Map<String, String> addResourceCmd(String resourceName, String stateModelDef,
+      int partition) {
+    Map<String, String> parameters = new HashMap<String, String>();
 
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_22");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave");
-    paraMap.put(JsonParameters.PARTITIONS, "144");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource);
+    parameters.put(JsonParameters.RESOURCE_GROUP_NAME, resourceName);
+    parameters.put(JsonParameters.STATE_MODEL_DEF_REF, stateModelDef);
+    parameters.put(JsonParameters.PARTITIONS, "" + partition);
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource);
 
-    String response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
-    Assert.assertTrue(response.contains("db_22"));
+    return parameters;
+  }
+
+  private void addResource(String clusterName, String resourceName, int partitions)
+      throws IOException {
+    final String reourcesUrl =
+        "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups";
+    String response =
+        assertSuccessPostOperation(reourcesUrl,
+            addResourceCmd(resourceName, "MasterSlave", partitions), false);
+    Assert.assertTrue(response.contains(resourceName));
+  }
 
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave");
-    paraMap.put(JsonParameters.PARTITIONS, "44");
+  @Test
+  public void testAddResource() throws Exception {
+    final String clusterName = "clusterTestAddResource";
+    addCluster(clusterName);
+
+    String reourcesUrl =
+        "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups";
+    String response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_22", "MasterSlave", 144), false);
+    Assert.assertTrue(response.contains("db_22"));
 
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_11", "MasterSlave", 44), false);
     Assert.assertTrue(response.contains("db_11"));
 
     // Add duplicate resource
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_22");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "OnlineOffline");
-    paraMap.put(JsonParameters.PARTITIONS, "55");
-
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, true);
+    response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_22", "OnlineOffline", 55), true);
 
     // drop resource now
-    String resourceUrl = getResourceUrl("clusterTest1", "db_11");
+    String resourceUrl = getResourceUrl(clusterName, "db_11");
     deleteUrl(resourceUrl, false);
-    Assert.assertFalse(_gZkClient.exists("/clusterTest1/IDEALSTATES/db_11"));
+    Assert.assertFalse(_gZkClient.exists("/" + clusterName + "/IDEALSTATES/db_11"));
 
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave");
-    paraMap.put(JsonParameters.PARTITIONS, "44");
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_11", "MasterSlave", 44), false);
     Assert.assertTrue(response.contains("db_11"));
 
-    Assert.assertTrue(_gZkClient.exists("/clusterTest1/IDEALSTATES/db_11"));
+    Assert.assertTrue(_gZkClient.exists("/" + clusterName + "/IDEALSTATES/db_11"));
 
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_33");
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_33", "MasterSlave", 44), false);
     Assert.assertTrue(response.contains("db_33"));
 
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_44");
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(reourcesUrl, addResourceCmd("db_44", "MasterSlave", 44), false);
     Assert.assertTrue(response.contains("db_44"));
   }
 
-  private void testDeactivateCluster() throws Exception, InterruptedException {
-    HelixDataAccessor accessor;
-    String path;
-    // deactivate cluster
-    String clusterUrl = getClusterUrl("clusterTest1");
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.ENABLED, "false");
-    paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster);
+  private Map<String, String> activateClusterCmd(String grandClusterName, boolean enabled) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.GRAND_CLUSTER, grandClusterName);
+    parameters.put(JsonParameters.ENABLED, "" + enabled);
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster);
 
-    String response = assertSuccessPostOperation(clusterUrl, paraMap, false);
+    return parameters;
+  }
+
+  @Test
+  public void testDeactivateCluster() throws Exception {
+    final String clusterName = "clusterTestDeactivateCluster";
+    final String controllerClusterName = "controllerClusterTestDeactivateCluster";
+
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    Map<String, ClusterDistributedController> distControllers =
+        new HashMap<String, ClusterDistributedController>();
+
+    // setup cluster
+    addCluster(clusterName);
+    addInstancesToCluster(clusterName, "localhost:123", 6, null);
+    addResource(clusterName, "db_11", 16);
+    rebalanceResource(clusterName, "db_11");
+
+    addCluster(controllerClusterName);
+    addInstancesToCluster(controllerClusterName, "controller_900", 2, null);
+
+    // start mock nodes
+    for (int i = 0; i < 6; i++) {
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
+    }
+
+    // start controller nodes
+    for (int i = 0; i < 2; i++) {
+      String controllerName = "controller_900" + i;
+      ClusterDistributedController distController =
+          new ClusterDistributedController(ZK_ADDR, controllerClusterName, controllerName);
+      distController.syncStart();
+      distControllers.put(controllerName, distController);
+    }
+
+    String clusterUrl = getClusterUrl(clusterName);
+
+    // activate cluster
+    assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), false);
+    boolean verifyResult =
+        ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
+            controllerClusterName));
+    Assert.assertTrue(verifyResult);
+
+    verifyResult =
+        ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
+            clusterName));
+    Assert.assertTrue(verifyResult);
+
+    // deactivate cluster
+    assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, false), false);
     Thread.sleep(6000);
-    Assert.assertFalse(_gZkClient.exists("/Klazt3rz/IDEALSTATES/clusterTest1"));
+    Assert.assertFalse(_gZkClient.exists("/" + controllerClusterName + "/IDEALSTATES/"
+        + clusterName));
 
-    accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor();
-    path = accessor.keyBuilder().controllerLeader().getPath();
+    HelixDataAccessor accessor = participants.get("localhost_1231").getHelixDataAccessor();
+    String path = accessor.keyBuilder().controllerLeader().getPath();
     Assert.assertFalse(_gZkClient.exists(path));
 
     deleteUrl(clusterUrl, true);
+    Assert.assertTrue(_gZkClient.exists("/" + clusterName));
 
-    Assert.assertTrue(_gZkClient.exists("/clusterTest1"));
     // leader node should be gone
-    for (StartCMResult result : _startCMResultMap.values()) {
-      result._manager.disconnect();
-      result._thread.interrupt();
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
     }
     deleteUrl(clusterUrl, false);
 
-    Assert.assertFalse(_gZkClient.exists("/clusterTest1"));
+    Assert.assertFalse(_gZkClient.exists("/" + clusterName));
+
+    // clean up
+    for (ClusterDistributedController controller : distControllers.values()) {
+      controller.syncStop();
+    }
+
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
+  }
+
+  private Map<String, String> addIdealStateCmd() {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addIdealState);
+
+    return parameters;
   }
 
-  private void testDropAddResource() throws Exception {
-    ZNRecord record = _gSetupTool._admin.getResourceIdealState("clusterTest1", "db_11").getRecord();
+  @Test
+  public void testDropAddResource() throws Exception {
+    final String clusterName = "clusterTestDropAddResource";
+
+    // setup cluster
+    addCluster(clusterName);
+    addResource(clusterName, "db_11", 22);
+    addInstancesToCluster(clusterName, "localhost_123", 6, null);
+    rebalanceResource(clusterName, "db_11");
+    ZNRecord record = _gSetupTool._admin.getResourceIdealState(clusterName, "db_11").getRecord();
     String x = ObjectToJson(record);
 
     FileWriter fos = new FileWriter("/tmp/temp.log");
@@ -379,217 +418,370 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     pw.write(x);
     pw.close();
 
-    String resourceUrl = getResourceUrl("clusterTest1", "db_11");
-    deleteUrl(resourceUrl, false);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900");
+    controller.syncStart();
 
+    // start mock nodes
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    for (int i = 0; i < 6; i++) {
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
+    }
     boolean verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource);
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11");
-    paraMap.put(JsonParameters.PARTITIONS, "22");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave");
-    String response =
-        assertSuccessPostOperation(getClusterUrl("clusterTest1") + "/resourceGroups", paraMap,
-            false);
 
-    String idealStateUrl = getResourceUrl("clusterTest1", "db_11") + "/idealState";
-    Assert.assertTrue(response.contains("db_11"));
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addIdealState);
+    String resourceUrl = getResourceUrl(clusterName, "db_11");
+    deleteUrl(resourceUrl, false);
+
+    verifyResult =
+        ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
+            clusterName));
+    Assert.assertTrue(verifyResult);
+    addResource(clusterName, "db_11", 22);
+
+    String idealStateUrl = getResourceUrl(clusterName, "db_11") + "/idealState";
     Map<String, String> extraform = new HashMap<String, String>();
     extraform.put(JsonParameters.NEW_IDEAL_STATE, x);
-    response = assertSuccessPostOperation(idealStateUrl, paraMap, extraform, false);
+    assertSuccessPostOperation(idealStateUrl, addIdealStateCmd(), extraform, false);
 
     verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
 
-    ZNRecord record2 =
-        _gSetupTool._admin.getResourceIdealState("clusterTest1", "db_11").getRecord();
+    ZNRecord record2 = _gSetupTool._admin.getResourceIdealState(clusterName, "db_11").getRecord();
     Assert.assertTrue(record2.equals(record));
+
+    // clean up
+    controller.syncStop();
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
   }
 
-  private void testExpandCluster() throws Exception {
-    boolean verifyResult;
+  private Map<String, String> addInstanceCmd(String instances) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.INSTANCE_NAMES, instances);
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
 
-    String clusterUrl = getClusterUrl("clusterTest1");
-    String instancesUrl = clusterUrl + "/instances";
+    return parameters;
+  }
+
+  private Map<String, String> expandClusterCmd() {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.expandCluster);
+
+    return parameters;
+  }
+
+  @Test
+  public void testExpandCluster() throws Exception {
 
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.INSTANCE_NAMES,
-        "localhost:12331;localhost:12341;localhost:12351;localhost:12361");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
+    final String clusterName = "clusterTestExpandCluster";
 
-    String response = assertSuccessPostOperation(instancesUrl, paraMap, false);
-    String[] hosts = "localhost:12331;localhost:12341;localhost:12351;localhost:12361".split(";");
+    // setup cluster
+    addCluster(clusterName);
+    addInstancesToCluster(clusterName, "localhost:123", 6, null);
+    addResource(clusterName, "db_11", 22);
+    rebalanceResource(clusterName, "db_11");
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900");
+    controller.syncStart();
+
+    // start mock nodes
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    for (int i = 0; i < 6; i++) {
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
+    }
+
+    boolean verifyResult =
+        ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
+            clusterName));
+    Assert.assertTrue(verifyResult);
+
+    String clusterUrl = getClusterUrl(clusterName);
+    String instancesUrl = clusterUrl + "/instances";
+
+    String instances = "localhost:12331;localhost:12341;localhost:12351;localhost:12361";
+    String response = assertSuccessPostOperation(instancesUrl, addInstanceCmd(instances), false);
+    String[] hosts = instances.split(";");
     for (String host : hosts) {
       Assert.assertTrue(response.contains(host.replace(':', '_')));
     }
-    paraMap.clear();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.expandCluster);
-    response = assertSuccessPostOperation(clusterUrl, paraMap, false);
+
+    response = assertSuccessPostOperation(clusterUrl, expandClusterCmd(), false);
 
     for (int i = 3; i <= 6; i++) {
-      StartCMResult result =
-          TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_123" + i + "1");
-      _startCMResultMap.put("localhost_123" + i + "1", result);
+      String instanceName = "localhost_123" + i + "1";
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
     }
 
     verifyResult =
-        ClusterStateVerifier.verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR,
-            "clusterTest1"));
+        ClusterStateVerifier
+            .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName));
     Assert.assertTrue(verifyResult);
 
     verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
+
+    // clean up
+    controller.syncStop();
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
   }
 
-  private void testEnablePartitions() throws IOException, InterruptedException {
-    HelixDataAccessor accessor;
-    accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor();
+  private Map<String, String> enablePartitionCmd(String resourceName, String partitions,
+      boolean enabled) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enablePartition);
+    parameters.put(JsonParameters.ENABLED, "" + enabled);
+    parameters.put(JsonParameters.PARTITION, partitions);
+    parameters.put(JsonParameters.RESOURCE, resourceName);
+
+    return parameters;
+  }
+
+  @Test
+  public void testEnablePartitions() throws IOException, InterruptedException {
+    final String clusterName = "clusterTestEnablePartitions";
+
+    // setup cluster
+    addCluster(clusterName);
+    addInstancesToCluster(clusterName, "localhost:123", 6, null);
+    addResource(clusterName, "db_11", 22);
+    rebalanceResource(clusterName, "db_11");
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900");
+    controller.syncStart();
+
+    // start mock nodes
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    for (int i = 0; i < 6; i++) {
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
+    }
+
+    HelixDataAccessor accessor = participants.get("localhost_1231").getHelixDataAccessor();
     // drop node should fail as not disabled
     String hostName = "localhost_1231";
-    String instanceUrl = getInstanceUrl("clusterTest1", hostName);
+    String instanceUrl = getInstanceUrl(clusterName, hostName);
     ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11"));
 
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enablePartition);
-    paraMap.put(JsonParameters.ENABLED, "false");
-    paraMap.put(JsonParameters.PARTITION, "db_11_0;db_11_15");
-    paraMap.put(JsonParameters.RESOURCE, "db_11");
-
-    String response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+    String response =
+        assertSuccessPostOperation(instanceUrl,
+            enablePartitionCmd("db_11", "db_11_0;db_11_11", false), false);
     Assert.assertTrue(response.contains("DISABLED_PARTITION"));
     Assert.assertTrue(response.contains("db_11_0"));
-    Assert.assertTrue(response.contains("db_11_15"));
+    Assert.assertTrue(response.contains("db_11_11"));
 
     boolean verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
 
     ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11"));
     Assert.assertEquals(ev.getStateMap("db_11_0").get(hostName), "OFFLINE");
-    Assert.assertEquals(ev.getStateMap("db_11_15").get(hostName), "OFFLINE");
+    Assert.assertEquals(ev.getStateMap("db_11_11").get(hostName), "OFFLINE");
 
-    paraMap.put(JsonParameters.ENABLED, "true");
-    response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(instanceUrl,
+            enablePartitionCmd("db_11", "db_11_0;db_11_11", true), false);
     Assert.assertFalse(response.contains("db_11_0"));
-    Assert.assertFalse(response.contains("db_11_15"));
+    Assert.assertFalse(response.contains("db_11_11"));
 
     verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
 
     ev = accessor.getProperty(accessor.keyBuilder().externalView("db_11"));
     Assert.assertEquals(ev.getStateMap("db_11_0").get(hostName), "MASTER");
-    Assert.assertEquals(ev.getStateMap("db_11_15").get(hostName), "SLAVE");
+    Assert.assertEquals(ev.getStateMap("db_11_11").get(hostName), "SLAVE");
+
+    // clean up
+    controller.syncStop();
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
+  }
+
+  private Map<String, String> enableInstanceCmd(boolean enabled) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance);
+    parameters.put(JsonParameters.ENABLED, "" + enabled);
+    return parameters;
   }
 
-  private void testInstanceOperations() throws Exception {
+  private Map<String, String> swapInstanceCmd(String oldInstance, String newInstance) {
+    Map<String, String> parameters = new HashMap<String, String>();
+
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance);
+    parameters.put(JsonParameters.OLD_INSTANCE, oldInstance);
+    parameters.put(JsonParameters.NEW_INSTANCE, newInstance);
+
+    return parameters;
+  }
+
+  @Test
+  public void testInstanceOperations() throws Exception {
+    final String clusterName = "clusterTestInstanceOperations";
+
+    // setup cluster
+    addCluster(clusterName);
+    addInstancesToCluster(clusterName, "localhost:123", 6, null);
+    addResource(clusterName, "db_11", 8);
+    rebalanceResource(clusterName, "db_11");
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_9900");
+    controller.syncStart();
+
+    // start mock nodes
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    for (int i = 0; i < 6; i++) {
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
+    }
+
     HelixDataAccessor accessor;
     // drop node should fail as not disabled
-    String instanceUrl = getInstanceUrl("clusterTest1", "localhost_1232");
+    String instanceUrl = getInstanceUrl(clusterName, "localhost_1232");
     deleteUrl(instanceUrl, true);
 
     // disabled node
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance);
-    paraMap.put(JsonParameters.ENABLED, "false");
-    String response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+    String response = assertSuccessPostOperation(instanceUrl, enableInstanceCmd(false), false);
     Assert.assertTrue(response.contains("false"));
 
     // Cannot drop / swap
     deleteUrl(instanceUrl, true);
 
-    String instancesUrl = getClusterUrl("clusterTest1") + "/instances";
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance);
-    paraMap.put(JsonParameters.OLD_INSTANCE, "localhost_1232");
-    paraMap.put(JsonParameters.NEW_INSTANCE, "localhost_12320");
-    response = assertSuccessPostOperation(instancesUrl, paraMap, true);
+    String instancesUrl = getClusterUrl(clusterName) + "/instances";
+    response =
+        assertSuccessPostOperation(instancesUrl,
+            swapInstanceCmd("localhost_1232", "localhost_12320"), true);
 
     // disconnect the node
-    _startCMResultMap.get("localhost_1232")._manager.disconnect();
-    _startCMResultMap.get("localhost_1232")._thread.interrupt();
+    participants.get("localhost_1232").syncStop();
 
     // add new node then swap instance
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
-    paraMap.put(JsonParameters.INSTANCE_NAME, "localhost_12320");
-    response = assertSuccessPostOperation(instancesUrl, paraMap, false);
+    response = assertSuccessPostOperation(instancesUrl, addInstanceCmd("localhost_12320"), false);
     Assert.assertTrue(response.contains("localhost_12320"));
 
     // swap instance. The instance get swapped out should not exist anymore
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.swapInstance);
-    paraMap.put(JsonParameters.OLD_INSTANCE, "localhost_1232");
-    paraMap.put(JsonParameters.NEW_INSTANCE, "localhost_12320");
-    response = assertSuccessPostOperation(instancesUrl, paraMap, false);
+    response =
+        assertSuccessPostOperation(instancesUrl,
+            swapInstanceCmd("localhost_1232", "localhost_12320"), false);
     Assert.assertTrue(response.contains("localhost_12320"));
     Assert.assertFalse(response.contains("localhost_1232\""));
 
-    accessor = _startCMResultMap.get("localhost_1231")._manager.getHelixDataAccessor();
+    accessor = participants.get("localhost_1231").getHelixDataAccessor();
     String path = accessor.keyBuilder().instanceConfig("localhost_1232").getPath();
     Assert.assertFalse(_gZkClient.exists(path));
 
-    _startCMResultMap.put("localhost_12320",
-        TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_12320"));
+    MockParticipantManager newParticipant =
+        new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12320");
+    newParticipant.syncStart();
+    participants.put("localhost_12320", newParticipant);
+
+    boolean verifyResult =
+        ClusterStateVerifier
+            .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName));
+    Assert.assertTrue(verifyResult);
+
+    // clean up
+    controller.syncStop();
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
   }
 
-  private void testStartCluster() throws Exception, InterruptedException {
+  @Test
+  public void testStartCluster() throws Exception {
+    final String clusterName = "clusterTestStartCluster";
+    final String controllerClusterName = "controllerClusterTestStartCluster";
+
+    Map<String, MockParticipantManager> participants =
+        new HashMap<String, MockParticipantManager>();
+    Map<String, ClusterDistributedController> distControllers =
+        new HashMap<String, ClusterDistributedController>();
+
+    // setup cluster
+    addCluster(clusterName);
+    addInstancesToCluster(clusterName, "localhost:123", 6, null);
+    addResource(clusterName, "db_11", 8);
+    rebalanceResource(clusterName, "db_11");
+
+    addCluster(controllerClusterName);
+    addInstancesToCluster(controllerClusterName, "controller_900", 2, null);
+
     // start mock nodes
     for (int i = 0; i < 6; i++) {
-      StartCMResult result =
-          TestHelper.startDummyProcess(ZK_ADDR, "clusterTest1", "localhost_123" + i);
-      _startCMResultMap.put("localhost_123" + i, result);
+      String instanceName = "localhost_123" + i;
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participant.syncStart();
+      participants.put(instanceName, participant);
     }
 
     // start controller nodes
     for (int i = 0; i < 2; i++) {
-      StartCMResult result =
-          TestHelper.startController("Klazt3rz", "controller_900" + i, ZK_ADDR,
-              HelixControllerMain.DISTRIBUTED);
-
-      _startCMResultMap.put("controller_900" + i, result);
+      String controllerName = "controller_900" + i;
+      ClusterDistributedController distController =
+          new ClusterDistributedController(ZK_ADDR, controllerClusterName, controllerName);
+      distController.syncStart();
+      distControllers.put(controllerName, distController);
     }
     Thread.sleep(100);
 
     // activate clusters
     // wrong grand clustername
-
-    String clusterUrl = getClusterUrl("clusterTest1");
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.ENABLED, "true");
-    paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazters");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster);
-
-    String response = assertSuccessPostOperation(clusterUrl, paraMap, true);
+    String clusterUrl = getClusterUrl(clusterName);
+    assertSuccessPostOperation(clusterUrl, activateClusterCmd("nonExistCluster", true), true);
 
     // wrong cluster name
-    clusterUrl = getClusterUrl("clusterTest2");
-    paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz");
-    response = assertSuccessPostOperation(clusterUrl, paraMap, true);
+    clusterUrl = getClusterUrl("nonExistCluster");
+    assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), true);
 
-    paraMap.put(JsonParameters.ENABLED, "true");
-    paraMap.put(JsonParameters.GRAND_CLUSTER, "Klazt3rz");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.activateCluster);
-    clusterUrl = getClusterUrl("clusterTest1");
-    response = assertSuccessPostOperation(clusterUrl, paraMap, false);
+    clusterUrl = getClusterUrl(clusterName);
+    assertSuccessPostOperation(clusterUrl, activateClusterCmd(controllerClusterName, true), false);
     Thread.sleep(500);
 
     deleteUrl(clusterUrl, true);
 
     // verify leader node
-    HelixDataAccessor accessor =
-        _startCMResultMap.get("controller_9001")._manager.getHelixDataAccessor();
+    HelixDataAccessor accessor = distControllers.get("controller_9001").getHelixDataAccessor();
     LiveInstance controllerLeader = accessor.getProperty(accessor.keyBuilder().controllerLeader());
     Assert.assertTrue(controllerLeader.getInstanceName().startsWith("controller_900"));
 
-    accessor = _startCMResultMap.get("localhost_1232")._manager.getHelixDataAccessor();
+    accessor = participants.get("localhost_1232").getHelixDataAccessor();
     LiveInstance leader = accessor.getProperty(accessor.keyBuilder().controllerLeader());
     for (int i = 0; i < 5; i++) {
       if (leader != null) {
@@ -601,81 +793,98 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     Assert.assertTrue(leader.getInstanceName().startsWith("controller_900"));
 
     boolean verifyResult =
-        ClusterStateVerifier.verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR,
-            "clusterTest1"));
+        ClusterStateVerifier
+            .verifyByZkCallback(new MasterNbInExtViewVerifier(ZK_ADDR, clusterName));
     Assert.assertTrue(verifyResult);
 
     verifyResult =
         ClusterStateVerifier.verifyByZkCallback(new BestPossAndExtViewZkVerifier(ZK_ADDR,
-            "clusterTest1"));
+            clusterName));
     Assert.assertTrue(verifyResult);
+    Thread.sleep(1000);
+
+    // clean up
+    for (ClusterDistributedController controller : distControllers.values()) {
+      controller.syncStop();
+    }
+    for (MockParticipantManager participant : participants.values()) {
+      participant.syncStop();
+    }
+  }
+
+  private Map<String, String> rebalanceCmd(int replicas, String prefix, String tag) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.REPLICAS, "" + replicas);
+    if (prefix != null) {
+      parameters.put(JsonParameters.RESOURCE_KEY_PREFIX, prefix);
+    }
+    if (tag != null) {
+      parameters.put(ClusterSetup.instanceGroupTag, tag);
+    }
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
+
+    return parameters;
   }
 
-  private void testRebalanceResource() throws Exception {
-    String resourceUrl = getResourceUrl("clusterTest1", "db_11");
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.REPLICAS, "3");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
+  private void rebalanceResource(String clusterName, String resourceName) throws IOException {
+    String resourceUrl = getResourceUrl(clusterName, resourceName);
+    String idealStateUrl = resourceUrl + "/idealState";
+
+    assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false);
+  }
+
+  @Test
+  public void testRebalanceResource() throws Exception {
+    // add a normal cluster
+    final String clusterName = "clusterTestRebalanceResource";
+    addCluster(clusterName);
 
-    String ISUrl = resourceUrl + "/idealState";
-    String response = assertSuccessPostOperation(ISUrl, paraMap, false);
+    addInstancesToCluster(clusterName, "localhost:123", 3, _tag1);
+    addResource(clusterName, "db_11", 44);
+
+    String resourceUrl = getResourceUrl(clusterName, "db_11");
+
+    String idealStateUrl = resourceUrl + "/idealState";
+    String response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false);
     ZNRecord record = JsonToObject(ZNRecord.class, response);
     Assert.assertTrue(record.getId().equalsIgnoreCase("db_11"));
-    Assert
-        .assertTrue((((List<String>) (record.getListFields().values().toArray()[0]))).size() == 3);
-    Assert.assertTrue((((Map<String, String>) (record.getMapFields().values().toArray()[0])))
-        .size() == 3);
+    Assert.assertEquals(record.getListField("db_11_0").size(), 3);
+    Assert.assertEquals(record.getMapField("db_11_0").size(), 3);
 
     deleteUrl(resourceUrl, false);
 
     // re-add and rebalance
-    String reourcesUrl = "http://localhost:" + ADMIN_PORT + "/clusters/clusterTest1/resourceGroups";
+    final String reourcesUrl =
+        "http://localhost:" + ADMIN_PORT + "/clusters/" + clusterName + "/resourceGroups";
+
     response = getUrl(reourcesUrl);
     Assert.assertFalse(response.contains("db_11"));
 
-    paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.RESOURCE_GROUP_NAME, "db_11");
-    paraMap.put(JsonParameters.STATE_MODEL_DEF_REF, "MasterSlave");
-    paraMap.put(JsonParameters.PARTITIONS, "48");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addResource);
-
-    response = assertSuccessPostOperation(reourcesUrl, paraMap, false);
-    Assert.assertTrue(response.contains("db_11"));
-
-    ISUrl = resourceUrl + "/idealState";
-    paraMap.put(JsonParameters.REPLICAS, "3");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
-    response = assertSuccessPostOperation(ISUrl, paraMap, false);
+    addResource(clusterName, "db_11", 48);
+    idealStateUrl = resourceUrl + "/idealState";
+    response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(3, null, null), false);
     record = JsonToObject(ZNRecord.class, response);
     Assert.assertTrue(record.getId().equalsIgnoreCase("db_11"));
-    Assert
-        .assertTrue((((List<String>) (record.getListFields().values().toArray()[0]))).size() == 3);
-    Assert.assertTrue((((Map<String, String>) (record.getMapFields().values().toArray()[0])))
-        .size() == 3);
+    Assert.assertEquals(record.getListField("db_11_0").size(), 3);
+    Assert.assertEquals(record.getMapField("db_11_0").size(), 3);
 
     // rebalance with key prefix
-    resourceUrl = getResourceUrl("clusterTest1", "db_22");
-    ISUrl = resourceUrl + "/idealState";
-    paraMap.put(JsonParameters.REPLICAS, "2");
-    paraMap.put(JsonParameters.RESOURCE_KEY_PREFIX, "alias");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
-    response = assertSuccessPostOperation(ISUrl, paraMap, false);
+    addResource(clusterName, "db_22", 55);
+    resourceUrl = getResourceUrl(clusterName, "db_22");
+    idealStateUrl = resourceUrl + "/idealState";
+    response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, "alias", null), false);
     record = JsonToObject(ZNRecord.class, response);
     Assert.assertTrue(record.getId().equalsIgnoreCase("db_22"));
-    Assert
-        .assertTrue((((List<String>) (record.getListFields().values().toArray()[0]))).size() == 2);
-    Assert.assertTrue((((Map<String, String>) (record.getMapFields().values().toArray()[0])))
-        .size() == 2);
+    Assert.assertEquals(record.getListField("alias_0").size(), 2);
+    Assert.assertEquals(record.getMapField("alias_0").size(), 2);
     Assert.assertTrue((((String) (record.getMapFields().keySet().toArray()[0])))
         .startsWith("alias_"));
     Assert.assertFalse(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString()));
-    resourceUrl = getResourceUrl("clusterTest1", "db_33");
-    ISUrl = resourceUrl + "/idealState";
-    paraMap.put(JsonParameters.REPLICAS, "2");
-    paraMap.remove(JsonParameters.RESOURCE_KEY_PREFIX);
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
-    paraMap.put(ClusterSetup.instanceGroupTag, _tag1);
-    response = assertSuccessPostOperation(ISUrl, paraMap, false);
+
+    addResource(clusterName, "db_33", 44);
+    resourceUrl = getResourceUrl(clusterName, "db_33");
+    idealStateUrl = resourceUrl + "/idealState";
+    response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, null, _tag1), false);
 
     Assert.assertTrue(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString()));
     Assert.assertTrue(response.contains(_tag1));
@@ -688,14 +897,10 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
       }
     }
 
-    resourceUrl = getResourceUrl("clusterTest1", "db_44");
-    ISUrl = resourceUrl + "/idealState";
-    paraMap.put(JsonParameters.REPLICAS, "2");
-    paraMap.remove(JsonParameters.RESOURCE_KEY_PREFIX);
-    paraMap.put(JsonParameters.RESOURCE_KEY_PREFIX, "alias");
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.rebalance);
-    paraMap.put(ClusterSetup.instanceGroupTag, _tag1);
-    response = assertSuccessPostOperation(ISUrl, paraMap, false);
+    addResource(clusterName, "db_44", 44);
+    resourceUrl = getResourceUrl(clusterName, "db_44");
+    idealStateUrl = resourceUrl + "/idealState";
+    response = assertSuccessPostOperation(idealStateUrl, rebalanceCmd(2, "alias", _tag1), false);
     Assert.assertTrue(response.contains(IdealStateProperty.INSTANCE_GROUP_TAG.toString()));
     Assert.assertTrue(response.contains(_tag1));
 
@@ -713,24 +918,66 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
     }
   }
 
-  private void testAddInstance() throws Exception {
-    String clusterUrl = getClusterUrl("clusterTest1");
-    Map<String, String> paraMap = new HashMap<String, String>();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
-    String response = null;
-    // Add instances to cluster
+  private void addInstancesToCluster(String clusterName, String instanceNamePrefix, int n,
+      String tag) throws IOException {
+    Map<String, String> parameters = new HashMap<String, String>();
+    final String clusterUrl = getClusterUrl(clusterName);
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
+
+    // add instances to cluster
     String instancesUrl = clusterUrl + "/instances";
-    for (int i = 0; i < 3; i++) {
+    for (int i = 0; i < n; i++) {
 
-      paraMap.put(JsonParameters.INSTANCE_NAME, "localhost:123" + i);
-      response = assertSuccessPostOperation(instancesUrl, paraMap, false);
-      Assert.assertTrue(response.contains(("localhost:123" + i).replace(':', '_')));
+      parameters.put(JsonParameters.INSTANCE_NAME, instanceNamePrefix + i);
+      String response = assertSuccessPostOperation(instancesUrl, parameters, false);
+      Assert.assertTrue(response.contains((instanceNamePrefix + i).replace(':', '_')));
     }
-    paraMap.remove(JsonParameters.INSTANCE_NAME);
-    paraMap.put(JsonParameters.INSTANCE_NAMES,
-        "localhost:1233;localhost:1234;localhost:1235;localhost:1236");
 
-    response = assertSuccessPostOperation(instancesUrl, paraMap, false);
+    // add tag to instance
+    if (tag != null && !tag.isEmpty()) {
+      parameters.clear();
+      parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag);
+      parameters.put(ClusterSetup.instanceGroupTag, tag);
+      for (int i = 0; i < n; i++) {
+        String instanceUrl = instancesUrl + "/" + (instanceNamePrefix + i).replace(':', '_');
+        String response = assertSuccessPostOperation(instanceUrl, parameters, false);
+        Assert.assertTrue(response.contains(_tag1));
+      }
+    }
+
+  }
+
+  private Map<String, String> addInstanceTagCmd(String tag) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag);
+    parameters.put(ClusterSetup.instanceGroupTag, tag);
+
+    return parameters;
+  }
+
+  private Map<String, String> removeInstanceTagCmd(String tag) {
+    Map<String, String> parameters = new HashMap<String, String>();
+    parameters.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.removeInstanceTag);
+    parameters.put(ClusterSetup.instanceGroupTag, tag);
+
+    return parameters;
+  }
+
+  @Test
+  public void testAddInstance() throws Exception {
+    final String clusterName = "clusterTestAddInstance";
+
+    // add normal cluster
+    addCluster(clusterName);
+
+    String clusterUrl = getClusterUrl(clusterName);
+
+    // Add instances to cluster
+    String instancesUrl = clusterUrl + "/instances";
+    addInstancesToCluster(clusterName, "localhost:123", 3, null);
+
+    String instances = "localhost:1233;localhost:1234;localhost:1235;localhost:1236";
+    String response = assertSuccessPostOperation(instancesUrl, addInstanceCmd(instances), false);
     for (int i = 3; i <= 6; i++) {
       Assert.assertTrue(response.contains("localhost_123" + i));
     }
@@ -749,42 +996,34 @@ public class TestHelixAdminScenariosRest extends AdminTestBase {
 
     // disable node
     instanceUrl = instancesUrl + "/localhost_1236";
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.enableInstance);
-    paraMap.put(JsonParameters.ENABLED, "false");
-    response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+    response = assertSuccessPostOperation(instanceUrl, enableInstanceCmd(false), false);
     Assert.assertTrue(response.contains("false"));
 
     deleteUrl(instanceUrl, false);
 
+    // add controller cluster
+    final String controllerClusterName = "controllerClusterTestAddInstance";
+    addCluster(controllerClusterName);
+
     // add node to controller cluster
-    paraMap.remove(JsonParameters.INSTANCE_NAME);
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstance);
-    paraMap.put(JsonParameters.INSTANCE_NAMES, "controller:9000;controller:9001");
-    String controllerUrl = getClusterUrl("Klazt3rz") + "/instances";
-    response = assertSuccessPostOperation(controllerUrl, paraMap, false);
+    String controllers = "controller:9000;controller:9001";
+    String controllerUrl = getClusterUrl(controllerClusterName) + "/instances";
+    response = assertSuccessPostOperation(controllerUrl, addInstanceCmd(controllers), false);
     Assert.assertTrue(response.contains("controller_9000"));
     Assert.assertTrue(response.contains("controller_9001"));
 
-    // add a dup host
-    paraMap.remove(JsonParameters.INSTANCE_NAMES);
-    paraMap.put(JsonParameters.INSTANCE_NAME, "localhost:1234");
-    response = assertSuccessPostOperation(instancesUrl, paraMap, true);
-
-    // add tags
+    // add a duplicated host
+    response = assertSuccessPostOperation(instancesUrl, addInstanceCmd("localhost:1234"), true);
 
-    paraMap.clear();
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.addInstanceTag);
-    paraMap.put(ClusterSetup.instanceGroupTag, _tag1);
+    // add/remove tags
     for (int i = 0; i < 4; i++) {
       instanceUrl = instancesUrl + "/localhost_123" + i;
-      response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+      response = assertSuccessPostOperation(instanceUrl, addInstanceTagCmd(_tag1), false);
       Assert.assertTrue(response.contains(_tag1));
-
     }
-    paraMap.put(JsonParameters.MANAGEMENT_COMMAND, ClusterSetup.removeInstanceTag);
+
     instanceUrl = instancesUrl + "/localhost_1233";
-    response = assertSuccessPostOperation(instanceUrl, paraMap, false);
+    response = assertSuccessPostOperation(instanceUrl, removeInstanceTagCmd(_tag1), false);
     Assert.assertFalse(response.contains(_tag1));
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java
index 9534cf5..fd12080 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetInstance.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+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.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -53,12 +53,8 @@ public class TestResetInstance extends AdminTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    // // start admin thread
-    // AdminThread adminThread = new AdminThread(ZK_ADDR, _port);
-    // adminThread.start();
-
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -69,16 +65,16 @@ public class TestResetInstance extends AdminTestBase {
     };
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+            new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -111,9 +107,6 @@ public class TestResetInstance extends AdminTestBase {
     Assert.assertTrue(result, "Cluster verification fails");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
-    // adminThread.stop();
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
index 9b07445..82a2607 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
@@ -30,10 +30,10 @@ 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.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
@@ -95,12 +95,8 @@ public class TestResetPartitionState extends AdminTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    // start admin thread
-    // AdminThread adminThread = new AdminThread(ZK_ADDR, _port);
-    // adminThread.start();
-
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>();
@@ -108,16 +104,16 @@ public class TestResetPartitionState extends AdminTestBase {
     errPartitions.put("OFFLINE-SLAVE", TestHelper.setOf("TestDB0_8"));
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+            new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -171,9 +167,6 @@ public class TestResetPartitionState extends AdminTestBase {
     Assert.assertEquals(_errToOfflineInvoked.get(), 2, "reset() should be invoked 2 times");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
-    // adminThread.stop();
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
@@ -185,8 +178,7 @@ public class TestResetPartitionState extends AdminTestBase {
   private void clearStatusUpdate(String clusterName, String instance, String resource,
       String partition) {
     // clear status update for error partition so verify() will not fail on
-    // old
-    // errors
+    // old errors
     ZKHelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java
index 96f4f6c..db9e9bb 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetResource.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+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.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -53,12 +53,8 @@ public class TestResetResource extends AdminTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    // start admin thread
-    // AdminThread adminThread = new AdminThread(ZK_ADDR, _port);
-    // adminThread.start();
-
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>() {
@@ -69,16 +65,16 @@ public class TestResetResource extends AdminTestBase {
     };
 
     // start mock participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
         participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+            new MockParticipantManager(ZK_ADDR, clusterName, instanceName); 
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -112,15 +108,11 @@ public class TestResetResource extends AdminTestBase {
     Assert.assertTrue(result, "Cluster verification fails");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
-    // adminThread.stop();
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 083cbd4..27b4d36 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
@@ -21,27 +21,30 @@ package org.apache.helix.agent;
 
 import java.io.File;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ExternalCommand;
 import org.apache.helix.ScriptTestHelper;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.mock.controller.ClusterController;
+import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public class TestHelixAgent extends ZkUnitTestBase {
+  private final static Logger LOG = Logger.getLogger(TestHelixAgent.class);
+
   final String workingDir = ScriptTestHelper.getPrefix() + ScriptTestHelper.INTEGRATION_SCRIPT_DIR;
   ExternalCommand serverCmd = null;
 
@@ -94,13 +97,9 @@ public class TestHelixAgent extends ZkUnitTestBase {
         "MasterSlave", true); // do rebalance
 
     // set cluster config
-    ZkClient client =
-        new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT, ZkClient.DEFAULT_CONNECTION_TIMEOUT,
-            new ZNRecordSerializer());
-
     HelixConfigScope scope =
         new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build();
-    ConfigAccessor configAccessor = new ConfigAccessor(client);
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
 
     // String pidFile = ScriptTestHelper.getPrefix() + ScriptTestHelper.INTEGRATION_LOG_DIR +
     // "/default/foo_{PARTITION_NAME}_pid.txt";
@@ -148,10 +147,11 @@ public class TestHelixAgent extends ZkUnitTestBase {
     configAccessor.set(scope, cmdConfig.toKeyValueMap());
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", zkAddr);
+    ClusterControllerManager controller = new ClusterControllerManager(zkAddr, clusterName, "controller_0");
     controller.syncStart();
 
     // start helix-agent
+    Map<String, Thread> agents = new HashMap<String, Thread>();
     for (int i = 0; i < n; i++) {
       final String instanceName = "localhost_" + (12918 + i);
       Thread agentThread = new Thread() {
@@ -163,11 +163,11 @@ public class TestHelixAgent extends ZkUnitTestBase {
                 "--stateModel", "MasterSlave"
             });
           } catch (Exception e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
+            LOG.error("Exception start helix-agent", e);
           }
         }
       };
+      agents.put(instanceName, agentThread);
       agentThread.start();
 
       // wait participant thread to start
@@ -197,6 +197,11 @@ public class TestHelixAgent extends ZkUnitTestBase {
             clusterName));
     Assert.assertTrue(result);
 
+    // clean up
+    controller.syncStop();
+    for (Thread agentThread : agents.values()) {
+      agentThread.interrupt();
+    }
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 24f8f1e..54e9943 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -102,6 +102,7 @@ public interface HelixManager {
    * @param listener
    * @deprecated replaced by addInstanceConfigChangeListener()
    */
+  @Deprecated
   void addConfigChangeListener(ConfigChangeListener listener) throws Exception;
 
   /**
@@ -155,6 +156,12 @@ public interface HelixManager {
   void addControllerListener(ControllerChangeListener listener);
 
   /**
+   * Add message listener for controller
+   * @param listener
+   */
+  void addControllerMessageListener(MessageListener listener);
+
+  /**
    * Removes the listener. If the same listener was used for multiple changes,
    * all change notifications will be removed.<br/>
    * This will invoke onChange method on the listener with

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/HelixProperty.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixProperty.java b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
index 9c0c25e..00189b6 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixProperty.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
@@ -28,11 +28,14 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.api.config.NamespacedConfig;
+import org.apache.log4j.Logger;
 
 /**
  * A wrapper class for ZNRecord. Used as a base class for IdealState, CurrentState, etc.
  */
 public class HelixProperty {
+  private static Logger LOG = Logger.getLogger(HelixProperty.class);
+
   public enum HelixPropertyAttribute {
     BUCKET_SIZE,
     BATCH_MESSAGE_MODE
@@ -138,8 +141,7 @@ public class HelixProperty {
       });
       return constructor.newInstance(record);
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Exception convert znrecord: " + record + " to class: " + clazz, e);
     }
 
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
index c5a7f22..a0ad6f3 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
@@ -9,6 +9,7 @@ import org.apache.helix.api.Cluster;
 import org.apache.helix.api.State;
 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.rebalancer.context.RebalancerConfig;
 import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
@@ -59,6 +60,7 @@ public class SemiAutoRebalancer implements HelixRebalancer {
       LOG.debug("Processing resource:" + config.getResourceId());
     }
     ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
+
     for (PartitionId partition : config.getPartitionSet()) {
       Map<ParticipantId, State> currentStateMap =
           currentState.getCurrentStateMap(config.getResourceId(), partition);
@@ -71,6 +73,7 @@ public class SemiAutoRebalancer implements HelixRebalancer {
       Map<State, String> upperBounds =
           ConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(),
               cluster.getConfig());
+
       Map<ParticipantId, State> bestStateForPartition =
           ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, cluster
               .getLiveParticipantMap().keySet(), stateModelDef, preferenceList, currentStateMap,

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
index a0ee1c1..84129de 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.rebalancer.util;
  */
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -110,38 +111,56 @@ public class ConstraintBasedAssignment {
   }
 
   /**
-   * compute best state for resource in SEMI_AUTO and FULL_AUTO modes
-   * @param upperBounds map of state to upper bound
-   * @param liveParticipantSet set of live participant ids
-   * @param stateModelDef
-   * @param participantPreferenceList
-   * @param currentStateMap
-   *          : participant->state for each partition
-   * @param disabledParticipantsForPartition
-   * @return
+   * Get a mapping for a partition for the current state participants who have been dropped or
+   * disabled for a given partition.
+   * @param currentStateMap current map of participant id to state for a partition
+   * @param participants participants selected to serve the partition
+   * @param disabledParticipants participants that have been disabled for this partition
+   * @param initialState the initial state of the resource state model
+   * @return map of participant id to state of dropped and disabled partitions
    */
-  public static Map<ParticipantId, State> computeAutoBestStateForPartition(
-      Map<State, String> upperBounds, Set<ParticipantId> liveParticipantSet,
-      StateModelDefinition stateModelDef, List<ParticipantId> participantPreferenceList,
-      Map<ParticipantId, State> currentStateMap, Set<ParticipantId> disabledParticipantsForPartition) {
+  public static Map<ParticipantId, State> dropAndDisablePartitions(
+      Map<ParticipantId, State> currentStateMap, Collection<ParticipantId> participants,
+      Set<ParticipantId> disabledParticipants, State initialState) {
     Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
-
     // if the resource is deleted, instancePreferenceList will be empty and
     // we should drop all resources.
     if (currentStateMap != null) {
       for (ParticipantId participantId : currentStateMap.keySet()) {
-        if ((participantPreferenceList == null || !participantPreferenceList
-            .contains(participantId)) && !disabledParticipantsForPartition.contains(participantId)) {
+        if ((participants == null || !participants.contains(participantId))
+            && !disabledParticipants.contains(participantId)) {
           // if dropped and not disabled, transit to DROPPED
           participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED));
         } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get(
             participantId).equals(State.from(HelixDefinedState.ERROR)))
-            && disabledParticipantsForPartition.contains(participantId)) {
+            && disabledParticipants.contains(participantId)) {
           // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
+          participantStateMap.put(participantId, initialState);
         }
       }
     }
+    return participantStateMap;
+  }
+
+  /**
+   * compute best state for resource in SEMI_AUTO and FULL_AUTO modes
+   * @param upperBounds map of state to upper bound
+   * @param liveParticipantSet set of live participant ids
+   * @param stateModelDef
+   * @param participantPreferenceList
+   * @param currentStateMap
+   *          : participant->state for each partition
+   * @param disabledParticipantsForPartition
+   * @return
+   */
+  public static Map<ParticipantId, State> computeAutoBestStateForPartition(
+      Map<State, String> upperBounds, Set<ParticipantId> liveParticipantSet,
+      StateModelDefinition stateModelDef, List<ParticipantId> participantPreferenceList,
+      Map<ParticipantId, State> currentStateMap, Set<ParticipantId> disabledParticipantsForPartition) {
+    // drop and disable participants if necessary
+    Map<ParticipantId, State> participantStateMap =
+        dropAndDisablePartitions(currentStateMap, participantPreferenceList,
+            disabledParticipantsForPartition, stateModelDef.getTypedInitialState());
 
     // resource is deleted
     if (participantPreferenceList == null) {


[05/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9af6ca8..af079c1 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,12 +30,11 @@ 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.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
+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.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterSetup;
@@ -89,15 +88,16 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     TestZkChildListener listener = new TestZkChildListener();
     _gZkClient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener);
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -110,7 +110,6 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
@@ -139,15 +138,16 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -157,7 +157,6 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // stop all participants
-    Thread.sleep(1000);
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
@@ -178,7 +177,7 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -191,7 +190,6 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
@@ -207,10 +205,9 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     String clusterName = className + "_" + methodName;
 
     final int n = 5;
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
-    // ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, // resource#
         6, // partition#
@@ -229,19 +226,20 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     final String hostToFail = "localhost_12921";
     final String partitionToFail = "TestDB0_4";
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (instanceName.equals(hostToFail)) {
         Map<String, Set<String>> errPartitions = new HashMap<String, Set<String>>();
-        errPartitions.put("SLAVE-MASTER", TestHelper.setOf(partitionToFail));
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+        errPartitions.put("SLAVE-MASTER", TestHelper.setOf("TestDB0_4"));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -298,7 +296,8 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     TestZkChildListener listener = new TestZkChildListener();
     _gZkClient.subscribeChildChanges(keyBuilder.messages("localhost_12918").getPath(), listener);
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // pause controller
@@ -308,11 +307,11 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
     });
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -336,7 +335,6 @@ public class TestBatchMessage extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 2ae8bf3..9bfdfb5 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
@@ -26,12 +26,12 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.messaging.handling.BatchMessageWrapper;
-import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockMSModelFactory;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -90,17 +90,19 @@ public class TestBatchMessageWrapper extends ZkUnitTestBase {
     idealState.setBatchMessageMode(true);
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[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 MockParticipant(ftys[i], clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].getStateMachineEngine().registerStateModelFactory("MasterSlave", ftys[i]);
       participants[i].syncStart();
 
       // wait for each participant to complete state transitions, so we have deterministic results
@@ -133,6 +135,12 @@ public class TestBatchMessageWrapper extends ZkUnitTestBase {
     Assert.assertEquals(wrapper._startCount, 2,
         "Expect 2 batch.end: O->S and S->M for 2nd participant");
 
+    // 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/incubator-helix/blob/f8e3b1af/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 6edca76..51b0d3c 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,10 +24,10 @@ import java.util.Date;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -45,7 +45,7 @@ public class TestBucketizedResource extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     // ClusterSetup setupTool = new ClusterSetup(ZK_ADDR);
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
@@ -66,13 +66,15 @@ public class TestBucketizedResource extends ZkIntegrationTestBase {
     idealState.setBucketSize(1);
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -87,6 +89,7 @@ public class TestBucketizedResource extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9d63a65..97b6ebb 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.controller.HelixControllerMain;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
@@ -39,7 +39,7 @@ public class TestCarryOverBadCurState extends ZkIntegrationTestBase {
     System.out.println("START testCarryOverBadCurState at " + new Date(System.currentTimeMillis()));
 
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -58,13 +58,15 @@ public class TestCarryOverBadCurState extends ZkIntegrationTestBase {
     _gZkClient.createPersistent(path, true);
     _gZkClient.writeData(path, badCurState);
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -78,6 +80,11 @@ public class TestCarryOverBadCurState extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(result);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
     System.out.println("END testCarryOverBadCurState at " + new Date(System.currentTimeMillis()));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 49c5576..93c58eb 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
@@ -26,11 +26,11 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 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.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -61,15 +61,16 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -84,7 +85,7 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
     admin.enableCluster(clusterName, false);
     // wait all pending zk-events being processed, otherwise remove current-state will cause
     // controller send O->S message
-    ZkTestHelper.tryWaitZkEventsCleaned(controller.getManager().getZkClient());
+    ZkTestHelper.tryWaitZkEventsCleaned(controller.getZkClient());
     // System.out.println("paused controller");
 
     // drop resource
@@ -121,8 +122,13 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
     Assert.assertNull(externalView, "external-view for TestDB0 should be removed, but was: "
         + externalView);
 
-    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+    // 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/incubator-helix/blob/f8e3b1af/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 c0ced72..e844a27 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.manager.zk.ZkClient;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.util.HelixUtil;
 import org.testng.Assert;
@@ -40,8 +39,8 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
     String namespace = "/" + CLUSTER_NAME;
-    if (_zkClient.exists(namespace)) {
-      _zkClient.deleteRecursive(namespace);
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
     }
     _setupTool = new ClusterSetup(ZK_ADDR);
 
@@ -58,13 +57,12 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
   @Override
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
+
   }
 
   @Override
   @AfterClass()
   public void afterClass() {
-    _zkClient.close();
   }
 
   @Test()
@@ -72,9 +70,8 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
     setupCluster();
     String controllerMsgPath =
         HelixUtil.getControllerPropertyPath(CLUSTER_NAME, PropertyType.MESSAGES_CONTROLLER);
-    _zkClient.deleteRecursive(controllerMsgPath);
+    _gZkClient.deleteRecursive(controllerMsgPath);
     HelixManager manager = null;
-    ;
 
     try {
       manager =
@@ -106,7 +103,7 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
 
     setupCluster();
     String stateModelPath = HelixUtil.getStateModelDefinitionPath(CLUSTER_NAME);
-    _zkClient.deleteRecursive(stateModelPath);
+    _gZkClient.deleteRecursive(stateModelPath);
 
     try {
       manager =
@@ -125,7 +122,7 @@ public class TestClusterStartsup extends ZkStandAloneCMTestBase {
     String instanceStatusUpdatePath =
         HelixUtil.getInstancePropertyPath(CLUSTER_NAME, "localhost_" + (START_PORT + 1),
             PropertyType.STATUSUPDATES);
-    _zkClient.deleteRecursive(instanceStatusUpdatePath);
+    _gZkClient.deleteRecursive(instanceStatusUpdatePath);
 
     try {
       manager =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 43cfa5a..5467932 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
@@ -32,18 +32,6 @@ import org.testng.annotations.Test;
 
 public class TestCustomIdealState extends ZkIntegrationTestBase {
   private static Logger LOG = Logger.getLogger(TestCustomIdealState.class);
-  ZkClient _zkClient;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
-  }
-
-  @AfterClass
-  public void afterClass() {
-    _zkClient.close();
-  }
 
   @Test
   public void testBasic() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 2c0badc..ce26e2e 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
@@ -105,12 +105,12 @@ public class TestCustomizedIdealStateRebalancer extends
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 3);
 
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, db2));
     Assert.assertTrue(result);
     Thread.sleep(1000);
     HelixDataAccessor accessor =
-        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2));
     Assert.assertEquals(ev.getPartitionSet().size(), 60);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 f6ddd34..cc7fd5f 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,10 +27,10 @@ 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.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterSetup;
@@ -52,7 +52,7 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -72,14 +72,15 @@ public class TestDisable extends ZkIntegrationTestBase {
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -120,7 +121,6 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
@@ -140,7 +140,7 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -152,14 +152,15 @@ public class TestDisable extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -200,7 +201,6 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
@@ -219,7 +219,7 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -239,14 +239,15 @@ public class TestDisable extends ZkIntegrationTestBase {
     accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -292,7 +293,6 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
@@ -311,7 +311,7 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -323,14 +323,15 @@ public class TestDisable extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -376,7 +377,6 @@ public class TestDisable extends ZkIntegrationTestBase {
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java
index 93c765c..be523d0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableNode.java
@@ -38,7 +38,7 @@ public class TestDisableNode extends ZkStandAloneCMTestBaseWithPropertyServerChe
             ZK_ADDR, CLUSTER_NAME));
     Assert.assertTrue(result);
 
-    ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient);
+    ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
     tool.enableInstance(CLUSTER_NAME, PARTICIPANT_PREFIX + "_12918", true);
 
     result =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java
index 3067a0b..ba7e8e4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisablePartition.java
@@ -56,7 +56,7 @@ public class TestDisablePartition extends ZkStandAloneCMTestBaseWithPropertyServ
 
     TestHelper.verifyState(CLUSTER_NAME, ZK_ADDR, map, "OFFLINE");
 
-    ZKHelixAdmin tool = new ZKHelixAdmin(_zkClient);
+    ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
     tool.enablePartition(true, CLUSTER_NAME, "localhost_12919", "TestDB", Arrays.asList("TestDB_9"));
 
     result =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 3155dd4..8135191 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
@@ -25,10 +25,10 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterDistributedController;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -77,11 +77,10 @@ public class TestDistributedCMMain extends ZkIntegrationTestBase {
         "LeaderStandby", true); // do rebalance
 
     // start distributed cluster controllers
-    ClusterController[] controllers = new ClusterController[n + n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[n + n];
     for (int i = 0; i < n; i++) {
       controllers[i] =
-          new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR,
-              HelixControllerMain.DISTRIBUTED.toString());
+          new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 
@@ -92,11 +91,11 @@ public class TestDistributedCMMain extends ZkIntegrationTestBase {
     Assert.assertTrue(result, "Controller cluster NOT in ideal state");
 
     // start first cluster
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     final String firstClusterName = clusterNamePrefix + "0_0";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost0_" + (12918 + i);
-      participants[i] = new MockParticipant(firstClusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, firstClusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -114,8 +113,7 @@ public class TestDistributedCMMain extends ZkIntegrationTestBase {
     setupTool.rebalanceStorageCluster(controllerClusterName, clusterNamePrefix + "0", 6);
     for (int i = n; i < 2 * n; i++) {
       controllers[i] =
-          new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR,
-              HelixControllerMain.DISTRIBUTED.toString());
+          new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 
@@ -157,7 +155,6 @@ public class TestDistributedCMMain extends ZkIntegrationTestBase {
     // clean up
     // wait for all zk callbacks done
     System.out.println("Cleaning up...");
-    Thread.sleep(2000);
     for (int i = 0; i < 5; i++) {
       result =
           ClusterStateVerifier
@@ -166,7 +163,6 @@ public class TestDistributedCMMain extends ZkIntegrationTestBase {
       controllers[i].syncStop();
     }
 
-    // Thread.sleep(2000);
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 672df10..f3def23 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
@@ -25,10 +25,10 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterDistributedController;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -76,11 +76,10 @@ public class TestDistributedClusterController extends ZkIntegrationTestBase {
         "LeaderStandby", true); // do rebalance
 
     // start distributed cluster controllers
-    ClusterController[] controllers = new ClusterController[n];
+    ClusterDistributedController[] controllers = new ClusterDistributedController[n];
     for (int i = 0; i < n; i++) {
       controllers[i] =
-          new ClusterController(controllerClusterName, "controller_" + i, ZK_ADDR,
-              HelixControllerMain.DISTRIBUTED.toString());
+          new ClusterDistributedController(ZK_ADDR, controllerClusterName, "controller_" + i);
       controllers[i].syncStart();
     }
 
@@ -91,11 +90,11 @@ public class TestDistributedClusterController extends ZkIntegrationTestBase {
     Assert.assertTrue(result, "Controller cluster NOT in ideal state");
 
     // start first cluster
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     final String firstClusterName = clusterNamePrefix + "0_0";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost0_" + (12918 + i);
-      participants[i] = new MockParticipant(firstClusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, firstClusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -114,11 +113,11 @@ public class TestDistributedClusterController extends ZkIntegrationTestBase {
     controllers[j].syncStop();
 
     // setup the second cluster
-    MockParticipant[] participants2 = new MockParticipant[n];
+    MockParticipantManager[] participants2 = new MockParticipantManager[n];
     final String secondClusterName = clusterNamePrefix + "0_1";
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost1_" + (12918 + i);
-      participants2[i] = new MockParticipant(secondClusterName, instanceName, ZK_ADDR, null);
+      participants2[i] = new MockParticipantManager(ZK_ADDR, secondClusterName, instanceName);
       participants2[i].syncStart();
     }
 
@@ -130,7 +129,6 @@ public class TestDistributedClusterController extends ZkIntegrationTestBase {
     // clean up
     // wait for all zk callbacks done
     System.out.println("Cleaning up...");
-    Thread.sleep(1000);
     for (int i = 0; i < 5; i++) {
       result =
           ClusterStateVerifier

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 b29e25d..2f2431b 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
@@ -21,21 +21,18 @@ package org.apache.helix.integration;
 
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
 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.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.IdealState.IdealStateProperty;
@@ -46,7 +43,6 @@ import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.TestCommand;
 import org.apache.helix.tools.TestCommand.CommandType;
-import org.apache.helix.tools.TestCommand.NodeOpArg;
 import org.apache.helix.tools.TestExecutor;
 import org.apache.helix.tools.TestExecutor.ZnodePropertyType;
 import org.apache.helix.tools.TestTrigger;
@@ -79,10 +75,8 @@ public class TestDriver {
     public final int _numNode;
     public final int _replica;
 
-    // public final Map<String, ZNRecord> _idealStateMap = new
-    // ConcurrentHashMap<String, ZNRecord>();
-    public final Map<String, StartCMResult> _startCMResultMap =
-        new ConcurrentHashMap<String, StartCMResult>();
+    public final Map<String, HelixManager> _managers =
+        new ConcurrentHashMap<String, HelixManager>();
 
     public TestInfo(String clusterName, ZkClient zkClient, int numDb, int numPartitionsPerDb,
         int numNode, int replica) {
@@ -118,10 +112,6 @@ public class TestDriver {
         replica, true);
   }
 
-  // public static void setupCluster(String uniqTestName, ZkClient zkClient, int
-  // numDb,
-  // int numPartitionPerDb, int numNodes, int replica, boolean doRebalance)
-  // throws Exception
   public static void setupCluster(String uniqClusterName, String zkAddr, int numResources,
       int numPartitionsPerResource, int numInstances, int replica, boolean doRebalance)
       throws Exception {
@@ -193,11 +183,15 @@ public class TestDriver {
     for (int id : instanceIds) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + id);
 
-      if (testInfo._startCMResultMap.containsKey(instanceName)) {
+      // if (testInfo._startCMResultMap.containsKey(instanceName)) {
+      if (testInfo._managers.containsKey(instanceName)) {
         LOG.warn("Dummy participant:" + instanceName + " has already started; skip starting it");
       } else {
-        StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, clusterName, instanceName);
-        testInfo._startCMResultMap.put(instanceName, result);
+        // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, clusterName, instanceName);
+        MockParticipantManager participant =
+            new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participant.syncStart();
+        testInfo._managers.put(instanceName, participant);
         // testInfo._instanceStarted.countDown();
       }
     }
@@ -220,13 +214,13 @@ public class TestDriver {
 
     for (int id : nodeIds) {
       String controllerName = CONTROLLER_PREFIX + "_" + id;
-      if (testInfo._startCMResultMap.containsKey(controllerName)) {
+      if (testInfo._managers.containsKey(controllerName)) {
         LOG.warn("Controller:" + controllerName + " has already started; skip starting it");
       } else {
-        StartCMResult result =
-            TestHelper.startController(clusterName, controllerName, ZK_ADDR,
-                HelixControllerMain.STANDALONE);
-        testInfo._startCMResultMap.put(controllerName, result);
+        ClusterControllerManager controller =
+            new ClusterControllerManager(ZK_ADDR, clusterName, controllerName);
+        controller.syncStart();
+        testInfo._managers.put(controllerName, controller);
       }
     }
   }
@@ -257,27 +251,22 @@ public class TestDriver {
     TestInfo testInfo = _testInfoMap.remove(uniqClusterName);
 
     // stop controller first
-    for (Iterator<Entry<String, StartCMResult>> it =
-        testInfo._startCMResultMap.entrySet().iterator(); it.hasNext();) {
-      Map.Entry<String, StartCMResult> entry = it.next();
-      String instanceName = entry.getKey();
+    for (String instanceName : testInfo._managers.keySet()) {
       if (instanceName.startsWith(CONTROLLER_PREFIX)) {
-        it.remove();
-        HelixManager manager = entry.getValue()._manager;
-        manager.disconnect();
-        Thread thread = entry.getValue()._thread;
-        thread.interrupt();
+        ClusterControllerManager controller =
+            (ClusterControllerManager) testInfo._managers.get(instanceName);
+        controller.syncStop();
       }
     }
 
     Thread.sleep(1000);
 
-    // stop the rest
-    for (Map.Entry<String, StartCMResult> entry : testInfo._startCMResultMap.entrySet()) {
-      HelixManager manager = entry.getValue()._manager;
-      manager.disconnect();
-      Thread thread = entry.getValue()._thread;
-      thread.interrupt();
+    for (String instanceName : testInfo._managers.keySet()) {
+      if (!instanceName.startsWith(CONTROLLER_PREFIX)) {
+        MockParticipantManager participant =
+            (MockParticipantManager) testInfo._managers.get(instanceName);
+        participant.syncStop();
+      }
     }
 
     testInfo._zkClient.close();
@@ -292,23 +281,24 @@ public class TestDriver {
     }
 
     TestInfo testInfo = _testInfoMap.get(uniqClusterName);
-    // String clusterName = testInfo._clusterName;
 
     String failHost = PARTICIPANT_PREFIX + "_" + (START_PORT + instanceId);
-    StartCMResult result = testInfo._startCMResultMap.remove(failHost);
+    MockParticipantManager participant =
+        (MockParticipantManager) testInfo._managers.remove(failHost);
 
     // TODO need sync
-    if (result == null || result._manager == null || result._thread == null) {
+    if (participant == null) {
       String errMsg = "Dummy participant:" + failHost + " seems not running";
       LOG.error(errMsg);
     } else {
       // System.err.println("try to stop participant: " +
       // result._manager.getInstanceName());
-      NodeOpArg arg = new NodeOpArg(result._manager, result._thread);
-      TestCommand command = new TestCommand(CommandType.STOP, new TestTrigger(beginTime), arg);
-      List<TestCommand> commandList = new ArrayList<TestCommand>();
-      commandList.add(command);
-      TestExecutor.executeTestAsync(commandList, ZK_ADDR);
+      // NodeOpArg arg = new NodeOpArg(result._manager, result._thread);
+      // TestCommand command = new TestCommand(CommandType.STOP, new TestTrigger(beginTime), arg);
+      // List<TestCommand> commandList = new ArrayList<TestCommand>();
+      // commandList.add(command);
+      // TestExecutor.executeTestAsync(commandList, ZK_ADDR);
+      participant.syncStop();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 f7a75a6..a81e35b 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
@@ -26,15 +26,17 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
+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.mock.participant.MockParticipant;
 import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.builder.CustomModeISBuilder;
 import org.apache.helix.tools.ClusterSetup;
@@ -45,7 +47,7 @@ import org.testng.annotations.Test;
 
 public class TestDrop extends ZkIntegrationTestBase {
   @Test
-  public void testDropErrorPartitionAutoIS() throws Exception {
+  public void testDropResourceWithErrorPartitionSemiAuto() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);
     String className = TestHelper.getTestClassName();
     String methodName = TestHelper.getTestMethodName();
@@ -54,7 +56,7 @@ public class TestDrop extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -66,7 +68,8 @@ public class TestDrop extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
@@ -78,11 +81,10 @@ public class TestDrop extends ZkIntegrationTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition(
-                errTransitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -108,19 +110,16 @@ public class TestDrop extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
-    // wait for all zk callbacks done
-    // Thread.sleep(1000);
-    // controller.syncStop();
-    // for (int i = 0; i < 5; i++)
-    // {
-    // participants[i].syncStop();
-    // }
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test
-  public void testDropErrorPartitionFailedAutoIS() throws Exception {
+  public void testFailToDropResourceWithErrorPartitionSemiAuto() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);
     String className = TestHelper.getTestClassName();
     String methodName = TestHelper.getTestMethodName();
@@ -129,7 +128,7 @@ public class TestDrop extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -141,7 +140,8 @@ public class TestDrop extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
@@ -153,11 +153,10 @@ public class TestDrop extends ZkIntegrationTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition(
-                errTransitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -192,19 +191,16 @@ public class TestDrop extends ZkIntegrationTestBase {
     Assert.assertEquals(disabledPartitions.get(0), "TestDB0_4");
 
     // clean up
-    // wait for all zk callbacks done
-    // Thread.sleep(1000);
-    // controller.syncStop();
-    // for (int i = 0; i < 5; i++)
-    // {
-    // participants[i].syncStop();
-    // }
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   @Test
-  public void testDropErrorPartitionCustomIS() throws Exception {
+  public void testDropResourceWithErrorPartitionCustom() throws Exception {
     // Logger.getRootLogger().setLevel(Level.INFO);
     String className = TestHelper.getTestClassName();
     String methodName = TestHelper.getTestMethodName();
@@ -213,7 +209,7 @@ public class TestDrop extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -240,7 +236,8 @@ public class TestDrop extends ZkIntegrationTestBase {
     accessor.setProperty(keyBuiler.idealStates("TestDB0"), isBuilder.build());
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
@@ -251,11 +248,10 @@ public class TestDrop extends ZkIntegrationTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       if (i == 0) {
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR, new ErrTransition(
-                errTransitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errTransitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -279,6 +275,12 @@ public class TestDrop extends ZkIntegrationTestBase {
             clusterName));
     Assert.assertTrue(result, "Should be empty exeternal-view");
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -292,7 +294,7 @@ public class TestDrop extends ZkIntegrationTestBase {
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -304,14 +306,15 @@ public class TestDrop extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -351,13 +354,74 @@ public class TestDrop extends ZkIntegrationTestBase {
         "schemata externalView should be empty but was \"" + extView + "\"");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
-    for (int i = 0; i < 5; i++) {
+    for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
+
+  /**
+   * Drop a single partition in a resource of semi-auto mode
+   */
+  @Test
+  public void testDropSinglePartitionSemiAuto() throws Exception {
+    String className = TestHelper.getTestClassName();
+    String methodName = TestHelper.getTestMethodName();
+    String clusterName = className + "_" + methodName;
+    int n = 2;
+
+    System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
+
+    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
+        "localhost", // participant name prefix
+        "TestDB", // resource name prefix
+        1, // resources
+        4, // partitions per resource
+        n, // number of nodes
+        2, // replicas
+        "MasterSlave", true); // do rebalance
+
+    ZkBaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_gZkClient);
+    ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor);
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
+    // start participants
+    MockParticipantManager[] participants = new MockParticipantManager[n];
+    for (int i = 0; i < n; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].syncStart();
+    }
+
+    boolean result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                clusterName));
+    Assert.assertTrue(result);
+
+    // remove one partition from ideal-state should drop that partition
+    String partitionToDrop = "TestDB0_1";
+    IdealState idealState = accessor.getProperty(keyBuilder.idealStates("TestDB0"));
+    idealState.getRecord().getListFields().remove(partitionToDrop);
+    idealState.getRecord().getMapFields().remove(partitionToDrop);
+    accessor.setProperty(keyBuilder.idealStates("TestDB0"), idealState);
+
+    result =
+        ClusterStateVerifier
+            .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
+                clusterName));
+    Assert.assertTrue(result);
+
+    ExternalView externalView = accessor.getProperty(keyBuilder.externalView("TestDB0"));
+    Assert.assertFalse(externalView.getPartitionSet().contains(partitionToDrop),
+        "TestDB0_0 should be dropped since it's not in ideal-state");
+    System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 ba88370..0d02d12 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.TestHelper.StartCMResult;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -59,9 +59,8 @@ public class TestDropResource extends ZkStandAloneCMTestBaseWithPropertyServerCh
 
     String hostToKill = "localhost_12920";
 
-    _startCMResultMap.get(hostToKill)._manager.disconnect();
+    _participants[2].syncStop();
     Thread.sleep(1000);
-    _startCMResultMap.get(hostToKill)._thread.interrupt();
 
     String command = "-zkSvr " + ZK_ADDR + " -dropResource " + CLUSTER_NAME + " " + "MyDB2";
     ClusterSetup.processCommandLineArgs(command.split(" "));
@@ -70,8 +69,8 @@ public class TestDropResource extends ZkStandAloneCMTestBaseWithPropertyServerCh
         TestHelper.<String> setOf("localhost_12918", "localhost_12919",
         /* "localhost_12920", */"localhost_12921", "localhost_12922"), ZK_ADDR);
 
-    StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, hostToKill);
-    _startCMResultMap.put(hostToKill, result);
+    _participants[2] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, hostToKill);
+    _participants[2].syncStart();
 
     TestHelper.verifyWithTimeout("verifyEmptyCurStateAndExtView", 30 * 1000, CLUSTER_NAME, "MyDB2",
         TestHelper.<String> setOf("localhost_12918", "localhost_12919", "localhost_12920",

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 616f63b..0291844 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,17 +26,20 @@ 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.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
+  private static Logger LOG = Logger.getLogger(TestEnablePartitionDuringDisable.class);
+
   static {
     // Logger.getRootLogger().setLevel(Level.INFO);
   }
@@ -65,8 +68,7 @@ public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
 
             ClusterSetup.processCommandLineArgs(command.split("\\s+"));
           } catch (Exception e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
+            LOG.error("Exception in cluster setup", e);
           }
 
         } else if (slaveToOfflineCnt > 0 && fromState.equals(State.from("OFFLINE"))
@@ -96,19 +98,21 @@ public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     EnablePartitionTransition transition = new EnablePartitionTransition();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
       if (instanceName.equals("localhost_12919")) {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, transition);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(transition);
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
     }
@@ -141,8 +145,6 @@ public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
     Assert.assertEquals(transition.offlineToSlave, 1, "should get 1 offlineToSlave transition");
 
     // clean up
-    // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 7e01c89..71c4339 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,10 +26,10 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.mock.participant.ErrTransition;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -38,7 +38,7 @@ public class TestErrorPartition extends ZkIntegrationTestBase {
   @Test()
   public void testErrorPartition() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START testErrorPartition() at " + new Date(System.currentTimeMillis()));
     ZKHelixAdmin tool = new ZKHelixAdmin(_gZkClient);
@@ -46,8 +46,10 @@ public class TestErrorPartition extends ZkIntegrationTestBase {
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, 10, 5, 3,
         "MasterSlave", true);
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
@@ -57,14 +59,12 @@ public class TestErrorPartition extends ZkIntegrationTestBase {
             put("SLAVE-MASTER", TestHelper.setOf("TestDB0_4"));
           }
         };
-        participants[i] =
-            new MockParticipant(clusterName, instanceName, ZK_ADDR,
-                new ErrTransition(errPartitions));
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+        participants[i].setTransition(new ErrTransition(errPartitions));
       } else {
-        participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+        participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       }
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     Map<String, Map<String, String>> errStates = new HashMap<String, Map<String, String>>();
@@ -113,7 +113,7 @@ public class TestErrorPartition extends ZkIntegrationTestBase {
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             ZK_ADDR, clusterName));
     Assert.assertTrue(result);
-    participants[0] = new MockParticipant(clusterName, "localhost_12918", ZK_ADDR);
+    participants[0] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918");
     new Thread(participants[0]).start();
 
     result =
@@ -121,6 +121,12 @@ public class TestErrorPartition extends ZkIntegrationTestBase {
             ZK_ADDR, clusterName));
     Assert.assertTrue(result);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END testErrorPartition() at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9f4c596..3e31f17 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
@@ -26,9 +26,9 @@ import java.util.List;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.MasterNbInExtViewVerifier;
@@ -42,7 +42,7 @@ public class TestExternalViewUpdates extends ZkIntegrationTestBase {
     System.out.println("START testExternalViewUpdates at " + new Date(System.currentTimeMillis()));
 
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     int resourceNb = 10;
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -53,13 +53,15 @@ public class TestExternalViewUpdates extends ZkIntegrationTestBase {
         1, // replicas
         "MasterSlave", true); // do rebalance
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -90,8 +92,11 @@ public class TestExternalViewUpdates extends ZkIntegrationTestBase {
       Assert.assertTrue(stat.getVersion() <= 2, "ExternalView should be updated at most 2 times");
     }
 
-    // TODO: need stop controller and participants
-
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
     System.out.println("END testExternalViewUpdates at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 b24f511..c7a1700 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
@@ -28,21 +28,22 @@ import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 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.mock.participant.MockJobIntf;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.participant.CustomCodeCallbackHandler;
 import org.apache.helix.participant.HelixCustomCodeRunner;
 import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
+  private static Logger LOG = Logger.getLogger(TestHelixCustomCodeRunner.class);
+
   private final String _clusterName = "CLUSTER_" + getShortClassName();
   private final int _nodeNb = 5;
   private final int _startPort = 12918;
@@ -75,8 +76,7 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
         customCodeRunner.invoke(_callback).on(ChangeType.LIVE_INSTANCE)
             .usingLeaderStandbyModel("TestParticLeader").start();
       } catch (Exception e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
+        LOG.error("Exception do pre-connect job", e);
       }
     }
 
@@ -101,15 +101,19 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
         _nodeNb, // replica
         "MasterSlave", true);
 
-    TestHelper.startController(_clusterName, "controller_0", ZK_ADDR,
-        HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, _clusterName, "controller_0");
+    controller.syncStart();
 
-    MockParticipant[] partics = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     for (int i = 0; i < _nodeNb; i++) {
       String instanceName = "localhost_" + (_startPort + i);
 
-      partics[i] = new MockParticipant(_clusterName, instanceName, ZK_ADDR, null, new MockJob());
-      partics[i].syncStart();
+      MockJob job = new MockJob();
+      participants[i] = new MockParticipantManager(ZK_ADDR, _clusterName, instanceName);
+
+      job.doPreConnectJob(participants[i]);
+      participants[i].syncStart();
     }
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -121,20 +125,26 @@ public class TestHelixCustomCodeRunner extends ZkIntegrationTestBase {
     _callback._isCallbackInvoked = false;
 
     // add a new live instance
-    ZkClient zkClient = new ZkClient(ZK_ADDR);
-    zkClient.setZkSerializer(new ZNRecordSerializer());
+    // ZkClient zkClient = new ZkClient(ZK_ADDR);
+    // zkClient.setZkSerializer(new ZNRecordSerializer());
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
+        new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
     LiveInstance newLiveIns = new LiveInstance("newLiveInstance");
-    newLiveIns.setHelixVersion("0.0.0");
+    newLiveIns.setHelixVersion("0.6.0");
     newLiveIns.setSessionId("randomSessionId");
     accessor.setProperty(keyBuilder.liveInstance("newLiveInstance"), newLiveIns);
 
     Thread.sleep(1000); // wait for the CALLBACK type callback to finish
     Assert.assertTrue(_callback._isCallbackInvoked);
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < _nodeNb; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + _clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
index 4484386..1f906d0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
@@ -33,8 +33,7 @@ import org.testng.annotations.Test;
 public class TestHelixInstanceTag extends ZkStandAloneCMTestBase {
   @Test
   public void testInstanceTag() throws Exception {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller;
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
 
     String DB2 = "TestDB2";

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 f69aeec..2761634 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,8 +2,7 @@ package org.apache.helix.integration;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.IdealState.RebalanceMode;
@@ -35,8 +34,7 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
 
   @Test
   public void testInstanceAutoJoin() throws Exception {
-    String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+    HelixManager manager = _participants[0];
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
 
     _setupTool.addResourceToCluster(CLUSTER_NAME, db2, 60, "OnlineOffline", RebalanceMode.FULL_AUTO
@@ -44,10 +42,13 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
 
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db2, 1);
     String instance2 = "localhost_279699";
-    StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2);
+    // StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2);
+    MockParticipantManager newParticipant =
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instance2);
+    newParticipant.syncStart();
 
     Thread.sleep(500);
-    Assert.assertFalse(result._thread.isAlive());
+    // Assert.assertFalse(result._thread.isAlive());
     Assert.assertTrue(null == manager.getHelixDataAccessor().getProperty(
         accessor.keyBuilder().liveInstance(instance2)));
 
@@ -55,12 +56,11 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
 
     manager.getConfigAccessor().set(scope, ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN, "true");
 
-    result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2);
-
-    StartCMResult result2 = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instance2);
+    newParticipant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instance2);
+    newParticipant.syncStart();
 
     Thread.sleep(500);
-    Assert.assertTrue(result._thread.isAlive() || result2._thread.isAlive());
+    // Assert.assertTrue(result._thread.isAlive() || result2._thread.isAlive());
     for (int i = 0; i < 20; i++) {
       if (null == manager.getHelixDataAccessor().getProperty(
           accessor.keyBuilder().liveInstance(instance2))) {
@@ -71,9 +71,6 @@ public class TestInstanceAutoJoin extends ZkStandAloneCMTestBase {
     Assert.assertTrue(null != manager.getHelixDataAccessor().getProperty(
         accessor.keyBuilder().liveInstance(instance2)));
 
-    result._manager.disconnect();
-    result2._manager.disconnect();
-    result._thread.interrupt();
-    result2._thread.interrupt();
+    newParticipant.syncStop();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 dbe183d..f2bf7f6 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
@@ -29,9 +29,9 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkUnitTestBase;
 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.ZKHelixAdmin;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
@@ -41,6 +41,7 @@ import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.helix.tools.StateModelConfigGenerator;
 import org.testng.Assert;
+import org.testng.annotations.Test;
 
 // Helix-50: integration test for generate message based on state priority
 public class TestInvalidAutoIdealState extends ZkUnitTestBase {
@@ -91,15 +92,16 @@ public class TestInvalidAutoIdealState extends ZkUnitTestBase {
     admin.setResourceIdealState(clusterName, "TestDB", idealState);
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     boolean result =
@@ -108,7 +110,7 @@ public class TestInvalidAutoIdealState extends ZkUnitTestBase {
     Assert.assertTrue(result);
 
     // make sure localhost_12919 is master on TestDB_1
-    HelixDataAccessor accessor = controller.getManager().getHelixDataAccessor();
+    HelixDataAccessor accessor = controller.getHelixDataAccessor();
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView extView = accessor.getProperty(keyBuilder.externalView(db));
     Map<String, String> stateMap = extView.getStateMap(db + "_1");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
index 268d6d0..bc7e972 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
@@ -43,9 +43,9 @@ import org.testng.annotations.Test;
 public class TestMessagePartitionStateMismatch extends ZkStandAloneCMTestBase {
   @Test
   public void testStateMismatch() throws InterruptedException {
-    String controllerName = CONTROLLER_PREFIX + "_0";
+    // String controllerName = CONTROLLER_PREFIX + "_0";
 
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller; // _startCMResultMap.get(controllerName)._manager;
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     Builder kb = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(kb.externalView(TEST_DB));


[02/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 be65ad1..00537a4 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
@@ -30,13 +30,13 @@ import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.ZkUnitTestBase;
+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.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
@@ -63,15 +63,16 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    final ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -80,9 +81,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
             .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
                 clusterName));
     Assert.assertTrue(result);
-    final ZkHelixTestManager controllerManager = controller.getManager();
-    final ZkHelixTestManager participantManagerToExpire =
-        participants[1].getManager();
+    final MockParticipantManager participantManagerToExpire = participants[1];
 
     // check controller zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -90,7 +89,8 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
       @Override
       public boolean verify() throws Exception {
         Map<String, Set<String>> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR);
-        Set<String> watchPaths = watchers.get("0x" + controllerManager.getSessionId());
+        // Set<String> watchPaths = watchers.get("0x" + controllerManager.getSessionId());
+        Set<String> watchPaths = watchers.get("0x" + controller.getSessionId());
         // System.out.println("controller watch paths: " + watchPaths);
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
@@ -118,7 +118,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
     // check HelixManager#_handlers
     // printHandlers(controllerManager);
     // printHandlers(participantManagerToExpire);
-    int controllerHandlerNb = controllerManager.getHandlers().size();
+    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");
@@ -145,7 +145,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
       @Override
       public boolean verify() throws Exception {
         Map<String, Set<String>> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR);
-        Set<String> watchPaths = watchers.get("0x" + controllerManager.getSessionId());
+        Set<String> watchPaths = watchers.get("0x" + controller.getSessionId());
         // System.out.println("controller watch paths after session expiry: " + watchPaths);
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
@@ -173,13 +173,19 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
     // check handlers
     // printHandlers(controllerManager);
     // printHandlers(participantManagerToExpire);
-    int handlerNb = controllerManager.getHandlers().size();
+    int handlerNb = controller.getHandlers().size();
     Assert.assertEquals(handlerNb, controllerHandlerNb,
         "controller callback handlers should not increase after participant session expiry");
     handlerNb = participantManagerToExpire.getHandlers().size();
     Assert.assertEquals(handlerNb, particHandlerNb,
         "participant callback handlers should not increase after participant session expiry");
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -202,15 +208,16 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    final ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -219,15 +226,16 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
             .verifyByZkCallback(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(ZK_ADDR,
                 clusterName));
     Assert.assertTrue(result);
-    final ZkHelixTestManager controllerManager = controller.getManager();
-    final ZkHelixTestManager participantManager = participants[0].getManager();
+    // final ZkHelixTestManager controllerManager = controller.getManager();
+    // final ZkHelixTestManager participantManager = participants[0].getManager();
+    final MockParticipantManager participantManager = participants[0];
 
     // wait until we get all the listeners registered
     result = TestHelper.verify(new TestHelper.Verifier() {
 
       @Override
       public boolean verify() throws Exception {
-        int controllerHandlerNb = controllerManager.getHandlers().size();
+        int controllerHandlerNb = controller.getHandlers().size();
         int particHandlerNb = participantManager.getHandlers().size();
         if (controllerHandlerNb == 9 && particHandlerNb == 2)
           return true;
@@ -236,21 +244,21 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
       }
     }, 1000);
 
-    int controllerHandlerNb = controllerManager.getHandlers().size();
+    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(controllerManager));
+            + controllerHandlerNb + ", " + printHandlers(controller));
     Assert.assertEquals(particHandlerNb, 2,
         "HelixParticipant should have 2 (msg+cur-state) callback handlers, but was "
             + particHandlerNb + ", " + printHandlers(participantManager));
 
     // expire controller
     System.out.println("Expiring controller session...");
-    String oldSessionId = controllerManager.getSessionId();
+    String oldSessionId = controller.getSessionId();
 
-    ZkTestHelper.expireSession(controllerManager.getZkClient());
-    String newSessionId = controllerManager.getSessionId();
+    ZkTestHelper.expireSession(controller.getZkClient());
+    String newSessionId = controller.getSessionId();
     System.out.println("Expired controller session. oldSessionId: " + oldSessionId
         + ", newSessionId: " + newSessionId);
 
@@ -265,7 +273,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
       @Override
       public boolean verify() throws Exception {
         Map<String, Set<String>> watchers = ZkTestHelper.getListenersBySession(ZK_ADDR);
-        Set<String> watchPaths = watchers.get("0x" + controllerManager.getSessionId());
+        Set<String> watchPaths = watchers.get("0x" + controller.getSessionId());
         // System.out.println("controller watch paths after session expiry: " + watchPaths);
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
@@ -292,15 +300,21 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
     // check HelixManager#_handlers
     // printHandlers(controllerManager);
-    int handlerNb = controllerManager.getHandlers().size();
+    int handlerNb = controller.getHandlers().size();
     Assert.assertEquals(handlerNb, controllerHandlerNb,
         "controller callback handlers should not increase after participant session expiry, but was "
-            + printHandlers(controllerManager));
+            + printHandlers(controller));
     handlerNb = participantManager.getHandlers().size();
     Assert.assertEquals(handlerNb, particHandlerNb,
         "participant callback handlers should not increase after participant session expiry, but was "
             + printHandlers(participantManager));
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
@@ -319,18 +333,20 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
         2, // replicas
         "MasterSlave", true);
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", zkAddr);
+    final ClusterControllerManager controller =
+        new ClusterControllerManager(zkAddr, clusterName, "controller_0");
     controller.syncStart();
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipant(clusterName, instanceName, zkAddr, null);
+      participants[i] = new MockParticipantManager(zkAddr, clusterName, instanceName);
       participants[i].syncStart();
 
       // register a controller listener on participant_0
       if (i == 0) {
-        ZkHelixTestManager manager = participants[0].getManager();
+        // ZkHelixTestManager manager = participants[0].getManager();
+        MockParticipantManager manager = participants[0];
         manager.addCurrentStateChangeListener(new CurrentStateChangeListener() {
           @Override
           public void onStateChange(String instanceName, List<CurrentState> statesInfo,
@@ -349,7 +365,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
                 clusterName));
     Assert.assertTrue(result);
 
-    ZkHelixTestManager participantToExpire = participants[0].getManager();
+    MockParticipantManager participantToExpire = participants[0];
     String oldSessionId = participantToExpire.getSessionId();
     PropertyKey.Builder keyBuilder = new PropertyKey.Builder(clusterName);
 
@@ -474,11 +490,18 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
             "Should have no exist-watches. exist-watches on CURRENTSTATE/{oldSessionId} and CURRENTSTATE/{oldSessionId}/TestDB0 should be cleared during handleNewSession");
 
     // Thread.sleep(1000);
+
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < n; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
 
   // debug code
-  static String printHandlers(ZkHelixTestManager manager) {
+  static String printHandlers(ZkTestManager manager) {
     StringBuilder sb = new StringBuilder();
     List<CallbackHandler> handlers = manager.getHandlers();
     sb.append(manager.getInstanceName() + " has " + handlers.size() + " cb-handlers. [");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java
index d04fbfd..9188e61 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/ZkIntegrationTestBase.java
@@ -19,14 +19,13 @@ package org.apache.helix.integration;
  * under the License.
  */
 
-import java.util.Map;
 import java.util.logging.Level;
 
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -37,7 +36,6 @@ import org.apache.helix.model.builder.ConfigScopeBuilder;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.util.ZKClientPool;
 import org.apache.log4j.Logger;
-import org.testng.Assert;
 import org.testng.AssertJUnit;
 import org.testng.annotations.AfterSuite;
 import org.testng.annotations.BeforeSuite;
@@ -85,7 +83,7 @@ public class ZkIntegrationTestBase {
 
   protected String getCurrentLeader(ZkClient zkClient, String clusterName) {
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(zkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
     LiveInstance leader = accessor.getProperty(keyBuilder.controllerLeader());
@@ -95,49 +93,6 @@ public class ZkIntegrationTestBase {
     return leader.getInstanceName();
   }
 
-  /**
-   * Stop current leader and returns the new leader
-   * @param zkClient
-   * @param clusterName
-   * @param startCMResultMap
-   * @return
-   */
-  protected String stopCurrentLeader(ZkClient zkClient, String clusterName,
-      Map<String, StartCMResult> startCMResultMap) {
-    String leader = getCurrentLeader(zkClient, clusterName);
-    Assert.assertTrue(leader != null);
-    System.out.println("stop leader: " + leader + " in " + clusterName);
-    Assert.assertTrue(leader != null);
-
-    StartCMResult result = startCMResultMap.remove(leader);
-    Assert.assertTrue(result._manager != null);
-    result._manager.disconnect();
-
-    Assert.assertTrue(result._thread != null);
-    result._thread.interrupt();
-
-    boolean isNewLeaderElected = false;
-    String newLeader = null;
-    try {
-      for (int i = 0; i < 5; i++) {
-        Thread.sleep(1000);
-        newLeader = getCurrentLeader(zkClient, clusterName);
-        if (!newLeader.equals(leader)) {
-          isNewLeaderElected = true;
-          System.out.println("new leader elected: " + newLeader + " in " + clusterName);
-          break;
-        }
-      }
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-    if (isNewLeaderElected == false) {
-      System.out.println("fail to elect a new leader in " + clusterName);
-    }
-    AssertJUnit.assertTrue(isNewLeaderElected);
-    return newLeader;
-  }
-
   protected void enableHealthCheck(String clusterName) {
     ConfigScope scope = new ConfigScopeBuilder().forCluster(clusterName).build();
     new ConfigAccessor(_gZkClient).set(scope, "healthChange" + ".enabled", "" + true);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 e759fc7..5d169d5 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
@@ -20,16 +20,9 @@ package org.apache.helix.integration;
  */
 
 import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.manager.zk.ZkClient;
+
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -57,8 +50,8 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
   protected final String CLASS_NAME = getShortClassName();
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
-  protected Map<String, StartCMResult> _startCMResultMap = new HashMap<String, StartCMResult>();
-  protected ZkClient _zkClient;
+  protected MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
+  protected ClusterControllerManager _controller;
 
   int _replica = 3;
 
@@ -67,11 +60,9 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
     String namespace = "/" + CLUSTER_NAME;
-    if (_zkClient.exists(namespace)) {
-      _zkClient.deleteRecursive(namespace);
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
     }
     _setupTool = new ClusterSetup(ZK_ADDR);
 
@@ -87,21 +78,14 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      if (_startCMResultMap.get(instanceName) != null) {
-        LOG.error("fail to start particpant:" + instanceName
-            + "(participant with same name already exists)");
-      } else {
-        StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName);
-        _startCMResultMap.put(instanceName, result);
-      }
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].syncStart();
     }
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult startResult =
-        TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    _startCMResultMap.put(controllerName, startResult);
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
 
     boolean result =
         ClusterStateVerifier
@@ -119,30 +103,11 @@ public class ZkStandAloneCMTestBase extends ZkIntegrationTestBase {
      * shutdown order: 1) disconnect the controller 2) disconnect participants
      */
 
-    StartCMResult result;
-    Iterator<Entry<String, StartCMResult>> it = _startCMResultMap.entrySet().iterator();
-    while (it.hasNext()) {
-      String instanceName = it.next().getKey();
-      if (instanceName.startsWith(CONTROLLER_PREFIX)) {
-        result = _startCMResultMap.get(instanceName);
-        result._manager.disconnect();
-        result._thread.interrupt();
-        it.remove();
-      }
-    }
-
-    Thread.sleep(100);
-    it = _startCMResultMap.entrySet().iterator();
-    while (it.hasNext()) {
-      String instanceName = it.next().getKey();
-      result = _startCMResultMap.get(instanceName);
-      result._manager.disconnect();
-      result._thread.interrupt();
-      it.remove();
+    _controller.syncStop();
+    for (int i = 0; i < NODE_NR; i++) {
+      _participants[i].syncStop();
     }
 
-    _zkClient.close();
-    // logger.info("END at " + new Date(System.currentTimeMillis()));
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java
index f19e5dd..c6fbea6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/ZkStandAloneCMTestBaseWithPropertyServerCheck.java
@@ -23,8 +23,11 @@ import java.util.List;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.restlet.ZKPropertyTransferServer;
 import org.apache.helix.controller.restlet.ZkPropertyTransferClient;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.StatusUpdate;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
@@ -36,6 +39,7 @@ import org.testng.annotations.BeforeClass;
  */
 
 public class ZkStandAloneCMTestBaseWithPropertyServerCheck extends ZkStandAloneCMTestBase {
+  @Override
   @BeforeClass
   public void beforeClass() throws Exception {
     ZKPropertyTransferServer.PERIOD = 500;
@@ -44,19 +48,20 @@ public class ZkStandAloneCMTestBaseWithPropertyServerCheck extends ZkStandAloneC
     super.beforeClass();
 
     Thread.sleep(1000);
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    Builder kb = accessor.keyBuilder();
+
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      if (_startCMResultMap.get(instanceName) != null) {
-        HelixDataAccessor accessor =
-            _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor();
-        Builder kb = accessor.keyBuilder();
-        List<StatusUpdate> statusUpdates =
-            accessor.getChildValues(kb.stateTransitionStatus(instanceName,
-                _startCMResultMap.get(instanceName)._manager.getSessionId(), TEST_DB));
+      String instanceName = _participants[i].getInstanceName();
+      List<StatusUpdate> statusUpdates =
+          accessor.getChildValues(kb.stateTransitionStatus(instanceName,
+              _participants[i].getSessionId(), TEST_DB));
+
         for (int j = 0; j < 10; j++) {
           statusUpdates =
               accessor.getChildValues(kb.stateTransitionStatus(instanceName,
-                  _startCMResultMap.get(instanceName)._manager.getSessionId(), TEST_DB));
+                _participants[i].getSessionId(), TEST_DB));
           if (statusUpdates.size() == 0) {
             Thread.sleep(500);
           } else {
@@ -70,10 +75,10 @@ public class ZkStandAloneCMTestBaseWithPropertyServerCheck extends ZkStandAloneC
           Assert
               .assertTrue(update.getRecord().getSimpleField(ZKPropertyTransferServer.SERVER) != null);
         }
-      }
     }
   }
 
+  @Override
   @AfterClass
   public void afterClass() throws Exception {
     super.afterClass();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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
index e0da9fb..b8f0f2b 100644
--- 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
@@ -22,12 +22,14 @@ package org.apache.helix.integration.manager;
 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.ControllerManager;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.log4j.Logger;
 
-public class ClusterControllerManager extends ControllerManager implements Runnable, ZkTestManager {
+public class ClusterControllerManager extends ZKHelixManager implements Runnable, ZkTestManager {
   private static Logger LOG = Logger.getLogger(ClusterControllerManager.class);
 
   private final CountDownLatch _startCountDown = new CountDownLatch(1);
@@ -35,7 +37,7 @@ public class ClusterControllerManager extends ControllerManager implements Runna
   private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
 
   public ClusterControllerManager(String zkAddr, String clusterName, String controllerName) {
-    super(zkAddr, clusterName, controllerName);
+    super(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr);
   }
 
   public void syncStop() {
@@ -43,8 +45,7 @@ public class ClusterControllerManager extends ControllerManager implements Runna
     try {
       _waitStopFinishCountDown.await();
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted waiting for finish", e);
     }
   }
 
@@ -54,8 +55,7 @@ public class ClusterControllerManager extends ControllerManager implements Runna
     try {
       _startCountDown.await();
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted waiting for start", e);
     }
   }
 
@@ -84,4 +84,7 @@ public class ClusterControllerManager extends ControllerManager implements Runna
     return _handlers;
   }
 
+  public List<HelixTimerTask> getControllerTimerTasks() {
+    return _controllerTimerTasks;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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
index 751c3cb..44d0957 100644
--- 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
@@ -22,12 +22,15 @@ package org.apache.helix.integration.manager;
 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.DistributedControllerManager;
+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 DistributedControllerManager implements Runnable,
+public class ClusterDistributedController extends ZKHelixManager implements Runnable,
     ZkTestManager {
   private static Logger LOG = Logger.getLogger(ClusterDistributedController.class);
 
@@ -36,7 +39,7 @@ public class ClusterDistributedController extends DistributedControllerManager i
   private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
 
   public ClusterDistributedController(String zkAddr, String clusterName, String controllerName) {
-    super(zkAddr, clusterName, controllerName);
+    super(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT, zkAddr);
   }
 
   public void syncStop() {
@@ -44,8 +47,7 @@ public class ClusterDistributedController extends DistributedControllerManager i
     try {
       _waitStopFinishCountDown.await();
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted waiting for finish", e);
     }
   }
 
@@ -55,14 +57,18 @@ public class ClusterDistributedController extends DistributedControllerManager i
     try {
       _startCountDown.await();
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      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();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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
index 8249f4a..34efe34 100644
--- 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
@@ -22,18 +22,20 @@ package org.apache.helix.integration.manager;
 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.ParticipantManager;
+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.MockJobIntf;
 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 ParticipantManager implements Runnable, ZkTestManager {
+public class MockParticipantManager extends ZKHelixManager implements Runnable, ZkTestManager {
   private static Logger LOG = Logger.getLogger(MockParticipantManager.class);
 
   private final CountDownLatch _startCountDown = new CountDownLatch(1);
@@ -43,7 +45,7 @@ public class MockParticipantManager extends ParticipantManager implements Runnab
   private final MockMSModelFactory _msModelFactory = new MockMSModelFactory(null);
 
   public MockParticipantManager(String zkAddr, String clusterName, String instanceName) {
-    super(zkAddr, clusterName, instanceName);
+    super(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr);
   }
 
   public void setTransition(MockTransition transition) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 b5ef255..aa00a8d 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
@@ -22,14 +22,16 @@ package org.apache.helix.integration.manager;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.helix.HelixManager;
+import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.DistributedControllerManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.model.LiveInstance;
@@ -61,11 +63,12 @@ public class TestDistributedControllerManager extends ZkIntegrationTestBase {
         2, // replicas
         "MasterSlave", true); // do rebalance
 
-    DistributedControllerManager[] distributedControllers = new DistributedControllerManager[n];
+    HelixManager[] distributedControllers = new HelixManager[n];
     for (int i = 0; i < n; i++) {
       int port = 12918 + i;
       distributedControllers[i] =
-          new DistributedControllerManager(ZK_ADDR, clusterName, "localhost_" + port);
+          new ZKHelixManager(clusterName, "localhost_" + port, InstanceType.CONTROLLER_PARTICIPANT,
+              ZK_ADDR);
       distributedControllers[i].getStateMachineEngine().registerStateModelFactory("MasterSlave",
           new MockMSModelFactory());
       distributedControllers[i].connect();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 a818fd3..82f583f 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
@@ -24,6 +24,8 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.helix.HelixManager;
+import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyPathConfig;
@@ -33,9 +35,8 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.integration.ZkIntegrationTestBase;
-import org.apache.helix.manager.zk.ControllerManager;
-import org.apache.helix.manager.zk.ParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.participant.MockMSModelFactory;
@@ -70,13 +71,14 @@ public class TestParticipantManager extends ZkIntegrationTestBase {
         1, // replicas
         "MasterSlave", true); // do rebalance
 
-    ParticipantManager participant =
-        new ParticipantManager(ZK_ADDR, clusterName, "localhost_12918");
+    HelixManager participant =
+        new ZKHelixManager(clusterName, "localhost_12918", InstanceType.PARTICIPANT, ZK_ADDR);
     participant.getStateMachineEngine().registerStateModelFactory("MasterSlave",
         new MockMSModelFactory());
     participant.connect();
 
-    ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0");
+    HelixManager controller =
+        new ZKHelixManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR);
     controller.connect();
 
     boolean result =
@@ -121,8 +123,9 @@ public class TestParticipantManager extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.connect();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
@@ -151,7 +154,7 @@ public class TestParticipantManager extends ZkIntegrationTestBase {
     Assert.assertNotSame(newSessionId, oldSessionId);
 
     // cleanup
-    controller.disconnect();
+    controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }
@@ -207,8 +210,9 @@ public class TestParticipantManager extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     // start controller
-    ControllerManager controller = new ControllerManager(ZK_ADDR, clusterName, "controller_0");
-    controller.connect();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
@@ -245,7 +249,7 @@ public class TestParticipantManager extends ZkIntegrationTestBase {
     Assert.assertTrue(errString.indexOf("InterruptedException") != -1);
 
     // cleanup
-    controller.disconnect();
+    controller.syncStop();
     for (int i = 0; i < n; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java b/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java
index 1b9ee62..2aa4544 100644
--- a/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java
+++ b/helix-core/src/test/java/org/apache/helix/josql/TestJosqlProcessor.java
@@ -40,8 +40,8 @@ public class TestJosqlProcessor extends ZkStandAloneCMTestBase {
     "integrationTest"
   })
   public void testJosqlQuery() throws Exception {
-    HelixManager manager =
-        ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager;
+    HelixManager manager = _participants[0];
+    // ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager;
 
     // Find the instance name that contains partition TestDB_2 and state is 'MASTER'
     String SQL =
@@ -183,8 +183,8 @@ public class TestJosqlProcessor extends ZkStandAloneCMTestBase {
 
   @Test(groups = ("unitTest"))
   public void testOrderby() throws Exception {
-    HelixManager manager =
-        ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager;
+    HelixManager manager = _participants[0];
+    // ((TestHelper.StartCMResult) (_startCMResultMap.values().toArray()[0]))._manager;
 
     Map<String, ZNRecord> scnMap = new HashMap<String, ZNRecord>();
     for (int i = 0; i < NODE_NR; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
index 4cef5a0..8b5b30c 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
@@ -30,10 +30,13 @@ import org.apache.helix.manager.zk.DefaultControllerMessageHandlerFactory.Defaul
 import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.log4j.Logger;
 import org.testng.AssertJUnit;
 import org.testng.annotations.Test;
 
 public class TestDefaultControllerMsgHandlerFactory {
+  private static Logger LOG = Logger.getLogger(TestDefaultControllerMsgHandlerFactory.class);
+
   @Test()
   public void testDefaultControllerMsgHandlerFactory() {
     System.out.println("START TestDefaultControllerMsgHandlerFactory at "
@@ -70,8 +73,7 @@ public class TestDefaultControllerMsgHandlerFactory {
     } catch (HelixException e) {
       exceptionCaught = true;
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted handling message", e);
     }
     AssertJUnit.assertTrue(exceptionCaught);
 
@@ -83,8 +85,7 @@ public class TestDefaultControllerMsgHandlerFactory {
     } catch (HelixException e) {
       exceptionCaught = true;
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted handling message", e);
     }
     AssertJUnit.assertFalse(exceptionCaught);
     System.out.println("END TestDefaultControllerMsgHandlerFactory at "

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 3cca10c..352cdd5 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
@@ -21,10 +21,10 @@ package org.apache.helix.manager.zk;
 
 import java.util.Date;
 
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -47,17 +47,17 @@ public class TestHandleNewSession extends ZkIntegrationTestBase {
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    ZKHelixManager manager =
-        new ZKHelixManager(clusterName, "localhost_12918", InstanceType.PARTICIPANT, ZK_ADDR);
-    manager.connect();
+    MockParticipantManager participant =
+        new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12918");
+    participant.syncStart();
 
     // Logger.getRootLogger().setLevel(Level.INFO);
-    String lastSessionId = manager.getSessionId();
+    String lastSessionId = participant.getSessionId();
     for (int i = 0; i < 3; i++) {
       // System.err.println("curSessionId: " + lastSessionId);
-      ZkTestHelper.expireSession(manager._zkClient);
+      ZkTestHelper.expireSession(participant.getZkClient());
 
-      String sessionId = manager.getSessionId();
+      String sessionId = participant.getSessionId();
       Assert.assertTrue(sessionId.compareTo(lastSessionId) > 0,
           "Session id should be increased after expiry");
       lastSessionId = sessionId;
@@ -71,7 +71,7 @@ public class TestHandleNewSession extends ZkIntegrationTestBase {
 
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("Disconnecting ...");
-    manager.disconnect();
+    participant.syncStop();
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 a49d655..547e863 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
@@ -19,10 +19,8 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -30,24 +28,20 @@ import org.testng.annotations.Test;
 public class TestLiveInstanceBounce extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
   @Test
   public void testInstanceBounce() throws Exception {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult controllerResult = _startCMResultMap.get(controllerName);
-    ZkHelixTestManager controller = controllerResult._manager;
-    int handlerSize = controller.getHandlers().size();
+    int handlerSize = _controller.getHandlers().size();
 
     for (int i = 0; i < 2; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       // kill 2 participants
-      _startCMResultMap.get(instanceName)._manager.disconnect();
-      _startCMResultMap.get(instanceName)._thread.interrupt();
+      _participants[i].syncStop();
       try {
         Thread.sleep(1000);
       } catch (InterruptedException e) {
         e.printStackTrace();
       }
       // restart the participant
-      StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName);
-      _startCMResultMap.put(instanceName, result);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].syncStart();
       Thread.sleep(100);
     }
     Thread.sleep(4000);
@@ -61,11 +55,11 @@ public class TestLiveInstanceBounce extends ZkStandAloneCMTestBaseWithPropertySe
     // and we will remove current-state listener on expired session
     // so the number of callback handlers is unchanged
     for (int j = 0; j < 10; j++) {
-      if (controller.getHandlers().size() == (handlerSize)) {
+      if (_controller.getHandlers().size() == (handlerSize)) {
         break;
       }
       Thread.sleep(400);
     }
-    Assert.assertEquals(controller.getHandlers().size(), handlerSize);
+    Assert.assertEquals(_controller.getHandlers().size(), handlerSize);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java
index c85f207..50a9a78 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKPropertyTransferServer.java
@@ -20,11 +20,10 @@ package org.apache.helix.manager.zk;
  */
 
 import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.restlet.ZKPropertyTransferServer;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -35,34 +34,28 @@ public class TestZKPropertyTransferServer extends ZkStandAloneCMTestBaseWithProp
   @Test
   public void TestControllerChange() throws Exception {
     String controllerName = CONTROLLER_PREFIX + "_0";
-    _startCMResultMap.get(controllerName)._manager.disconnect();
+    _controller.syncStop();
 
     Thread.sleep(1000);
 
     // kill controller, participant should not know about the svc url
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       HelixDataAccessor accessor =
-          _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor();
+          _participants[i].getHelixDataAccessor();
       ZKHelixDataAccessor zkAccessor = (ZKHelixDataAccessor) accessor;
       Assert.assertTrue(zkAccessor._zkPropertyTransferSvcUrl == null
           || zkAccessor._zkPropertyTransferSvcUrl.equals(""));
     }
-    _startCMResultMap.get(controllerName)._thread.interrupt();
-    _startCMResultMap.remove(controllerName);
 
-    StartCMResult startResult =
-        TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    _startCMResultMap.put(controllerName, startResult);
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
 
     Thread.sleep(1000);
 
     // create controller again, the svc url is notified to the participants
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       HelixDataAccessor accessor =
-          _startCMResultMap.get(instanceName)._manager.getHelixDataAccessor();
+          _participants[i].getHelixDataAccessor();
       ZKHelixDataAccessor zkAccessor = (ZKHelixDataAccessor) accessor;
       Assert.assertTrue(zkAccessor._zkPropertyTransferSvcUrl.equals(ZKPropertyTransferServer
           .getInstance().getWebserviceUrl()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 c099232..83dc986 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,6 +27,12 @@ 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.builder.ConfigScopeBuilder;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
@@ -35,14 +41,10 @@ import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.ZkUnitTestBase;
 import org.apache.helix.manager.MockListener;
-import org.apache.helix.model.HelixConfigScope;
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
@@ -65,6 +67,7 @@ public class TestZkClusterManager extends ZkUnitTestBase {
 
     ZKHelixManager controller =
         new ZKHelixManager(clusterName, null, InstanceType.CONTROLLER, ZK_ADDR);
+
     try {
       controller.connect();
       Assert.fail("Should throw HelixException if initial cluster structure is not setup");
@@ -193,8 +196,9 @@ public class TestZkClusterManager extends ZkUnitTestBase {
 
     // //////////////////////////////////
 
-    ZkHelixTestManager manager2 =
-        new ZkHelixTestManager(clusterName, "localhost_3", InstanceType.PARTICIPANT, ZK_ADDR);
+    MockParticipantManager manager2 =
+        new MockParticipantManager(ZK_ADDR, clusterName, "localhost_3");
+
     manager2.setLiveInstanceInfoProvider(new provider(true));
 
     manager2.connect();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 5b35148..c329e9d 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
@@ -13,8 +13,8 @@ import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.Verifier;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.LiveInstance;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.testng.Assert;
@@ -134,10 +134,11 @@ public class TestZkFlapping extends ZkUnitTestBase {
         "MasterSlave", false);
 
     final String instanceName = "localhost_12918";
-    MockParticipant participant = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+    MockParticipantManager participant =
+        new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
     participant.syncStart();
 
-    final ZkClient client = participant.getManager().getZkClient();
+    final ZkClient client = participant.getZkClient();
     final ZkStateCountListener listener = new ZkStateCountListener();
     client.subscribeStateChanges(listener);
 
@@ -212,10 +213,11 @@ public class TestZkFlapping extends ZkUnitTestBase {
         1, // replicas
         "MasterSlave", false);
 
-    ClusterController controller = new ClusterController(clusterName, "controller", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller");
     controller.syncStart();
 
-    final ZkClient client = controller.getManager().getZkClient();
+    final ZkClient client = controller.getZkClient();
     final ZkStateCountListener listener = new ZkStateCountListener();
     client.subscribeStateChanges(listener);
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 249fcea..a62e39d 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
@@ -19,13 +19,12 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.util.UUID;
 
-import org.apache.helix.InstanceType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -46,8 +45,8 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase {
         "MasterSlave", true); // do rebalance
 
     String instanceName = "localhost_" + (12918 + 0);
-    ZkHelixTestManager manager =
-        new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, ZK_ADDR);
+    MockParticipantManager manager = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+
     manager.connect();
     ZkClient zkClient = manager.getZkClient();
     ZkTestHelper.expireSession(zkClient);
@@ -69,58 +68,59 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase {
     Assert.assertFalse(manager.isConnected());
   }
 
-  @Test(enabled = false)
-  public void testDisconnectFlappingWindow() throws Exception {
-    String className = TestHelper.getTestClassName();
-    String methodName = TestHelper.getTestMethodName();
-    String instanceName = "localhost_" + (12918 + 1);
-    final String clusterName = className + "_" + methodName + UUID.randomUUID();
-
-    testDisconnectFlappingWindow2(instanceName, InstanceType.PARTICIPANT);
-    testDisconnectFlappingWindow2("admin", InstanceType.ADMINISTRATOR);
-  }
-
-  public void testDisconnectFlappingWindow2(String instanceName, InstanceType type)
-      throws Exception {
-    String className = TestHelper.getTestClassName();
-    String methodName = TestHelper.getTestMethodName();
-    final String clusterName = className + "_" + methodName + UUID.randomUUID();
-
-    TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
-        "localhost", // participant name prefix
-        "TestDB", // resource name prefix
-        1, // resources
-        10, // partitions per resource
-        5, // number of nodes
-        3, // replicas
-        "MasterSlave", true); // do rebalance
-
-    // flapping time window to 5 sec
-    System.setProperty("helixmanager.flappingTimeWindow", "15000");
-    System.setProperty("helixmanager.maxDisconnectThreshold", "7");
-    ZkHelixTestManager manager2 = new ZkHelixTestManager(clusterName, instanceName, type, ZK_ADDR);
-    manager2.connect();
-    ZkClient zkClient = manager2.getZkClient();
-    for (int i = 0; i < 3; i++) {
-      ZkTestHelper.expireSession(zkClient);
-      Thread.sleep(500);
-      Assert.assertTrue(manager2.isConnected());
-    }
-    Thread.sleep(15000);
-    // Old entries should be cleaned up
-    for (int i = 0; i < 7; i++) {
-      ZkTestHelper.expireSession(zkClient);
-      Thread.sleep(1000);
-      Assert.assertTrue(manager2.isConnected());
-    }
-    ZkTestHelper.disconnectSession(zkClient);
-    for (int i = 0; i < 20; i++) {
-      Thread.sleep(500);
-      if (!manager2.isConnected())
-        break;
-    }
-    Assert.assertFalse(manager2.isConnected());
-  }
+  // TODO test was disabled. check if it is still needed
+  // @Test(enabled = false)
+  // public void testDisconnectFlappingWindow() throws Exception {
+  // String className = TestHelper.getTestClassName();
+  // String methodName = TestHelper.getTestMethodName();
+  // String instanceName = "localhost_" + (12918 + 1);
+  // final String clusterName = className + "_" + methodName + UUID.randomUUID();
+  //
+  // testDisconnectFlappingWindow2(instanceName, InstanceType.PARTICIPANT);
+  // testDisconnectFlappingWindow2("admin", InstanceType.ADMINISTRATOR);
+  // }
+  //
+  // public void testDisconnectFlappingWindow2(String instanceName, InstanceType type)
+  // throws Exception {
+  // String className = TestHelper.getTestClassName();
+  // String methodName = TestHelper.getTestMethodName();
+  // final String clusterName = className + "_" + methodName + UUID.randomUUID();
+  //
+  // TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
+  // "localhost", // participant name prefix
+  // "TestDB", // resource name prefix
+  // 1, // resources
+  // 10, // partitions per resource
+  // 5, // number of nodes
+  // 3, // replicas
+  // "MasterSlave", true); // do rebalance
+  //
+  // // flapping time window to 5 sec
+  // System.setProperty("helixmanager.flappingTimeWindow", "15000");
+  // System.setProperty("helixmanager.maxDisconnectThreshold", "7");
+  // ZkHelixTestManager manager2 = new ZkHelixTestManager(clusterName, instanceName, type, ZK_ADDR);
+  // manager2.connect();
+  // ZkClient zkClient = manager2.getZkClient();
+  // for (int i = 0; i < 3; i++) {
+  // ZkTestHelper.expireSession(zkClient);
+  // Thread.sleep(500);
+  // Assert.assertTrue(manager2.isConnected());
+  // }
+  // Thread.sleep(15000);
+  // // Old entries should be cleaned up
+  // for (int i = 0; i < 7; i++) {
+  // ZkTestHelper.expireSession(zkClient);
+  // Thread.sleep(1000);
+  // Assert.assertTrue(manager2.isConnected());
+  // }
+  // ZkTestHelper.disconnectSession(zkClient);
+  // for (int i = 0; i < 20; i++) {
+  // Thread.sleep(500);
+  // if (!manager2.isConnected())
+  // break;
+  // }
+  // Assert.assertFalse(manager2.isConnected());
+  // }
 
   // @Test
   public void testDisconnectFlappingWindowController() throws Exception {
@@ -140,8 +140,7 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase {
     // flapping time window to 5 sec
     System.setProperty("helixmanager.flappingTimeWindow", "5000");
     System.setProperty("helixmanager.maxDisconnectThreshold", "3");
-    ZkHelixTestManager manager2 =
-        new ZkHelixTestManager(clusterName, null, InstanceType.CONTROLLER, ZK_ADDR);
+    ClusterControllerManager manager2 = new ClusterControllerManager(ZK_ADDR, clusterName, null);
     manager2.connect();
     Thread.sleep(100);
     ZkClient zkClient = manager2.getZkClient();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 4f15f90..aeb32f9 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
@@ -19,68 +19,70 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestZkStateChangeListener extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
-  @Test
+  // TODO this test has been covered by TestZkFlapping. check if still needed
+  // @Test
   public void testDisconnectHistory() throws Exception {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult controllerResult = _startCMResultMap.get(controllerName);
-    ZKHelixManager controller = controllerResult._manager;
-    ZkStateChangeListener listener1 = new ZkStateChangeListener(controller, 5000, 10);
+    // String controllerName = CONTROLLER_PREFIX + "_0";
+    // StartCMResult controllerResult = _startCMResultMap.get(controllerName);
+    // ZKHelixManager controller = (ZKHelixManager) controllerResult._manager;
+    // ZkStateChangeListener listener1 = new ZkStateChangeListener(controller, 5000, 10);
+    // ZkStateChangeListener listener1 = new ZkStateChangeListener(_controller, 5000, 10);
+
     // 11 disconnects in 5 sec
     for (int i = 0; i < 11; i++) {
       Thread.sleep(200);
-      listener1.handleStateChanged(KeeperState.Disconnected);
+      _controller.handleStateChanged(KeeperState.Disconnected);
       if (i < 10) {
-        Assert.assertTrue(controller.isConnected());
+        Assert.assertTrue(_controller.isConnected());
       } else {
-        Assert.assertFalse(controller.isConnected());
+        Assert.assertFalse(_controller.isConnected());
       }
     }
 
     // If maxDisconnectThreshold is 0 it should be set to 1
-    String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    ZKHelixManager manager = _startCMResultMap.get(instanceName)._manager;
+    // String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
+    // ZKHelixManager manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager;
 
-    ZkStateChangeListener listener2 = new ZkStateChangeListener(manager, 5000, 0);
+    // ZkStateChangeListener listener2 = new ZkStateChangeListener(_participants[0], 5000, 0);
     for (int i = 0; i < 2; i++) {
       Thread.sleep(200);
-      listener2.handleStateChanged(KeeperState.Disconnected);
+      _participants[0].handleStateChanged(KeeperState.Disconnected);
       if (i < 1) {
-        Assert.assertTrue(manager.isConnected());
+        Assert.assertTrue(_participants[0].isConnected());
       } else {
-        Assert.assertFalse(manager.isConnected());
+        Assert.assertFalse(_participants[0].isConnected());
       }
     }
 
     // If there are long time after disconnect, older history should be cleanup
-    instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1);
-    manager = _startCMResultMap.get(instanceName)._manager;
+    // instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1);
+    // manager = (ZKHelixManager) _startCMResultMap.get(instanceName)._manager;
 
-    ZkStateChangeListener listener3 = new ZkStateChangeListener(manager, 5000, 5);
+    // ZkStateChangeListener listener3 = new ZkStateChangeListener(_participants[1], 5000, 5);
     for (int i = 0; i < 3; i++) {
       Thread.sleep(200);
-      listener3.handleStateChanged(KeeperState.Disconnected);
-      Assert.assertTrue(manager.isConnected());
+      _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);
-      listener3.handleStateChanged(KeeperState.Disconnected);
-      Assert.assertTrue(manager.isConnected());
+      _participants[1].handleStateChanged(KeeperState.Disconnected);
+      Assert.assertTrue(_participants[1].isConnected());
     }
     for (int i = 0; i < 2; i++) {
       Thread.sleep(200);
-      listener3.handleStateChanged(KeeperState.Disconnected);
-      Assert.assertTrue(manager.isConnected());
+      _participants[1].handleStateChanged(KeeperState.Disconnected);
+      Assert.assertTrue(_participants[1].isConnected());
     }
-    listener3.handleStateChanged(KeeperState.Disconnected);
-    Assert.assertFalse(manager.isConnected());
+    _participants[1].handleStateChanged(KeeperState.Disconnected);
+    Assert.assertFalse(_participants[1].isConnected());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 f4566a0..c71a782 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
@@ -77,7 +77,7 @@ public class TestConfigThreadpoolSize extends ZkStandAloneCMTestBase {
   @Test
   public void TestThreadPoolSizeConfig() {
     String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+    HelixManager manager = _participants[0];
 
     ConfigAccessor accessor = manager.getConfigAccessor();
     ConfigScope scope =
@@ -91,9 +91,9 @@ public class TestConfigThreadpoolSize extends ZkStandAloneCMTestBase {
     for (int i = 0; i < NODE_NR; i++) {
       instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
 
-      _startCMResultMap.get(instanceName)._manager.getMessagingService()
-          .registerMessageHandlerFactory("TestMsg", new TestMessagingHandlerFactory());
-      _startCMResultMap.get(instanceName)._manager.getMessagingService()
+      _participants[i].getMessagingService().registerMessageHandlerFactory("TestMsg",
+          new TestMessagingHandlerFactory());
+      _participants[i].getMessagingService()
           .registerMessageHandlerFactory("TestMsg2", new TestMessagingHandlerFactory2());
 
     }
@@ -102,7 +102,7 @@ public class TestConfigThreadpoolSize extends ZkStandAloneCMTestBase {
       instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
 
       DefaultMessagingService svc =
-          (DefaultMessagingService) (_startCMResultMap.get(instanceName)._manager
+          (DefaultMessagingService) (_participants[i]
               .getMessagingService());
       HelixTaskExecutor helixExecutor = svc.getExecutor();
       ThreadPoolExecutor executor =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 9104866..a5777ab 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
@@ -35,8 +35,7 @@ import org.testng.annotations.Test;
 public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
   @Test
   public void TestThreadPoolSizeConfig() {
-    String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+    HelixManager manager = _participants[0];
     ConfigAccessor accessor = manager.getConfigAccessor();
     ConfigScope scope =
         new ConfigScopeBuilder().forCluster(manager.getClusterName()).forResource("NextDB").build();
@@ -52,11 +51,8 @@ public class TestResourceThreadpoolSize extends ZkStandAloneCMTestBase {
 
     long taskcount = 0;
     for (int i = 0; i < NODE_NR; i++) {
-      instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-
       DefaultMessagingService svc =
-          (DefaultMessagingService) (_startCMResultMap.get(instanceName)._manager
-              .getMessagingService());
+          (DefaultMessagingService) (_participants[i].getMessagingService());
       HelixTaskExecutor helixExecutor = svc.getExecutor();
       ThreadPoolExecutor executor =
           (ThreadPoolExecutor) (helixExecutor._executorMap.get(MessageType.STATE_TRANSITION + "."

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java b/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java
deleted file mode 100644
index a04a213..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/controller/ClusterController.java
+++ /dev/null
@@ -1,127 +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.util.concurrent.CountDownLatch;
-
-import org.apache.helix.InstanceType;
-import org.apache.helix.ZkHelixTestManager;
-import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.participant.DistClusterControllerStateModelFactory;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-
-public class ClusterController extends Thread {
-  private static Logger LOG = Logger.getLogger(ClusterController.class);
-
-  private final CountDownLatch _startCountDown = new CountDownLatch(1);
-  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
-  private final String _controllerMode;
-  private final String _zkAddr;
-
-  private ZkHelixTestManager _manager;
-
-  public ClusterController(String clusterName, String controllerName, String zkAddr)
-      throws Exception {
-    this(clusterName, controllerName, zkAddr, HelixControllerMain.STANDALONE.toString());
-  }
-
-  public ClusterController(String clusterName, String controllerName, String zkAddr,
-      String controllerMode) throws Exception {
-    _controllerMode = controllerMode;
-    _zkAddr = zkAddr;
-
-    if (_controllerMode.equals(HelixControllerMain.STANDALONE.toString())) {
-      _manager =
-          new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER, zkAddr);
-    } else if (_controllerMode.equals(HelixControllerMain.DISTRIBUTED.toString())) {
-      _manager =
-          new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER_PARTICIPANT,
-              zkAddr);
-    } else {
-      throw new IllegalArgumentException("Controller mode: " + controllerMode + " NOT recoginized");
-    }
-  }
-
-  public ZkHelixTestManager getManager() {
-    return _manager;
-  }
-
-  public void syncStop() {
-    if (_manager == null) {
-      LOG.warn("manager already stopped");
-      return;
-    }
-
-    _stopCountDown.countDown();
-    try {
-      _waitStopFinishCountDown.await();
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  public void syncStart() {
-    // TODO: prevent start multiple times
-
-    super.start();
-    try {
-      _startCountDown.await();
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  @Override
-  public void run() {
-    try {
-      try {
-        if (_controllerMode.equals(HelixControllerMain.STANDALONE.toString())) {
-          _manager.connect();
-        } else if (_controllerMode.equals(HelixControllerMain.DISTRIBUTED.toString())) {
-          DistClusterControllerStateModelFactory stateModelFactory =
-              new DistClusterControllerStateModelFactory(_zkAddr);
-
-          StateMachineEngine stateMach = _manager.getStateMachineEngine();
-          stateMach.registerStateModelFactory("LeaderStandby", stateModelFactory);
-          _manager.connect();
-        }
-      } catch (Exception e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      } finally {
-        _startCountDown.countDown();
-        _stopCountDown.await();
-      }
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } finally {
-      synchronized (_manager) {
-        _manager.disconnect();
-        _manager = null;
-      }
-      _waitStopFinishCountDown.countDown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
index 59d9a0a..31811bb 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/participant/MockHealthReportParticipant.java
@@ -33,6 +33,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.healthcheck.HealthReportProvider;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.log4j.Logger;
 
 public class MockHealthReportParticipant {
@@ -209,9 +210,9 @@ public class MockHealthReportParticipant {
 
   // NOT working for kill -9, working for kill -2/-15
   static class MockHealthReportParticipantShutdownHook extends Thread {
-    final MockParticipant _participant;
+    final MockParticipantManager _participant;
 
-    MockHealthReportParticipantShutdownHook(MockParticipant participant) {
+    MockHealthReportParticipantShutdownHook(MockParticipantManager participant) {
       _participant = participant;
     }
 
@@ -231,12 +232,11 @@ public class MockHealthReportParticipant {
 
     String instanceName = hostStr + "_" + portStr;
 
-    MockParticipant participant =
-        new MockParticipant(clusterName, instanceName, zkConnectStr, null, // new
-                                                                           // StoreAccessDiffNodeTransition(),
-                                                                           // // new
-                                                                           // StoreAccessOneNodeTransition(),
-            new MockHealthReportJob());
+    MockParticipantManager participant =
+        new MockParticipantManager(zkConnectStr, clusterName, instanceName);
+    // participant.setTransition(new StoreAccessDiffNodeTransition());
+    // participant.setTransition(new StoreAccessOneNodeTransition()));
+    // new MockHealthReportJob());
     Runtime.getRuntime().addShutdownHook(new MockHealthReportParticipantShutdownHook(participant));
 
     // Espresso_driver.py will consume this

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java b/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java
deleted file mode 100644
index 4030b99..0000000
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/MockParticipant.java
+++ /dev/null
@@ -1,181 +0,0 @@
-package org.apache.helix.mock.participant;
-
-/*
- * 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.concurrent.CountDownLatch;
-
-import org.apache.helix.InstanceType;
-import org.apache.helix.ZkHelixTestManager;
-import org.apache.helix.mock.participant.DummyProcess.DummyLeaderStandbyStateModelFactory;
-import org.apache.helix.mock.participant.DummyProcess.DummyOnlineOfflineStateModelFactory;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.helix.participant.statemachine.StateModelFactory;
-import org.apache.log4j.Logger;
-
-public class MockParticipant extends Thread {
-  private static Logger LOG = Logger.getLogger(MockParticipant.class);
-  private final String _clusterName;
-  private final String _instanceName;
-  // private final String _zkAddr;
-
-  private final CountDownLatch _startCountDown = new CountDownLatch(1);
-  private final CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private final CountDownLatch _waitStopFinishCountDown = new CountDownLatch(1);
-
-  private final ZkHelixTestManager _manager;
-  private final StateModelFactory _msModelFactory;
-  private final MockJobIntf _job;
-
-  public MockParticipant(String clusterName, String instanceName, String zkAddr) throws Exception {
-    this(clusterName, instanceName, zkAddr, null, null);
-  }
-
-  public MockParticipant(String clusterName, String instanceName, String zkAddr,
-      MockTransition transition) throws Exception {
-    this(clusterName, instanceName, zkAddr, transition, null);
-  }
-
-  public MockParticipant(String clusterName, String instanceName, String zkAddr,
-      MockTransition transition, MockJobIntf job) throws Exception {
-    _clusterName = clusterName;
-    _instanceName = instanceName;
-    _msModelFactory = new MockMSModelFactory(transition);
-
-    _manager =
-        new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr);
-    _job = job;
-  }
-
-  public MockParticipant(StateModelFactory factory, String clusterName, String instanceName,
-      String zkAddr, MockJobIntf job) throws Exception {
-    _clusterName = clusterName;
-    _instanceName = instanceName;
-    _msModelFactory = factory;
-
-    _manager =
-        new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr);
-    _job = job;
-  }
-
-  public StateModelFactory getStateModelFactory() {
-    return _msModelFactory;
-  }
-
-  public MockParticipant(ZkHelixTestManager manager, MockTransition transition) {
-    _clusterName = manager.getClusterName();
-    _instanceName = manager.getInstanceName();
-    _manager = manager;
-
-    _msModelFactory = new MockMSModelFactory(transition);
-    _job = null;
-  }
-
-  public void setTransition(MockTransition transition) {
-    if (_msModelFactory instanceof MockMSModelFactory) {
-      ((MockMSModelFactory) _msModelFactory).setTrasition(transition);
-    }
-  }
-
-  public ZkHelixTestManager getManager() {
-    return _manager;
-  }
-
-  public String getInstanceName() {
-    return _instanceName;
-  }
-
-  public String getClusterName() {
-    return _clusterName;
-  }
-
-  public void syncStop() {
-    _stopCountDown.countDown();
-    try {
-      _waitStopFinishCountDown.await();
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-
-    // synchronized (_manager)
-    // {
-    // _manager.disconnect();
-    // }
-  }
-
-  public void syncStart() {
-    super.start();
-    try {
-      _startCountDown.await();
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  @Override
-  public void run() {
-    try {
-      StateMachineEngine stateMach = _manager.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);
-      // MockBootstrapModelFactory bootstrapFactory = new MockBootstrapModelFactory();
-      // stateMach.registerStateModelFactory("Bootstrap", bootstrapFactory);
-
-      if (_job != null) {
-        _job.doPreConnectJob(_manager);
-      }
-
-      _manager.connect();
-      _startCountDown.countDown();
-
-      if (_job != null) {
-        _job.doPostConnectJob(_manager);
-      }
-
-      _stopCountDown.await();
-    } catch (InterruptedException e) {
-      String msg =
-          "participant: " + _instanceName + ", " + Thread.currentThread().getName()
-              + " is interrupted";
-      LOG.info(msg);
-      System.err.println(msg);
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } finally {
-      _startCountDown.countDown();
-
-      synchronized (_manager) {
-        _manager.disconnect();
-      }
-      _waitStopFinishCountDown.countDown();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 6de77b2..d97b22a 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
@@ -261,4 +261,10 @@ public class MockZKHelixManager implements HelixManager {
     return null;
   }
 
+  @Override
+  public void addControllerMessageListener(MessageListener listener) {
+    // TODO Auto-generated method stub
+
+  }
+
 }


[07/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 afd35e6..b71304a 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
@@ -19,18 +19,15 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedList;
 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.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
@@ -40,12 +37,12 @@ import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixConstants.ChangeType;
 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.HelixConstants.ChangeType;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
 import org.apache.helix.InstanceType;
@@ -59,141 +56,222 @@ 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.restlet.ZKPropertyTransferServer;
+import org.apache.helix.controller.GenericHelixController;
 import org.apache.helix.healthcheck.HealthStatsAggregationTask;
 import org.apache.helix.healthcheck.HealthStatsAggregator;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.healthcheck.ParticipantHealthReportTask;
 import org.apache.helix.messaging.DefaultMessagingService;
-import org.apache.helix.messaging.handling.MessageHandlerFactory;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.helix.model.builder.ConfigScopeBuilder;
 import org.apache.helix.monitoring.ZKPathDataDumpTask;
-import org.apache.helix.participant.DistClusterControllerElection;
 import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.helix.participant.StateMachineEngine;
-import org.apache.helix.participant.statemachine.ScheduledTaskStateModelFactory;
 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.Watcher.Event.EventType;
+import org.apache.zookeeper.ZooKeeper.States;
+
+public class ZKHelixManager implements HelixManager, IZkStateListener {
+  private static Logger LOG = Logger.getLogger(ZKHelixManager.class);
+
+  public static final int FLAPPING_TIME_WINDIOW = 300000; // Default to 300 sec
+  public static final int MAX_DISCONNECT_THRESHOLD = 5;
+  public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "allowParticipantAutoJoin";
 
-public class ZKHelixManager implements HelixManager {
-  private static Logger logger = Logger.getLogger(ZKHelixManager.class);
-  private static final int RETRY_LIMIT = 3;
-  private static final int CONNECTIONTIMEOUT = 60 * 1000;
+  protected final String _zkAddress;
   private final String _clusterName;
   private final String _instanceName;
-  private final String _zkConnectString;
-  private static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000;
-  private ZKHelixDataAccessor _helixAccessor;
-  private ConfigAccessor _configAccessor;
-  protected ZkClient _zkClient;
-  protected final List<CallbackHandler> _handlers = new ArrayList<CallbackHandler>();
-  private final ZkStateChangeListener _zkStateChangeListener;
   private final InstanceType _instanceType;
-  volatile String _sessionId;
-  private Timer _timer;
-  private CallbackHandler _leaderElectionHandler;
-  private ParticipantHealthReportCollectorImpl _participantHealthCheckInfoCollector;
-  private ParticipantHealthReportTask _participantHealthReportTask;
-  private final DefaultMessagingService _messagingService;
-  private ZKHelixAdmin _managementTool;
-  private final String _version;
+  private final int _sessionTimeout;
+  private final List<PreConnectCallback> _preConnectCallbacks;
+  protected final List<CallbackHandler> _handlers;
   private final HelixManagerProperties _properties;
-  private final StateMachineEngine _stateMachEngine;
-  private int _sessionTimeout;
-  private ZkHelixPropertyStore<ZNRecord> _helixPropertyStore;
-  private final List<HelixTimerTask> _controllerTimerTasks;
+
+  /**
+   * helix version#
+   */
+  private final String _version;
+
+  protected ZkClient _zkclient = null;
+  private final DefaultMessagingService _messagingService;
+
   private BaseDataAccessor<ZNRecord> _baseDataAccessor;
-  List<PreConnectCallback> _preConnectCallbacks = new LinkedList<PreConnectCallback>();
-  ZKPropertyTransferServer _transferServer = null;
-  int _flappingTimeWindowMs;
-  int _maxDisconnectThreshold;
-  public static final int FLAPPING_TIME_WINDIOW = 300000; // Default to 300 sec
-  public static final int MAX_DISCONNECT_THRESHOLD = 5;
-  LiveInstanceInfoProvider _liveInstanceInfoProvider = null;
-  public static final String ALLOW_PARTICIPANT_AUTO_JOIN = "allowParticipantAutoJoin";
+  private ZKHelixDataAccessor _dataAccessor;
+  private final Builder _keyBuilder;
+  private ConfigAccessor _configAccessor;
+  private ZkHelixPropertyStore<ZNRecord> _helixPropertyStore;
+  protected LiveInstanceInfoProvider _liveInstanceInfoProvider = null;
 
-  public ZKHelixManager(String clusterName, String instanceName, InstanceType instanceType,
-      String zkConnectString) {
-    logger.info("Create a zk-based cluster manager. clusterName:" + clusterName + ", instanceName:"
-        + instanceName + ", type:" + instanceType + ", zkSvr:" + zkConnectString);
-    _flappingTimeWindowMs = FLAPPING_TIME_WINDIOW;
-    try {
-      _flappingTimeWindowMs =
-          Integer.parseInt(System.getProperty("helixmanager.flappingTimeWindow", ""
-              + FLAPPING_TIME_WINDIOW));
-    } catch (NumberFormatException e) {
-      logger.warn("Exception while parsing helixmanager.flappingTimeWindow: "
-          + System.getProperty("helixmanager.flappingTimeWindow", "" + FLAPPING_TIME_WINDIOW));
-    }
-    _maxDisconnectThreshold = MAX_DISCONNECT_THRESHOLD;
-    try {
-      _maxDisconnectThreshold =
-          Integer.parseInt(System.getProperty("helixmanager.maxDisconnectThreshold", ""
-              + MAX_DISCONNECT_THRESHOLD));
-    } catch (NumberFormatException e) {
-      logger.warn("Exception while parsing helixmanager.maxDisconnectThreshold: "
-          + System
-              .getProperty("helixmanager.maxDisconnectThreshold", "" + MAX_DISCONNECT_THRESHOLD));
-    }
-    int sessionTimeoutInt = -1;
-    try {
-      sessionTimeoutInt =
-          Integer.parseInt(System.getProperty("zk.session.timeout", "" + DEFAULT_SESSION_TIMEOUT));
-    } catch (NumberFormatException e) {
-      logger.warn("Exception while parsing session timeout: "
-          + System.getProperty("zk.session.timeout", "" + DEFAULT_SESSION_TIMEOUT));
+  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 ParticipantHealthReportCollectorImpl _participantHealthInfoCollector;
+  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 ZkClient zkclient;
+    final HelixManager helixController;
+
+    public StatusDumpTask(ZkClient zkclient, HelixManager helixController) {
+      this.zkclient = zkclient;
+      this.helixController = helixController;
+    }
+
+    @Override
+    public void start() {
+      long initialDelay = 30 * 60 * 1000;
+      long period = 120 * 60 * 1000;
+      int timeThresholdNoChange = 180 * 60 * 1000;
+
+      if (_timer == null) {
+        LOG.info("Start StatusDumpTask");
+        _timer = new Timer("StatusDumpTimerTask", true);
+        _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, zkclient,
+            timeThresholdNoChange), initialDelay, period);
+      }
     }
-    if (sessionTimeoutInt > 0) {
-      _sessionTimeout = sessionTimeoutInt;
-    } else {
-      _sessionTimeout = DEFAULT_SESSION_TIMEOUT;
+
+    @Override
+    public void stop() {
+      if (_timer != null) {
+        LOG.info("Stop StatusDumpTask");
+        _timer.cancel();
+        _timer = null;
+      }
     }
+  }
+
+  public ZKHelixManager(String clusterName, String instanceName, InstanceType instanceType,
+      String zkAddress) {
+
+    LOG.info("Create a zk-based cluster manager. zkSvr: " + zkAddress + ", clusterName: "
+        + clusterName + ", instanceName: " + instanceName + ", type: " + instanceType);
+
+    _zkAddress = zkAddress;
+    _clusterName = clusterName;
+    _instanceType = instanceType;
+
     if (instanceName == null) {
       try {
         instanceName =
             InetAddress.getLocalHost().getCanonicalHostName() + "-" + instanceType.toString();
       } catch (UnknownHostException e) {
         // can ignore it
-        logger.info("Unable to get host name. Will set it to UNKNOWN, mostly ignorable", e);
+        LOG.info("Unable to get host name. Will set it to UNKNOWN, mostly ignorable", e);
         instanceName = "UNKNOWN";
       }
     }
 
-    _clusterName = clusterName;
     _instanceName = instanceName;
-    _instanceType = instanceType;
-    _zkConnectString = zkConnectString;
-    _zkStateChangeListener =
-        new ZkStateChangeListener(this, _flappingTimeWindowMs, _maxDisconnectThreshold);
-    _timer = null;
+    _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);
 
-    _properties = new HelixManagerProperties("cluster-manager-version.properties");
-    _version = _properties.getVersion();
+    /**
+     * use system property if available
+     */
+    _flappingTimeWindowMs =
+        getSystemPropertyAsInt("helixmanager.flappingTimeWindow",
+            ZKHelixManager.FLAPPING_TIME_WINDIOW);
+
+    _maxDisconnectThreshold =
+        getSystemPropertyAsInt("helixmanager.maxDisconnectThreshold",
+            ZKHelixManager.MAX_DISCONNECT_THRESHOLD);
 
-    _stateMachEngine = new HelixStateMachineEngine(this);
+    _sessionTimeout =
+        getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT);
+
+    /**
+     * instance type specific init
+     */
+    switch (instanceType) {
+    case PARTICIPANT:
+      _stateMachineEngine = new HelixStateMachineEngine(this);
+      _participantHealthInfoCollector =
+          new ParticipantHealthReportCollectorImpl(this, _instanceName);
+
+      _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
+
+      break;
+    case CONTROLLER:
+      _stateMachineEngine = null;
+      _participantHealthInfoCollector = null;
+      _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
+      _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
+
+      break;
+    case CONTROLLER_PARTICIPANT:
+      _stateMachineEngine = new HelixStateMachineEngine(this);
+      _participantHealthInfoCollector =
+          new ParticipantHealthReportCollectorImpl(this, _instanceName);
+
+      _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
 
-    // add all timer tasks
-    _controllerTimerTasks = new ArrayList<HelixTimerTask>();
-    if (_instanceType == InstanceType.CONTROLLER) {
       _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
+      _controllerTimerTasks.add(new StatusDumpTask(_zkclient, this));
+
+      break;
+    case ADMINISTRATOR:
+    case SPECTATOR:
+      _stateMachineEngine = null;
+      _participantHealthInfoCollector = null;
+      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);
+    }
+
+    return propertyDefaultValue;
+  }
+
   @Override
   public boolean removeListener(PropertyKey key, Object listener) {
-    logger.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: "
+    LOG.info("Removing listener: " + listener + " on path: " + key.getPath() + " from cluster: "
         + _clusterName + " by instance: " + _instanceName);
 
     synchronized (this) {
@@ -216,7 +294,13 @@ public class ZKHelixManager implements HelixManager {
     return true;
   }
 
-  private void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType,
+  void checkConnected() {
+    if (!isConnected()) {
+      throw new HelixException("HelixManager is not connected. Call HelixManager#connect()");
+    }
+  }
+
+  void addListener(Object listener, PropertyKey propertyKey, ChangeType changeType,
       EventType[] eventType) {
     checkConnected();
 
@@ -227,16 +311,18 @@ public class ZKHelixManager implements HelixManager {
         // compare property-key path and listener reference
         if (handler.getPath().equals(propertyKey.getPath())
             && handler.getListener().equals(listener)) {
-          logger.info("Listener: " + listener + " on path: " + propertyKey.getPath()
-              + " already exists. skip adding it");
+          LOG.info("Listener: " + listener + " on path: " + propertyKey.getPath()
+              + " already exists. skip add");
+
           return;
         }
       }
 
       CallbackHandler newHandler =
-          createCallBackHandler(propertyKey, listener, eventType, changeType);
+          new CallbackHandler(this, _zkclient, propertyKey, listener, eventType, changeType);
+
       _handlers.add(newHandler);
-      logger.info("Add listener: " + listener + " for type: " + type + " to path: "
+      LOG.info("Added listener: " + listener + " for type: " + type + " to path: "
           + newHandler.getPath());
     }
   }
@@ -259,7 +345,7 @@ public class ZKHelixManager implements HelixManager {
   }
 
   @Override
-  public void addConfigChangeListener(ConfigChangeListener listener) {
+  public void addConfigChangeListener(ConfigChangeListener listener) throws Exception {
     addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG,
         new EventType[] {
           EventType.NodeChildrenChanged
@@ -267,7 +353,8 @@ public class ZKHelixManager implements HelixManager {
   }
 
   @Override
-  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener) {
+  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
+      throws Exception {
     addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG,
         new EventType[] {
           EventType.NodeChildrenChanged
@@ -275,7 +362,8 @@ public class ZKHelixManager implements HelixManager {
   }
 
   @Override
-  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope) {
+  public void addConfigChangeListener(ScopedConfigChangeListener listener, ConfigScopeProperty scope)
+      throws Exception {
     Builder keyBuilder = new Builder(_clusterName);
 
     PropertyKey propertyKey = null;
@@ -298,7 +386,7 @@ public class ZKHelixManager implements HelixManager {
         EventType.NodeChildrenChanged
       });
     } else {
-      logger.error("Can't add listener to config scope: " + scope);
+      LOG.error("Can't add listener to config scope: " + scope);
     }
   }
 
@@ -312,7 +400,8 @@ public class ZKHelixManager implements HelixManager {
         });
   }
 
-  void addControllerMessageListener(MessageListener listener) {
+  @Override
+  public void addControllerMessageListener(MessageListener listener) {
     addListener(listener, new Builder(_clusterName).controllerMessages(),
         ChangeType.MESSAGES_CONTROLLER, new EventType[] {
             EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
@@ -321,7 +410,7 @@ public class ZKHelixManager implements HelixManager {
 
   @Override
   public void addCurrentStateChangeListener(CurrentStateChangeListener listener,
-      String instanceName, String sessionId) {
+      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
@@ -329,7 +418,8 @@ public class ZKHelixManager implements HelixManager {
   }
 
   @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName) {
+  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
+      throws Exception {
     addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH,
         new EventType[] {
             EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
@@ -337,7 +427,7 @@ public class ZKHelixManager implements HelixManager {
   }
 
   @Override
-  public void addExternalViewChangeListener(ExternalViewChangeListener listener) {
+  public void addExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.EXTERNAL_VIEW,
         new EventType[] {
             EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
@@ -355,7 +445,7 @@ public class ZKHelixManager implements HelixManager {
   @Override
   public HelixDataAccessor getHelixDataAccessor() {
     checkConnected();
-    return _helixAccessor;
+    return _dataAccessor;
   }
 
   @Override
@@ -374,484 +464,158 @@ public class ZKHelixManager implements HelixManager {
     return _instanceName;
   }
 
-  @Override
-  public void connect() throws Exception {
-    logger.info("ClusterManager.connect()");
-    if (_zkStateChangeListener.isConnected()) {
-      logger.warn("Cluster manager " + _clusterName + " " + _instanceName + " already connected");
-      return;
-    }
-
-    try {
-      createClient(_zkConnectString);
-      _messagingService.onConnected();
-    } catch (Exception e) {
-      logger.error(e);
-      disconnect();
-      throw e;
-    }
-  }
-
-  @Override
-  public void disconnect() {
-    if (!isConnected()) {
-      logger.error("ClusterManager " + _instanceName + " already disconnected");
-      return;
-    }
-    disconnectInternal();
-  }
-
-  void disconnectInternal() {
-    // This function can be called when the connection are in bad state(e.g. flapping),
-    // in which isConnected() could be false and we want to disconnect from cluster.
-    logger.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName);
-
-    /**
-     * shutdown thread pool first to avoid reset() being invoked in the middle of state
-     * transition
-     */
-    _messagingService.getExecutor().shutdown();
-    resetHandlers();
-
-    _helixAccessor.shutdown();
-
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.reset();
-    }
-
-    if (_participantHealthCheckInfoCollector != null) {
-      _participantHealthReportTask.stop();
-    }
-
-    if (_timer != null) {
-      _timer.cancel();
-      _timer = null;
-    }
-
-    if (_instanceType == InstanceType.CONTROLLER) {
-      stopTimerTasks();
-    }
-
-    // unsubscribe accessor from controllerChange
-    _zkClient.unsubscribeAll();
-
-    _zkClient.close();
-
-    // HACK seems that zkClient is not sending DISCONNECT event
-    _zkStateChangeListener.disconnect();
-    logger.info("Cluster manager: " + _instanceName + " disconnected");
-
-  }
-
-  @Override
-  public String getSessionId() {
-    checkConnected();
-    return _sessionId;
-  }
-
-  @Override
-  public boolean isConnected() {
-    return _zkStateChangeListener.isConnected();
-  }
-
-  @Override
-  public long getLastNotificationTime() {
-    return -1;
-  }
-
-  private void addLiveInstance() {
-    LiveInstance liveInstance = new LiveInstance(_instanceName);
-    liveInstance.setSessionId(_sessionId);
-    liveInstance.setHelixVersion(_version);
-    liveInstance.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName());
-
-    if (_liveInstanceInfoProvider != null) {
-      logger.info("invoking _liveInstanceInfoProvider");
-      ZNRecord additionalLiveInstanceInfo =
-          _liveInstanceInfoProvider.getAdditionalLiveInstanceInfo();
-      if (additionalLiveInstanceInfo != null) {
-        additionalLiveInstanceInfo.merge(liveInstance.getRecord());
-        ZNRecord mergedLiveInstance = new ZNRecord(additionalLiveInstanceInfo, _instanceName);
-        liveInstance = new LiveInstance(mergedLiveInstance);
-        logger.info("liveInstance content :" + _instanceName + " " + liveInstance.toString());
-      }
-    }
-
-    logger.info("Add live instance: InstanceName: " + _instanceName + " Session id:" + _sessionId);
-    Builder keyBuilder = _helixAccessor.keyBuilder();
-    if (!_helixAccessor.createProperty(keyBuilder.liveInstance(_instanceName), liveInstance)) {
-      String errorMsg =
-          "Fail to create live instance node after waiting, so quit. instance:" + _instanceName;
-      logger.warn(errorMsg);
-      throw new HelixException(errorMsg);
-
-    }
-    String currentStatePathParent =
-        PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName,
-            getSessionId());
-
-    if (!_zkClient.exists(currentStatePathParent)) {
-      _zkClient.createPersistent(currentStatePathParent);
-      logger.info("Creating current state path " + currentStatePathParent);
-    }
-  }
-
-  private void startStatusUpdatedumpTask() {
-    long initialDelay = 30 * 60 * 1000;
-    long period = 120 * 60 * 1000;
-    int timeThresholdNoChange = 180 * 60 * 1000;
+  BaseDataAccessor<ZNRecord> createBaseDataAccessor() {
+    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkclient);
 
-    if (_timer == null) {
-      _timer = new Timer(true);
-      _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(this, _zkClient, timeThresholdNoChange),
-          initialDelay, period);
-    }
+    return baseDataAccessor;
   }
 
-  private void createClient(String zkServers) throws Exception {
-    // by default use ZNRecordStreamingSerializer except for paths within the property
-    // store which expects raw byte[] serialization/deserialization
+  void createClient() throws Exception {
     PathBasedZkSerializer zkSerializer =
         ChainedPathZkSerializer.builder(new ZNRecordStreamingSerializer()).build();
 
-    _zkClient = new ZkClient(zkServers, _sessionTimeout, CONNECTIONTIMEOUT, zkSerializer);
+    _zkclient =
+        new ZkClient(_zkAddress, _sessionTimeout, ZkClient.DEFAULT_CONNECTION_TIMEOUT, zkSerializer);
 
-    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-    if (_instanceType == InstanceType.PARTICIPANT) {
-      String curStatePath =
-          PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName);
-      _baseDataAccessor =
-          new ZkCacheBaseDataAccessor<ZNRecord>(baseDataAccessor, Arrays.asList(curStatePath));
-    } else if (_instanceType == InstanceType.CONTROLLER) {
-      String extViewPath = PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW,
+    _baseDataAccessor = createBaseDataAccessor();
 
-      _clusterName);
-      _baseDataAccessor =
-          new ZkCacheBaseDataAccessor<ZNRecord>(baseDataAccessor, Arrays.asList(extViewPath));
+    _dataAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor);
+    _configAccessor = new ConfigAccessor(_zkclient);
 
-    } else {
-      _baseDataAccessor = baseDataAccessor;
-    }
-
-    _helixAccessor = new ZKHelixDataAccessor(_clusterName, _instanceType, _baseDataAccessor);
-    _configAccessor = new ConfigAccessor(_zkClient);
     int retryCount = 0;
 
-    _zkClient.subscribeStateChanges(_zkStateChangeListener);
-    while (retryCount < RETRY_LIMIT) {
+    _zkclient.subscribeStateChanges(this);
+    while (retryCount < 3) {
       try {
-        _zkClient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS);
-        _zkStateChangeListener.handleStateChanged(KeeperState.SyncConnected);
-        _zkStateChangeListener.handleNewSession();
+        _zkclient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS);
+        handleStateChanged(KeeperState.SyncConnected);
+        handleNewSession();
         break;
       } catch (HelixException e) {
-        logger.error("fail to createClient.", e);
+        LOG.error("fail to createClient.", e);
         throw e;
       } catch (Exception e) {
         retryCount++;
 
-        logger.error("fail to createClient. retry " + retryCount, e);
-        if (retryCount == RETRY_LIMIT) {
+        LOG.error("fail to createClient. retry " + retryCount, e);
+        if (retryCount == 3) {
           throw e;
         }
       }
     }
   }
 
-  private CallbackHandler createCallBackHandler(PropertyKey propertyKey, Object listener,
-      EventType[] eventTypes, ChangeType changeType) {
-    if (listener == null) {
-      throw new HelixException("Listener cannot be null");
+  @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;
     }
-    return new CallbackHandler(this, _zkClient, propertyKey, listener, eventTypes, changeType);
-  }
-
-  /**
-   * This will be invoked when ever a new session is created<br/>
-   * case 1: the cluster manager was a participant carry over current state, add live
-   * instance, and invoke message listener; case 2: the cluster manager was controller and
-   * was a leader before do leader election, and if it becomes leader again, invoke ideal
-   * state listener, current state listener, etc. if it fails to become leader in the new
-   * session, then becomes standby; case 3: the cluster manager was controller and was NOT
-   * a leader before do leader election, and if it becomes leader, instantiate and invoke
-   * ideal state listener, current state listener, etc. if if fails to become leader in
-   * the new session, stay as standby
-   */
 
-  protected void handleNewSession() {
-    boolean isConnected = _zkClient.waitUntilConnected(CONNECTIONTIMEOUT, TimeUnit.MILLISECONDS);
-    while (!isConnected) {
-      logger.error("Could NOT connect to zk server in " + CONNECTIONTIMEOUT + "ms. zkServer: "
-          + _zkConnectString + ", expiredSessionId: " + _sessionId + ", clusterName: "
-          + _clusterName);
-      isConnected = _zkClient.waitUntilConnected(CONNECTIONTIMEOUT, TimeUnit.MILLISECONDS);
+    try {
+      createClient();
+      _messagingService.onConnected();
+    } catch (Exception e) {
+      LOG.error("fail to connect " + _instanceName, e);
+      disconnect();
+      throw e;
     }
+  }
 
-    ZkConnection zkConnection = ((ZkConnection) _zkClient.getConnection());
-
-    synchronized (this) {
-      _sessionId = Long.toHexString(zkConnection.getZookeeper().getSessionId());
+  @Override
+  public void disconnect() {
+    if (_zkclient == null) {
+      LOG.info("instanceName: " + _instanceName + " already disconnected");
+      return;
     }
-    _baseDataAccessor.reset();
-
-    // reset all handlers so they have a chance to unsubscribe zk changes from zkclient
-    // abandon all callback-handlers added in expired session
-    resetHandlers();
 
-    logger.info("Handling new session, session id:" + _sessionId + ", instance:" + _instanceName
-        + ", instanceTye: " + _instanceType + ", cluster: " + _clusterName);
+    LOG.info("disconnect " + _instanceName + "(" + _instanceType + ") from " + _clusterName);
 
-    logger.info(zkConnection.getZookeeper());
-
-    if (!ZKUtil.isClusterSetup(_clusterName, _zkClient)) {
-      throw new HelixException("Initial cluster structure is not set up for cluster:"
-          + _clusterName);
-    }
-    // Read cluster config and see if instance can auto join the cluster
-    boolean autoJoin = false;
     try {
-      HelixConfigScope scope =
-          new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(getClusterName())
-              .build();
-      autoJoin = Boolean.parseBoolean(getConfigAccessor().get(scope, ALLOW_PARTICIPANT_AUTO_JOIN));
-      logger.info("Auto joining " + _clusterName + " is true");
-    } catch (Exception e) {
-    }
-    if (!ZKUtil.isInstanceSetup(_zkClient, _clusterName, _instanceName, _instanceType)) {
-      if (!autoJoin) {
-        throw new HelixException("Initial cluster structure is not set up for instance:"
-            + _instanceName + " instanceType:" + _instanceType);
-      } else {
-        logger.info("Auto joining instance " + _instanceName);
-        InstanceConfig instanceConfig = new InstanceConfig(_instanceName);
-        String hostName = _instanceName;
-        String port = "";
-        int lastPos = _instanceName.lastIndexOf("_");
-        if (lastPos > 0) {
-          hostName = _instanceName.substring(0, lastPos);
-          port = _instanceName.substring(lastPos + 1);
-        }
-        instanceConfig.setHostName(hostName);
-        instanceConfig.setPort(port);
-        instanceConfig.setInstanceEnabled(true);
-        getClusterManagmentTool().addInstance(_clusterName, instanceConfig);
-      }
-    }
+      /**
+       * stop all timer tasks
+       */
+      stopTimerTasks();
 
-    if (_instanceType == InstanceType.PARTICIPANT
-        || _instanceType == InstanceType.CONTROLLER_PARTICIPANT) {
-      handleNewSessionAsParticipant();
-    }
+      /**
+       * 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 (_instanceType == InstanceType.CONTROLLER
-        || _instanceType == InstanceType.CONTROLLER_PARTICIPANT) {
-      addControllerMessageListener(_messagingService.getExecutor());
-      MessageHandlerFactory defaultControllerMsgHandlerFactory =
-          new DefaultControllerMessageHandlerFactory();
-      _messagingService.getExecutor().registerMessageHandlerFactory(
-          defaultControllerMsgHandlerFactory.getMessageType(), defaultControllerMsgHandlerFactory);
-      MessageHandlerFactory defaultSchedulerMsgHandlerFactory =
-          new DefaultSchedulerMessageHandlerFactory(this);
-      _messagingService.getExecutor().registerMessageHandlerFactory(
-          defaultSchedulerMsgHandlerFactory.getMessageType(), defaultSchedulerMsgHandlerFactory);
-      MessageHandlerFactory defaultParticipantErrorMessageHandlerFactory =
-          new DefaultParticipantErrorMessageHandlerFactory(this);
-      _messagingService.getExecutor().registerMessageHandlerFactory(
-          defaultParticipantErrorMessageHandlerFactory.getMessageType(),
-          defaultParticipantErrorMessageHandlerFactory);
+      _dataAccessor.shutdown();
 
       if (_leaderElectionHandler != null) {
         _leaderElectionHandler.reset();
-        _leaderElectionHandler.init();
-      } else {
-        _leaderElectionHandler =
-            createCallBackHandler(new Builder(_clusterName).controller(),
-                new DistClusterControllerElection(_zkConnectString), new EventType[] {
-                    EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-                }, ChangeType.CONTROLLER);
       }
-    }
 
-    if (_instanceType == InstanceType.PARTICIPANT
-        || _instanceType == InstanceType.CONTROLLER_PARTICIPANT
-        || (_instanceType == InstanceType.CONTROLLER && isLeader())) {
-      initHandlers();
+    } finally {
+      _zkclient.close();
+      _zkclient = null;
+      LOG.info("Cluster manager: " + _instanceName + " disconnected");
     }
   }
 
-  private void handleNewSessionAsParticipant() {
-    // In case there is a live instance record on zookeeper
-    Builder keyBuilder = _helixAccessor.keyBuilder();
-
-    if (_helixAccessor.getProperty(keyBuilder.liveInstance(_instanceName)) != null) {
-      logger.warn("Found another instance with same instanceName: " + _instanceName
-          + " in cluster " + _clusterName);
-      // Wait for a while, in case previous storage node exits unexpectedly
-      // and its liveinstance
-      // still hangs around until session timeout happens
-      try {
-        Thread.sleep(_sessionTimeout + 5000);
-      } catch (InterruptedException e) {
-        logger.warn("Sleep interrupted while waiting for previous liveinstance to go away.", e);
-      }
+  @Override
+  public String getSessionId() {
+    checkConnected();
+    return _sessionId;
+  }
 
-      if (_helixAccessor.getProperty(keyBuilder.liveInstance(_instanceName)) != null) {
-        String errorMessage =
-            "instance " + _instanceName + " already has a liveinstance in cluster " + _clusterName;
-        logger.error(errorMessage);
-        throw new HelixException(errorMessage);
-      }
+  @Override
+  public boolean isConnected() {
+    if (_zkclient == null) {
+      return false;
     }
-    // Invoke the PreConnectCallbacks
-    for (PreConnectCallback callback : _preConnectCallbacks) {
-      callback.onPreConnect();
+    ZkConnection zkconnection = (ZkConnection) _zkclient.getConnection();
+    if (zkconnection != null) {
+      States state = zkconnection.getZookeeperState();
+      return state == States.CONNECTED;
     }
-    addLiveInstance();
-    carryOverPreviousCurrentState();
-
-    // In case the cluster manager is running as a participant, setup message
-    // listener
-    _messagingService.registerMessageHandlerFactory(MessageType.STATE_TRANSITION.toString(),
-        _stateMachEngine);
-    addMessageListener(_messagingService.getExecutor(), _instanceName);
-    addControllerListener(_helixAccessor);
-
-    ScheduledTaskStateModelFactory stStateModelFactory =
-        new ScheduledTaskStateModelFactory(_messagingService.getExecutor());
-    _stateMachEngine.registerStateModelFactory(
-        DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, stStateModelFactory);
+    return false;
+  }
 
-    if (_participantHealthCheckInfoCollector == null) {
-      _participantHealthCheckInfoCollector =
-          new ParticipantHealthReportCollectorImpl(this, _instanceName);
-      _participantHealthReportTask =
-          new ParticipantHealthReportTask(_participantHealthCheckInfoCollector);
-      _participantHealthReportTask.start();
-    }
-    // start the participant health check timer, also create zk path for health
-    // check info
-    String healthCheckInfoPath = _helixAccessor.keyBuilder().healthReports(_instanceName).getPath();
-    if (!_zkClient.exists(healthCheckInfoPath)) {
-      _zkClient.createPersistent(healthCheckInfoPath, true);
-      logger.info("Creating healthcheck info path " + healthCheckInfoPath);
-    }
+  @Override
+  public long getLastNotificationTime() {
+    return 0;
   }
 
   @Override
   public void addPreConnectCallback(PreConnectCallback callback) {
-    logger.info("Adding preconnect callback");
+    LOG.info("Adding preconnect callback: " + callback);
     _preConnectCallbacks.add(callback);
   }
 
-  private void resetHandlers() {
-    synchronized (this) {
-      if (_handlers != null) {
-        // get a copy of the list and iterate over the copy list
-        // in case handler.reset() will modify the original handler list
-        List<CallbackHandler> tmpHandlers = new ArrayList<CallbackHandler>();
-        tmpHandlers.addAll(_handlers);
-
-        for (CallbackHandler handler : tmpHandlers) {
-          handler.reset();
-          logger.info("reset handler: " + handler.getPath() + ", " + handler.getListener());
-        }
-      }
-    }
-  }
-
-  private void initHandlers() {
-    synchronized (this) {
-      if (_handlers != null) {
-        // may add new currentState and message listeners during init()
-        // so make a copy and iterate over the copy
-        List<CallbackHandler> tmpHandlers = new ArrayList<CallbackHandler>();
-        tmpHandlers.addAll(_handlers);
-        for (CallbackHandler handler : tmpHandlers) {
-          handler.init();
-          logger.info("init handler: " + handler.getPath() + ", " + handler.getListener());
-        }
-      }
-    }
-  }
-
   @Override
   public boolean isLeader() {
-    if (!isConnected()) {
+    if (_instanceType != InstanceType.CONTROLLER
+        && _instanceType != InstanceType.CONTROLLER_PARTICIPANT) {
       return false;
     }
 
-    if (_instanceType != InstanceType.CONTROLLER) {
-      return false;
-    }
-
-    Builder keyBuilder = _helixAccessor.keyBuilder();
-    LiveInstance leader = _helixAccessor.getProperty(keyBuilder.controllerLeader());
-    if (leader == null) {
+    if (!isConnected()) {
       return false;
-    } else {
-      String leaderName = leader.getInstanceName();
-      // TODO need check sessionId also, but in distributed mode, leader's
-      // sessionId is
-      // not equal to
-      // the leader znode's sessionId field which is the sessionId of the
-      // controller_participant that
-      // successfully creates the leader node
-      if (leaderName == null || !leaderName.equals(_instanceName)) {
-        return false;
-      }
     }
-    return true;
-  }
-
-  /**
-   * carry over current-states from last sessions
-   * set to initial state for current session only when the state doesn't exist in current session
-   */
-  private void carryOverPreviousCurrentState() {
-    Builder keyBuilder = _helixAccessor.keyBuilder();
-    List<String> sessions = _helixAccessor.getChildNames(keyBuilder.sessions(_instanceName));
 
-    // carry-over
-    for (String session : sessions) {
-      if (session.equals(_sessionId)) {
-        continue;
-      }
-
-      List<CurrentState> lastCurStates =
-          _helixAccessor.getChildValues(keyBuilder.currentStates(_instanceName, session));
-
-      for (CurrentState lastCurState : lastCurStates) {
-        logger.info("Carrying over old session: " + session + ", resource: " + lastCurState.getId()
-            + " to current session: " + _sessionId);
-        String stateModelDefRef = lastCurState.getStateModelDefRef();
-        if (stateModelDefRef == null) {
-          logger
-              .error("skip carry-over because previous current state doesn't have a state model definition. previous current-state: "
-                  + lastCurState);
-          continue;
+    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;
         }
-        StateModelDefinition stateModel =
-            _helixAccessor.getProperty(keyBuilder.stateModelDef(stateModelDefRef));
-
-        String curStatePath =
-            keyBuilder.currentState(_instanceName, _sessionId, lastCurState.getResourceName())
-                .getPath();
-        _helixAccessor.getBaseDataAccessor().update(curStatePath,
-            new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialState(), lastCurState),
-            AccessOption.PERSISTENT);
-      }
-    }
-
-    // remove previous current states
-    for (String session : sessions) {
-      if (session.equals(_sessionId)) {
-        continue;
       }
-
-      String path = _helixAccessor.keyBuilder().currentStates(_instanceName, session).getPath();
-      logger.info("Removing current states from previous sessions. path: " + path);
-      _zkClient.deleteRecursive(path);
+    } catch (Exception e) {
+      // log
     }
+    return false;
   }
 
   @Override
@@ -862,7 +626,7 @@ public class ZKHelixManager implements HelixManager {
       String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName);
 
       _helixPropertyStore =
-          new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkClient), path,
+          new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient), path,
               null);
     }
 
@@ -872,13 +636,12 @@ public class ZKHelixManager implements HelixManager {
   @Override
   public synchronized HelixAdmin getClusterManagmentTool() {
     checkConnected();
-    if (_zkClient != null) {
-      _managementTool = new ZKHelixAdmin(_zkClient);
-    } else {
-      logger.error("Couldn't get ZKClusterManagementTool because zkClient is null");
+    if (_zkclient != null) {
+      return new ZKHelixAdmin(_zkclient);
     }
 
-    return _managementTool;
+    LOG.error("Couldn't get ZKClusterManagementTool because zkclient is null");
+    return null;
   }
 
   @Override
@@ -891,7 +654,7 @@ public class ZKHelixManager implements HelixManager {
   @Override
   public ParticipantHealthReportCollector getHealthReportCollector() {
     checkConnected();
-    return _participantHealthCheckInfoCollector;
+    return _participantHealthInfoCollector;
   }
 
   @Override
@@ -899,12 +662,6 @@ public class ZKHelixManager implements HelixManager {
     return _instanceType;
   }
 
-  private void checkConnected() {
-    if (!isConnected()) {
-      throw new HelixException("ClusterManager not connected. Call clusterManager.connect()");
-    }
-  }
-
   @Override
   public String getVersion() {
     return _version;
@@ -917,21 +674,20 @@ public class ZKHelixManager implements HelixManager {
 
   @Override
   public StateMachineEngine getStateMachineEngine() {
-    return _stateMachEngine;
+    return _stateMachineEngine;
   }
 
   // TODO: rename this and not expose this function as part of interface
   @Override
   public void startTimerTasks() {
-    for (HelixTimerTask task : _controllerTimerTasks) {
+    for (HelixTimerTask task : _timerTasks) {
       task.start();
     }
-    startStatusUpdatedumpTask();
   }
 
   @Override
   public void stopTimerTasks() {
-    for (HelixTimerTask task : _controllerTimerTasks) {
+    for (HelixTimerTask task : _timerTasks) {
       task.stop();
     }
   }
@@ -940,4 +696,209 @@ public class ZKHelixManager implements HelixManager {
   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());
+        }
+      }
+    }
+  }
+
+  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());
+        }
+      }
+    }
+  }
+
+  /**
+   * 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;
+    }
+    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);
+    }
+    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();
+      }
+      break;
+    case Expired:
+      LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId + ", instance: "
+          + _instanceName + ", type: " + _instanceType);
+      break;
+    default:
+      break;
+    }
+  }
+
+  @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:
+      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();
+    }
+
+    participantHelper.createLiveInstance();
+    participantHelper.carryOverPreviousCurrentState();
+
+    /**
+     * setup message listener
+     */
+    participantHelper.setupMsgHandler();
+
+    /**
+     * start health check timer task
+     */
+    participantHelper.createHealthCheckPath();
+  }
+
+  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);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
index 5865863..755ca52 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
@@ -143,8 +143,7 @@ public class ZkAsyncCallbacks {
           }
         }
       } catch (InterruptedException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
+        LOG.error("Interrupted waiting for success", e);
       }
       return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java
deleted file mode 100644
index e11444b..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkStateChangeListener.java
+++ /dev/null
@@ -1,127 +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 java.util.LinkedList;
-import java.util.List;
-
-import org.I0Itec.zkclient.IZkStateListener;
-import org.I0Itec.zkclient.ZkConnection;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-
-public class ZkStateChangeListener implements IZkStateListener {
-  private volatile boolean _isConnected;
-  private volatile boolean _hasSessionExpired;
-  private final ZKHelixManager _zkHelixManager;
-
-  // 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
-  List<Long> _disconnectTimeHistory = new LinkedList<Long>();
-  int _timeWindowLengthMs;
-  int _maxDisconnectThreshold;
-
-  private static Logger logger = Logger.getLogger(ZkStateChangeListener.class);
-
-  public ZkStateChangeListener(ZKHelixManager zkHelixManager, int timeWindowLengthMs,
-      int maxDisconnectThreshold) {
-    this._zkHelixManager = zkHelixManager;
-    _timeWindowLengthMs = timeWindowLengthMs;
-    // _maxDisconnectThreshold min value is 1.
-    // We don't want to disconnect from zk for the first time zkState become Disconnected
-    _maxDisconnectThreshold = maxDisconnectThreshold > 0 ? maxDisconnectThreshold : 1;
-  }
-
-  @Override
-  public void handleNewSession() {
-    // TODO:bug in zkclient .
-    // zkclient does not invoke handleStateChanged when a session expires but
-    // directly invokes handleNewSession
-    _isConnected = true;
-    _hasSessionExpired = false;
-    _zkHelixManager.handleNewSession();
-  }
-
-  @Override
-  public void handleStateChanged(KeeperState keeperState) throws Exception {
-    switch (keeperState) {
-    case SyncConnected:
-      ZkConnection zkConnection = ((ZkConnection) _zkHelixManager._zkClient.getConnection());
-      logger.info("KeeperState: " + keeperState + ", zookeeper:" + zkConnection.getZookeeper());
-      _isConnected = true;
-      break;
-    case Disconnected:
-      logger.info("KeeperState:" + keeperState + ", disconnectedSessionId: "
-          + _zkHelixManager._sessionId + ", instance: " + _zkHelixManager.getInstanceName()
-          + ", type: " + _zkHelixManager.getInstanceType());
-
-      _isConnected = false;
-      // Track the time stamp that the disconnected happens, then check history and see if
-      // we should disconnect the _zkHelixManager
-      _disconnectTimeHistory.add(System.currentTimeMillis());
-      if (isFlapping()) {
-        logger.error("isFlapping() returns true, so disconnect the helix manager. "
-            + _zkHelixManager.getInstanceName() + " " + _maxDisconnectThreshold
-            + " disconnects in " + _timeWindowLengthMs + " Ms.");
-        _zkHelixManager.disconnectInternal();
-      }
-      break;
-    case Expired:
-      logger.info("KeeperState:" + keeperState + ", expiredSessionId: "
-          + _zkHelixManager._sessionId + ", instance: " + _zkHelixManager.getInstanceName()
-          + ", type: " + _zkHelixManager.getInstanceType());
-
-      _isConnected = false;
-      _hasSessionExpired = true;
-      break;
-    }
-  }
-
-  boolean isConnected() {
-    return _isConnected;
-  }
-
-  void disconnect() {
-    _isConnected = false;
-  }
-
-  boolean hasSessionExpired() {
-    return _hasSessionExpired;
-  }
-
-  /**
-   * 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;
-    }
-    long mostRecentTimestamp = _disconnectTimeHistory.get(_disconnectTimeHistory.size() - 1);
-    // Remove disconnect history timestamp that are older than _timeWindowLengthMs ago
-    while ((_disconnectTimeHistory.get(0) + _timeWindowLengthMs) < mostRecentTimestamp) {
-      _disconnectTimeHistory.remove(0);
-    }
-    return _disconnectTimeHistory.size() > _maxDisconnectThreshold;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 73b69a8..422d35e 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
@@ -117,9 +117,6 @@ public class DefaultMessagingService implements ClusterMessagingService {
         Builder keyBuilder = accessor.keyBuilder();
 
         if (receiverType == InstanceType.CONTROLLER) {
-          // _manager.getDataAccessor().setProperty(PropertyType.MESSAGES_CONTROLLER,
-          // tempMessage,
-          // tempMessage.getId());
           accessor.setProperty(keyBuilder.controllerMessage(tempMessage.getId()), tempMessage);
         }
 
@@ -137,6 +134,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
     return totalMessageCount;
   }
 
+  @Override
   public Map<InstanceType, List<Message>> generateMessage(final Criteria recipientCriteria,
       final Message message) {
     Map<InstanceType, List<Message>> messagesToSendMap = new HashMap<InstanceType, List<Message>>();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
index 8672e7e..6da3dc2 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
@@ -32,7 +32,6 @@ import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.ResourceId;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 
 /**
@@ -94,20 +93,18 @@ public class ResourceAssignment extends HelixProperty {
   /**
    * Get the participant, state pairs for a partition
    * @param partition the Partition to look up
-   * @return immutable map of (participant id, state)
+   * @return map of (participant id, state)
    */
   public Map<ParticipantId, State> getReplicaMap(PartitionId partitionId) {
     Map<String, String> rawReplicaMap = _record.getMapField(partitionId.stringify());
-    if (rawReplicaMap == null) {
-      return Collections.emptyMap();
-    }
-    ImmutableMap.Builder<ParticipantId, State> builder =
-        new ImmutableMap.Builder<ParticipantId, State>();
-    for (String participantName : rawReplicaMap.keySet()) {
-      builder.put(ParticipantId.from(participantName),
-          State.from(rawReplicaMap.get(participantName)));
+    Map<ParticipantId, State> replicaMap = Maps.newHashMap();
+    if (rawReplicaMap != null) {
+      for (String participantName : rawReplicaMap.keySet()) {
+        replicaMap.put(ParticipantId.from(participantName),
+            State.from(rawReplicaMap.get(participantName)));
+      }
     }
-    return builder.build();
+    return replicaMap;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 0558764..919eb00 100644
--- a/helix-core/src/test/java/org/apache/helix/Mocks.java
+++ b/helix-core/src/test/java/org/apache/helix/Mocks.java
@@ -472,6 +472,12 @@ public class Mocks {
       return _properties;
     }
 
+    @Override
+    public void addControllerMessageListener(MessageListener listener) {
+      // TODO Auto-generated method stub
+
+    }
+
   }
 
   public static class MockAccessor implements HelixDataAccessor // DataAccessor

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 4bdd423..871d717 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelper.java
@@ -50,7 +50,6 @@ 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.controller.HelixControllerMain;
 import org.apache.helix.integration.manager.ZkTestManager;
 import org.apache.helix.manager.zk.CallbackHandler;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -65,8 +64,6 @@ import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty;
-import org.apache.helix.participant.DistClusterControllerStateModelFactory;
-import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.store.zk.ZNode;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.util.ZKClientPool;
@@ -131,7 +128,7 @@ public class TestHelper {
           try {
             zkClient.deleteRecursive(rootNamespace);
           } catch (Exception e) {
-            LOG.error("fail to deleteRecursive path:" + rootNamespace + "\nexception:" + e);
+            LOG.error("fail to deleteRecursive path:" + rootNamespace, e);
           }
         }
       }
@@ -152,90 +149,6 @@ public class TestHelper {
     }
   }
 
-  public static StartCMResult startDummyProcess(final String zkAddr, final String clusterName,
-      final String instanceName) throws Exception {
-    StartCMResult result = new StartCMResult();
-    ZkHelixTestManager manager = null;
-    manager = new ZkHelixTestManager(clusterName, instanceName, InstanceType.PARTICIPANT, zkAddr);
-    result._manager = manager;
-    Thread thread = new Thread(new DummyProcessThread(manager, instanceName));
-    result._thread = thread;
-    thread.start();
-
-    return result;
-  }
-
-  private static ZkHelixTestManager startHelixController(final String zkConnectString,
-      final String clusterName, final String controllerName, final String controllerMode) {
-    ZkHelixTestManager manager = null;
-    try {
-      if (controllerMode.equalsIgnoreCase(HelixControllerMain.STANDALONE)) {
-        manager =
-            new ZkHelixTestManager(clusterName, controllerName, InstanceType.CONTROLLER,
-                zkConnectString);
-        manager.connect();
-      } else if (controllerMode.equalsIgnoreCase(HelixControllerMain.DISTRIBUTED)) {
-        manager =
-            new ZkHelixTestManager(clusterName, controllerName,
-                InstanceType.CONTROLLER_PARTICIPANT, zkConnectString);
-
-        DistClusterControllerStateModelFactory stateModelFactory =
-            new DistClusterControllerStateModelFactory(zkConnectString);
-
-        StateMachineEngine stateMach = manager.getStateMachineEngine();
-        stateMach.registerStateModelFactory("LeaderStandby", stateModelFactory);
-        manager.connect();
-      } else {
-        LOG.error("cluster controller mode:" + controllerMode + " NOT supported");
-      }
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-
-    return manager;
-  }
-
-  // TODO refactor this
-  public static StartCMResult startController(final String clusterName,
-      final String controllerName, final String zkConnectString, final String controllerMode)
-      throws Exception {
-    final StartCMResult result = new StartCMResult();
-    final ZkHelixTestManager manager =
-        startHelixController(zkConnectString, clusterName, controllerName, controllerMode);
-    result._manager = manager;
-
-    Thread thread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        // ClusterManager manager = null;
-
-        try {
-
-          Thread.currentThread().join();
-        } catch (InterruptedException e) {
-          String msg =
-              "controller:" + controllerName + ", " + Thread.currentThread().getName()
-                  + " interrupted";
-          LOG.info(msg);
-          // System.err.println(msg);
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    });
-
-    thread.start();
-    result._thread = thread;
-    return result;
-  }
-
-  public static class StartCMResult {
-    public Thread _thread;
-    public ZkHelixTestManager _manager;
-
-  }
-
   public static void setupEmptyCluster(ZkClient zkClient, String clusterName) {
     ZKHelixAdmin admin = new ZKHelixAdmin(zkClient);
     admin.addCluster(clusterName, true);
@@ -251,11 +164,6 @@ public class TestHelper {
     return set;
   }
 
-  // public static void verifyWithTimeout(String verifierName, Object... args)
-  // {
-  // verifyWithTimeout(verifierName, 30 * 1000, args);
-  // }
-
   /**
    * generic method for verification with a timeout
    * @param verifierName
@@ -292,8 +200,7 @@ public class TestHelper {
 
       Assert.assertTrue(result);
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Exception in verify: " + verifierName, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java b/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java
index 37c68a2..0decbd8 100644
--- a/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java
+++ b/helix-core/src/test/java/org/apache/helix/TestZkClientWrapper.java
@@ -23,6 +23,7 @@ import org.I0Itec.zkclient.IZkStateListener;
 import org.I0Itec.zkclient.ZkConnection;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
@@ -34,6 +35,8 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestZkClientWrapper extends ZkUnitTestBase {
+  private static Logger LOG = Logger.getLogger(TestZkClientWrapper.class);
+
   ZkClient _zkClient;
 
   @BeforeClass
@@ -69,7 +72,7 @@ public class TestZkClientWrapper extends ZkUnitTestBase {
   }
 
   @Test()
-  void testSessioExpire() {
+  void testSessioExpire() throws Exception {
     IZkStateListener listener = new IZkStateListener() {
 
       @Override
@@ -82,31 +85,27 @@ public class TestZkClientWrapper extends ZkUnitTestBase {
         System.out.println("In Old connection New session");
       }
     };
+
     _zkClient.subscribeStateChanges(listener);
     ZkConnection connection = ((ZkConnection) _zkClient.getConnection());
     ZooKeeper zookeeper = connection.getZookeeper();
     System.out.println("old sessionId= " + zookeeper.getSessionId());
-    try {
-      Watcher watcher = new Watcher() {
-        @Override
-        public void process(WatchedEvent event) {
-          System.out.println("In New connection In process event:" + event);
-        }
-      };
-      ZooKeeper newZookeeper =
-          new ZooKeeper(connection.getServers(), zookeeper.getSessionTimeout(), watcher,
-              zookeeper.getSessionId(), zookeeper.getSessionPasswd());
-      Thread.sleep(3000);
-      System.out.println("New sessionId= " + newZookeeper.getSessionId());
-      Thread.sleep(3000);
-      newZookeeper.close();
-      Thread.sleep(10000);
-      connection = ((ZkConnection) _zkClient.getConnection());
-      zookeeper = connection.getZookeeper();
-      System.out.println("After session expiry sessionId= " + zookeeper.getSessionId());
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+    Watcher watcher = new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        System.out.println("In New connection In process event:" + event);
+      }
+    };
+    ZooKeeper newZookeeper =
+        new ZooKeeper(connection.getServers(), zookeeper.getSessionTimeout(), watcher,
+            zookeeper.getSessionId(), zookeeper.getSessionPasswd());
+    Thread.sleep(3000);
+    System.out.println("New sessionId= " + newZookeeper.getSessionId());
+    Thread.sleep(3000);
+    newZookeeper.close();
+    Thread.sleep(10000);
+    connection = ((ZkConnection) _zkClient.getConnection());
+    zookeeper = connection.getZookeeper();
+    System.out.println("After session expiry sessionId= " + zookeeper.getSessionId());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java b/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java
index 9c71fdd..fdf6e72 100644
--- a/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java
+++ b/helix-core/src/test/java/org/apache/helix/TestZnodeModify.java
@@ -215,8 +215,7 @@ public class TestZnodeModify extends ZkUnitTestBase {
           zkClient.createPersistent(pathChild1, true);
           zkClient.writeData(pathChild1, record);
         } catch (InterruptedException e) {
-          // TODO Auto-generated catch block
-          e.printStackTrace();
+          logger.error("Interrupted sleep", e);
         }
       }
     }.start();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java b/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java
deleted file mode 100644
index b660a1d..0000000
--- a/helix-core/src/test/java/org/apache/helix/ZkHelixTestManager.java
+++ /dev/null
@@ -1,44 +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.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZKHelixManager;
-import org.apache.helix.manager.zk.ZkClient;
-
-// ZkHelixManager used for test only. expose more class members
-public class ZkHelixTestManager extends ZKHelixManager {
-
-  public ZkHelixTestManager(String clusterName, String instanceName, InstanceType instanceType,
-      String zkConnectString) throws Exception {
-    super(clusterName, instanceName, instanceType, zkConnectString);
-    // TODO Auto-generated constructor stub
-  }
-
-  public ZkClient getZkClient() {
-    return _zkClient;
-  }
-
-  public List<CallbackHandler> getHandlers() {
-    return _handlers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 369ad68..0d597f6 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
@@ -39,10 +39,10 @@ import org.apache.helix.controller.stages.ClusterEvent;
 import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
 import org.apache.helix.controller.stages.BestPossibleStateOutput;
 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.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -59,8 +59,8 @@ public class TestNewStages extends ZkUnitTestBase {
   final int n = 2;
   final int p = 8;
   final int r = 2;
-  MockParticipant[] _participants = new MockParticipant[n];
-  ClusterController _controller;
+  MockParticipantManager[] _participants = new MockParticipantManager[n];
+  ClusterControllerManager _controller;
 
   ClusterId _clusterId;
   HelixDataAccessor _dataAccessor;
@@ -217,14 +217,14 @@ public class TestNewStages extends ZkUnitTestBase {
         r, // replicas
         "MasterSlave", true); // do rebalance
 
-    _controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    _controller = new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     _controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      _participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       _participants[i].syncStart();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 26da8ee..fc9b7d5 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
@@ -254,4 +254,10 @@ public class DummyClusterManager implements HelixManager {
     // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public void addControllerMessageListener(MessageListener listener) {
+    // TODO Auto-generated method stub
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
index ba61361..dd5f441 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -31,9 +32,11 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.api.State;
 import org.apache.helix.api.accessor.ClusterAccessor;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.MessageId;
+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.pipeline.Pipeline;
@@ -43,8 +46,10 @@ import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.ConstraintItem;
+import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.ResourceAssignment;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -62,7 +67,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
 
     // ideal state: node0 is MASTER, node1 is SLAVE
     // replica=2 means 1 master and 1 slave
-    setupIdealState(clusterName, new int[] {
+    List<IdealState> idealStates = setupIdealState(clusterName, new int[] {
         0, 1
     }, new String[] {
       "TestDB"
@@ -78,6 +83,10 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     ClusterEvent event = new ClusterEvent("testEvent");
     event.addAttribute("helixmanager", manager);
 
+    // get an empty best possible output for the partitions
+    BestPossibleStateOutput bestPossOutput = getEmptyBestPossibleStateOutput(idealStates);
+    event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossOutput);
+
     MessageThrottleStage throttleStage = new MessageThrottleStage();
     try {
       runStage(event, throttleStage);
@@ -138,7 +147,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
 
     // ideal state: node0 is MASTER, node1 is SLAVE
     // replica=2 means 1 master and 1 slave
-    setupIdealState(clusterName, new int[] {
+    List<IdealState> idealStates = setupIdealState(clusterName, new int[] {
         0, 1
     }, new String[] {
       "TestDB"
@@ -270,6 +279,10 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     ClusterEvent event = new ClusterEvent("testEvent");
     event.addAttribute("helixmanager", manager);
 
+    // get an empty best possible output for the partitions
+    BestPossibleStateOutput bestPossOutput = getEmptyBestPossibleStateOutput(idealStates);
+    event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossOutput);
+
     Pipeline dataRefresh = new Pipeline();
     dataRefresh.addStage(new ReadClusterDataStage());
     runPipeline(event, dataRefresh);
@@ -329,5 +342,18 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     return false;
   }
 
+  private BestPossibleStateOutput getEmptyBestPossibleStateOutput(List<IdealState> idealStates) {
+    BestPossibleStateOutput output = new BestPossibleStateOutput();
+    for (IdealState idealState : idealStates) {
+      ResourceId resourceId = idealState.getResourceId();
+      ResourceAssignment assignment = new ResourceAssignment(resourceId);
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
+        Map<ParticipantId, State> emptyMap = Collections.emptyMap();
+        assignment.addReplicaMap(partitionId, emptyMap);
+      }
+      output.setResourceAssignment(resourceId, assignment);
+    }
+    return output;
+  }
   // add pending message test case
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
index 8e7b85a..2dbf5f6 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
@@ -27,8 +27,8 @@ import org.testng.annotations.Test;
 public class TestParseInfoFromAlert extends ZkStandAloneCMTestBase {
   @Test
   public void TestParse() {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+
+    HelixManager manager = _controller;
 
     String instanceName =
         StatsAggregationStage.parseInstanceName("localhost_12918.TestStat@DB=123.latency", manager);
@@ -50,6 +50,7 @@ public class TestParseInfoFromAlert extends ZkStandAloneCMTestBase {
     String partitionName =
         StatsAggregationStage.parsePartitionName(
             "localhost_12918.TestStat@DB=TestDB;Partition=TestDB_22.latency", manager);
+
     Assert.assertTrue(partitionName.equals("TestDB_22"));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 4129f66..2e17ad3 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
@@ -37,6 +37,16 @@ import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.pipeline.Pipeline;
+import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
+import org.apache.helix.controller.stages.ClusterEvent;
+import org.apache.helix.controller.stages.CurrentStateComputationStage;
+import org.apache.helix.controller.stages.MessageSelectionStage;
+import org.apache.helix.controller.stages.MessageThrottleStage;
+import org.apache.helix.controller.stages.ReadClusterDataStage;
+import org.apache.helix.controller.stages.ResourceComputationStage;
+import org.apache.helix.controller.stages.TaskAssignmentStage;
+import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -138,6 +148,9 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
+    HelixManager manager = new DummyClusterManager(clusterName, accessor);
+    ClusterEvent event = new ClusterEvent("testEvent");
+
     final String resourceName = "testResource_dup";
     String[] resourceGroups = new String[] {
       resourceName
@@ -161,8 +174,10 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor);
     clusterAccessor.initClusterStructure();
 
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // round1: controller sends O->S to both node0 and node1
     Thread.sleep(1000);


[06/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java
index 0269764..aea9b70 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestShufflingTwoStateStrategy.java
@@ -19,7 +19,6 @@ package org.apache.helix.controller.strategy;
  * under the License.
  */
 
-import java.io.IOException;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.util.ArrayList;
@@ -29,8 +28,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.ZNRecord;
-import org.codehaus.jackson.JsonGenerationException;
-import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;
 import org.testng.AssertJUnit;
@@ -68,44 +65,33 @@ public class TestShufflingTwoStateStrategy {
 
     // ByteArrayOutputStream baos = new ByteArrayOutputStream();
     StringWriter sw = new StringWriter();
-    try {
-      mapper.writeValue(sw, result);
-      // System.out.println(sw.toString());
-
-      ZNRecord zn = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
-      System.out.println(result.toString());
-      System.out.println(zn.toString());
-      AssertJUnit.assertTrue(zn.toString().equalsIgnoreCase(result.toString()));
-      System.out.println();
-
-      sw = new StringWriter();
-      mapper.writeValue(sw, result2);
-
-      ZNRecord zn2 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
-      System.out.println(result2.toString());
-      System.out.println(zn2.toString());
-      AssertJUnit.assertTrue(zn2.toString().equalsIgnoreCase(result2.toString()));
-
-      sw = new StringWriter();
-      mapper.writeValue(sw, result3);
-      System.out.println();
-
-      ZNRecord zn3 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
-      System.out.println(result3.toString());
-      System.out.println(zn3.toString());
-      AssertJUnit.assertTrue(zn3.toString().equalsIgnoreCase(result3.toString()));
-      System.out.println();
-
-    } catch (JsonGenerationException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (JsonMappingException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (IOException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+    mapper.writeValue(sw, result);
+    // System.out.println(sw.toString());
+
+    ZNRecord zn = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
+    System.out.println(result.toString());
+    System.out.println(zn.toString());
+    AssertJUnit.assertTrue(zn.toString().equalsIgnoreCase(result.toString()));
+    System.out.println();
+
+    sw = new StringWriter();
+    mapper.writeValue(sw, result2);
+
+    ZNRecord zn2 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
+    System.out.println(result2.toString());
+    System.out.println(zn2.toString());
+    AssertJUnit.assertTrue(zn2.toString().equalsIgnoreCase(result2.toString()));
+
+    sw = new StringWriter();
+    mapper.writeValue(sw, result3);
+    System.out.println();
+
+    ZNRecord zn3 = mapper.readValue(new StringReader(sw.toString()), ZNRecord.class);
+    System.out.println(result3.toString());
+    System.out.println(zn3.toString());
+    AssertJUnit.assertTrue(zn3.toString().equalsIgnoreCase(result3.toString()));
+    System.out.println();
+
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
index e0e1544..cadbdc7 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
@@ -28,17 +28,15 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
@@ -49,7 +47,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestAddDropAlert extends ZkIntegrationTestBase {
-  ZkClient _zkClient;
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr =
       "EXP(accumulate()(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
@@ -58,15 +55,11 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class AddDropAlertTransition extends MockTransition {
@@ -114,7 +107,7 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
   @Test()
   public void testAddDropAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START TestAddDropAlert at " + new Date(System.currentTimeMillis()));
 
@@ -130,18 +123,18 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
 
     _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) // !!!change back to 5
     {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new AddDropAlertTransition());
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new AddDropAlertTransition());
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     boolean result =
@@ -152,10 +145,10 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
     // drop alert soon after adding, but leave enough time for alert to fire once
     // Thread.sleep(3000);
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     String instance = "localhost_12918";
     ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
     Map<String, Map<String, String>> recMap = record.getMapFields();
@@ -163,7 +156,7 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
     Assert.assertTrue(keySet.size() > 0);
 
     _setupTool.getClusterManagementTool().dropAlert(clusterName, _alertStr);
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     // other verifications go here
     // for (int i = 0; i < 1; i++) //change 1 back to 5
     // {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
index f1d2ba6..37f8205 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
@@ -38,10 +38,13 @@ import org.apache.helix.model.InstanceConfig.InstanceConfigProperty;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
+  private static Logger LOG = Logger.getLogger(TestAlertActionTriggering.class);
+
   String _statName = "TestStat@DB=db1";
   String _stat = "TestStat";
   String metricName1 = "TestMetric1";
@@ -49,8 +52,7 @@ public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropert
 
   void setHealthData(int[] val1, int[] val2) {
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+      HelixManager manager = _participants[i];
       ZNRecord record = new ZNRecord(_stat);
       Map<String, String> valMap = new HashMap<String, String>();
       valMap.put(metricName1, val1[i] + "");
@@ -66,15 +68,13 @@ public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropert
     try {
       Thread.sleep(1000);
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("sleep interrupted", e);
     }
   }
 
   void setHealthData2(int[] val1) {
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+      HelixManager manager = _participants[i];
       ZNRecord record = new ZNRecord(_stat);
       Map<String, String> valMap = new HashMap<String, String>();
       valMap.put(metricName2, val1[i] + "");
@@ -89,8 +89,7 @@ public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropert
     try {
       Thread.sleep(1000);
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("sleep interrupted", e);
     }
   }
 
@@ -125,11 +124,9 @@ public class TestAlertActionTriggering extends ZkStandAloneCMTestBaseWithPropert
     };
     setHealthData(metrics1, metrics2);
 
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller;
 
-    HealthStatsAggregator task =
-        new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager);
+    HealthStatsAggregator task = new HealthStatsAggregator(manager);
     task.aggregate();
     Thread.sleep(4000);
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
index 24595d0..8618b1c 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
@@ -27,14 +27,16 @@ import java.util.TreeMap;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
-import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.HelixTimerTask;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
 import org.apache.helix.model.AlertHistory;
 import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -44,6 +46,8 @@ import org.testng.annotations.Test;
  */
 
 public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServerCheck {
+  private final static Logger LOG = Logger.getLogger(TestAlertFireHistory.class);
+
   String _statName = "TestStat@DB=db1";
   String _stat = "TestStat";
   String metricName1 = "TestMetric1";
@@ -55,8 +59,7 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
 
   void setHealthData(int[] val1, int[] val2) {
     for (int i = 0; i < NODE_NR; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      HelixManager manager = _startCMResultMap.get(instanceName)._manager;
+      HelixManager manager = _participants[i];
       ZNRecord record = new ZNRecord(_stat);
       Map<String, String> valMap = new HashMap<String, String>();
       valMap.put(metricName1, val1[i] + "");
@@ -72,13 +75,12 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
     try {
       Thread.sleep(1000);
     } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Interrupted sleep", e);
     }
   }
 
   @Test
-  public void TestAlertDisable() throws InterruptedException {
+  public void testAlertDisable() throws InterruptedException {
 
     int[] metrics1 = {
         10, 15, 22, 24, 16
@@ -88,29 +90,27 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
     };
     setHealthData(metrics1, metrics2);
 
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller;
     manager.startTimerTasks();
 
     _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr1);
     _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr2);
 
-    // ConfigScope scope = new ConfigScopeBuilder().forCluster(CLUSTER_NAME).build();
     HelixConfigScope scope =
         new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(CLUSTER_NAME).build();
     Map<String, String> properties = new HashMap<String, String>();
     properties.put("healthChange.enabled", "false");
     _setupTool.getClusterManagementTool().setConfig(scope, properties);
 
-    HealthStatsAggregator task =
-        new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager);
+    HealthStatsAggregator task = new HealthStatsAggregator(_controller);
+
     task.aggregate();
     Thread.sleep(100);
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     Builder keyBuilder = helixDataAccessor.keyBuilder();
 
     AlertHistory history = manager.getHelixDataAccessor().getProperty(keyBuilder.alertHistory());
-    //
+
     Assert.assertEquals(history, null);
 
     properties.put("healthChange.enabled", "true");
@@ -126,7 +126,7 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
   }
 
   @Test
-  public void TestAlertHistory() throws InterruptedException {
+  public void testAlertHistory() throws InterruptedException {
     int[] metrics1 = {
         10, 15, 22, 24, 16
     };
@@ -135,9 +135,10 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
     };
     setHealthData(metrics1, metrics2);
 
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
-    manager.stopTimerTasks();
+    HelixManager manager = _controller;
+    for (HelixTimerTask task : _controller.getControllerTimerTasks()) {
+      task.stop();
+    }
 
     _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr1);
     _setupTool.getClusterManagementTool().addAlert(CLUSTER_NAME, _alertStr2);
@@ -152,8 +153,8 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
       historySize = property.getRecord().getMapFields().size();
     }
 
-    HealthStatsAggregator task =
-        new HealthStatsAggregator(_startCMResultMap.get(controllerName)._manager);
+    HealthStatsAggregator task = new HealthStatsAggregator(_controller);
+
     task.aggregate();
     Thread.sleep(100);
 
@@ -419,3 +420,4 @@ public class TestAlertFireHistory extends ZkStandAloneCMTestBaseWithPropertyServ
   }
 
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
index efbd3b4..29f4893 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
@@ -29,10 +29,10 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.Message;
@@ -73,7 +73,7 @@ public class TestDummyAlerts extends ZkIntegrationTestBase {
     String clusterName = className + "_" + methodName;
     final int n = 5;
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -95,15 +95,16 @@ public class TestDummyAlerts extends ZkIntegrationTestBase {
             "EXP(decay(1.0)(*.defaultPerfCounters@defaultPerfCounters.availableCPUs))CMP(GREATER)CON(2)");
 
     // start controller
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new DummyAlertsTransition());
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new DummyAlertsTransition());
       participants[i].syncStart();
     }
 
@@ -137,7 +138,6 @@ public class TestDummyAlerts extends ZkIntegrationTestBase {
     }
 
     // clean up
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
index f429b5f..23741c3 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
@@ -28,18 +28,16 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
@@ -50,7 +48,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestExpandAlert extends ZkIntegrationTestBase {
-  ZkClient _zkClient;
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr =
       "EXP(decay(1.0)(localhost_*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(16)";
@@ -60,15 +57,12 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class ExpandAlertTransition extends MockTransition {
@@ -119,7 +113,7 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
   @Test()
   public void testExpandAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START TestExpandAlert at " + new Date(System.currentTimeMillis()));
 
@@ -135,18 +129,19 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
 
     _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) // !!!change back to 5
     {
       String instanceName = "localhost_" + (12918 + i);
 
       participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new ExpandAlertTransition());
-      participants[i].start();
-      // new Thread(participants[i]).start();
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new ExpandAlertTransition());
+      participants[i].syncStart();
     }
 
     boolean result =
@@ -157,13 +152,14 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
     Thread.sleep(1000);
     // HealthAggregationTask is supposed to run by a timer every 30s
     // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    // new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     // sleep for a few seconds to give stats stage time to trigger
     Thread.sleep(3000);
 
     // other verifications go here
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
     // for (int i = 0; i < 1; i++) //change 1 back to 5
@@ -180,6 +176,12 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
     Assert.assertFalse(fired);
     // }
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+
+    }
     System.out.println("END TestExpandAlert at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
index 6d33df0..dbbd7aa 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
@@ -28,18 +28,16 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
@@ -50,7 +48,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestSimpleAlert extends ZkIntegrationTestBase {
-  ZkClient _zkClient;
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr =
       "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
@@ -59,15 +56,11 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
-
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class SimpleAlertTransition extends MockTransition {
@@ -124,7 +117,7 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
   @Test()
   public void testSimpleAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START TestSimpleAlert at " + new Date(System.currentTimeMillis()));
 
@@ -139,10 +132,11 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
 
     // enableHealthCheck(clusterName);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    cmResult._manager.startTimerTasks();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+    controller.startTimerTasks();
+
     _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
     // start participants
     for (int i = 0; i < 5; i++) // !!!change back to 5
@@ -150,9 +144,9 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
       String instanceName = "localhost_" + (12918 + i);
 
       participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new SimpleAlertTransition(15));
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new SimpleAlertTransition(15));
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     boolean result =
@@ -162,13 +156,14 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
 
     // HealthAggregationTask is supposed to run by a timer every 30s
     // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    // new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     // sleep for a few seconds to give stats stage time to trigger
     Thread.sleep(3000);
 
     // other verifications go here
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
     // for (int i = 0; i < 1; i++) //change 1 back to 5
     // {
@@ -196,6 +191,11 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
             .equals("ON"));
     // }
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
     System.out.println("END TestSimpleAlert at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
index 3eb31ed..90223ad 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
@@ -27,29 +27,29 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
+import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
-  ZkClient _zkClient;
+  private static Logger LOG = Logger.getLogger(TestSimpleWildcardAlert.class);
+
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr =
       "EXP(decay(1.0)(localhost_12918.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
@@ -58,15 +58,12 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class SimpleAlertTransition extends MockTransition {
@@ -123,7 +120,7 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
   @Test()
   public void testSimpleWildcardAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START testSimpleWildcardAlert at " + new Date(System.currentTimeMillis()));
 
@@ -138,10 +135,10 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
 
     // enableHealthCheck(clusterName);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    cmResult._manager.stopTimerTasks();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+    controller.stopTimerTasks();
 
     String alertwildcard =
         "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(10)";
@@ -153,9 +150,9 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
       String instanceName = "localhost_" + (12944 + i);
 
       participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new SimpleAlertTransition(i * 5));
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new SimpleAlertTransition(i * 5));
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     boolean result =
@@ -166,13 +163,13 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
     Thread.sleep(1000);
     // HealthAggregationTask is supposed to run by a timer every 30s
     // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     // sleep for a few seconds to give stats stage time to trigger
     Thread.sleep(1000);
 
     // other verifications go here
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
     ZNRecord record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
     Map<String, Map<String, String>> recMap = record.getMapFields();
@@ -208,7 +205,7 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
     alertwildcard =
         "EXP(decay(1.0)(localhost*.RestQueryStats@DBName=TestDB0.latency))CMP(GREATER)CON(15)";
     _setupTool.getClusterManagementTool().addAlert(clusterName, alertwildcard);
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     Thread.sleep(1000);
 
     record = accessor.getProperty(keyBuilder.alertStatus()).getRecord();
@@ -240,6 +237,11 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
       Assert.assertTrue(delta.get(alertString).equals("ON"));
     }
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
     System.out.println("END testSimpleWildcardAlert at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
index 2661560..76784d3 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
@@ -28,18 +28,16 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
@@ -50,7 +48,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestStalenessAlert extends ZkIntegrationTestBase {
-  ZkClient _zkClient;
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr = "EXP(decay(1)(localhost_*.reportingage))CMP(GREATER)CON(600)";
   protected final String _alertStatusStr = _alertStr + " : (localhost_12918.reportingage)";
@@ -58,15 +55,12 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class StalenessAlertTransition extends MockTransition {
@@ -117,7 +111,7 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
   @Test()
   public void testStalenessAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START TestStalenessAlert at " + new Date(System.currentTimeMillis()));
 
@@ -133,18 +127,19 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
 
     _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     // start participants
     for (int i = 0; i < 5; i++) // !!!change back to 5
     {
       String instanceName = "localhost_" + (12918 + i);
 
       participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new StalenessAlertTransition());
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new StalenessAlertTransition());
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     boolean result =
@@ -154,13 +149,13 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
 
     // HealthAggregationTask is supposed to run by a timer every 30s
     // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
     // sleep for a few seconds to give stats stage time to trigger
     Thread.sleep(3000);
 
     // other verifications go here
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
     // for (int i = 0; i < 1; i++) //change 1 back to 5
     // {
@@ -176,6 +171,11 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
     // Assert.assertFalse(fired);
     // }
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
     System.out.println("END TestStalenessAlert at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
index 5265ebb..79c85ca 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
@@ -43,18 +43,16 @@ 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.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.MockEspressoHealthReportProvider;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
 import org.apache.helix.model.Message;
 import org.apache.helix.monitoring.mbeans.ClusterAlertMBeanCollection;
@@ -125,7 +123,6 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
   }
 
   private static final Logger _logger = Logger.getLogger(TestWildcardAlert.class);
-  ZkClient _zkClient;
   protected ClusterSetup _setupTool = null;
   protected final String _alertStr =
       "EXP(decay(1)(localhost_*.RestQueryStats@DBName=TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)";
@@ -134,15 +131,12 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
 
   @BeforeClass()
   public void beforeClass() throws Exception {
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
 
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
   }
 
   @AfterClass
   public void afterClass() {
-    _zkClient.close();
   }
 
   public class WildcardAlertTransition extends MockTransition {
@@ -208,7 +202,7 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
   @Test()
   public void testWildcardAlert() throws Exception {
     String clusterName = getShortClassName();
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
 
     System.out.println("START TestWildcardAlert at " + new Date(System.currentTimeMillis()));
 
@@ -227,18 +221,18 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
     _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr);
     // _setupTool.getClusterManagementTool().addAlert(clusterName, _alertStr2);
 
-    StartCMResult cmResult =
-        TestHelper.startController(clusterName, "controller_0", ZK_ADDR,
-            HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
     // start participants
     for (int i = 0; i < 5; i++) // !!!change back to 5
     {
       String instanceName = "localhost_" + (12918 + i);
 
       participants[i] =
-          new MockParticipant(clusterName, instanceName, ZK_ADDR, new WildcardAlertTransition());
+          new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].setTransition(new WildcardAlertTransition());
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     TestClusterMBeanObserver jmxMBeanObserver =
@@ -251,13 +245,13 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
     Thread.sleep(3000);
     // HealthAggregationTask is supposed to run by a timer every 30s
     // To make sure HealthAggregationTask is run, we invoke it explicitly for this test
-    new HealthStatsAggregator(cmResult._manager).aggregate();
+    new HealthStatsAggregator(controller).aggregate();
 
     // sleep for a few seconds to give stats stage time to trigger and for bean to trigger
     Thread.sleep(3000);
 
     ZKHelixDataAccessor accessor =
-        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
 
     // for (int i = 0; i < 1; i++) //change 1 back to 5
@@ -292,6 +286,12 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
             "EXP(decay(1)(localhost_%.RestQueryStats@DBName#TestDB0.latency)|EXPAND|SUMEACH)CMP(GREATER)CON(10)--(%)");
     // }
 
+    // clean up
+    controller.syncStop();
+    for (int i = 0; i < 5; i++) {
+      participants[i].syncStop();
+    }
+
     System.out.println("END TestWildcardAlert at " + new Date(System.currentTimeMillis()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 82e45cc..32fdcff 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
@@ -20,14 +20,9 @@ package org.apache.helix.integration;
  */
 
 import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.controller.HelixControllerMain;
+
+import org.apache.helix.integration.manager.ClusterDistributedController;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
@@ -44,13 +39,15 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase {
   protected static final int START_PORT = 12918;
   protected static final String STATE_MODEL = "MasterSlave";
   protected ClusterSetup _setupTool = null;
-  protected Map<String, StartCMResult> _startCMResultMap = new HashMap<String, StartCMResult>();
 
   protected final String CLASS_NAME = getShortClassName();
   protected final String CONTROLLER_CLUSTER = CONTROLLER_CLUSTER_PREFIX + "_" + CLASS_NAME;
 
   protected static final String TEST_DB = "TestDB";
 
+  MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
+  ClusterDistributedController[] _distControllers = new ClusterDistributedController[NODE_NR];
+
   @BeforeClass
   public void beforeClass() throws Exception {
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
@@ -88,29 +85,18 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase {
         "MasterSlave", 3, true);
 
     // start dummy participants for the first cluster
-    for (int i = 0; i < 5; i++) {
+    for (int i = 0; i < NODE_NR; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      if (_startCMResultMap.get(instanceName) != null) {
-        LOG.error("fail to start participant:" + instanceName
-            + "(participant with the same name already running");
-      } else {
-        StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, firstCluster, instanceName);
-        _startCMResultMap.put(instanceName, result);
-      }
+      _participants[i] = new MockParticipantManager(ZK_ADDR, firstCluster, instanceName);
+      _participants[i].syncStart();
     }
 
     // start distributed cluster controllers
-    for (int i = 0; i < 5; i++) {
+    for (int i = 0; i < NODE_NR; i++) {
       String controllerName = CONTROLLER_PREFIX + "_" + i;
-      if (_startCMResultMap.get(controllerName) != null) {
-        LOG.error("fail to start controller:" + controllerName
-            + "(controller with the same name already running");
-      } else {
-        StartCMResult result =
-            TestHelper.startController(CONTROLLER_CLUSTER, controllerName, ZK_ADDR,
-                HelixControllerMain.DISTRIBUTED);
-        _startCMResultMap.put(controllerName, result);
-      }
+      _distControllers[i] =
+          new ClusterDistributedController(ZK_ADDR, CONTROLLER_CLUSTER, controllerName);
+      _distControllers[i].syncStart();
     }
 
     verifyClusters();
@@ -132,36 +118,22 @@ public class TestAddClusterV2 extends ZkIntegrationTestBase {
      * 3) disconnect leader/disconnect participant
      */
     String leader = getCurrentLeader(_gZkClient, CONTROLLER_CLUSTER);
-    // pauseController(_startCMResultMap.get(leader)._manager.getDataAccessor());
-
-    StartCMResult result;
-
-    Iterator<Entry<String, StartCMResult>> it = _startCMResultMap.entrySet().iterator();
-
-    while (it.hasNext()) {
-      String instanceName = it.next().getKey();
-      if (!instanceName.equals(leader) && instanceName.startsWith(CONTROLLER_PREFIX)) {
-        result = _startCMResultMap.get(instanceName);
-        result._manager.disconnect();
-        result._thread.interrupt();
-        it.remove();
+    int leaderIdx = -1;
+    for (int i = 0; i < NODE_NR; i++) {
+      if (!_distControllers[i].getInstanceName().equals(leader)) {
+        _distControllers[i].syncStop();
+        verifyClusters();
+      } else {
+        leaderIdx = i;
       }
-      verifyClusters();
     }
+    Assert.assertNotSame(leaderIdx, -1);
 
-    result = _startCMResultMap.remove(leader);
-    result._manager.disconnect();
-    result._thread.interrupt();
-
-    it = _startCMResultMap.entrySet().iterator();
-    while (it.hasNext()) {
-      String instanceName = it.next().getKey();
-      result = _startCMResultMap.get(instanceName);
-      result._manager.disconnect();
-      result._thread.interrupt();
-      it.remove();
-    }
+    _distControllers[leaderIdx].syncStop();
 
+    for (int i = 0; i < NODE_NR; i++) {
+      _participants[i].syncStop();
+    }
     System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 d1014ed..79d8b89 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
@@ -22,15 +22,14 @@ package org.apache.helix.integration;
 import java.util.Date;
 import java.util.List;
 
-import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
-import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.controller.HelixControllerMain;
+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.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
@@ -50,16 +49,17 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, "localhost", "TestDB", 1, 20, nodeNr - 1,
         3, "MasterSlave", true);
 
-    MockParticipant[] participants = new MockParticipant[nodeNr];
+    MockParticipantManager[] participants = new MockParticipantManager[nodeNr];
     for (int i = 0; i < nodeNr - 1; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
-      new Thread(participants[i]).start();
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
+      participants[i].syncStart();
     }
 
-    ZkHelixTestManager controller =
-        new ZkHelixTestManager(clusterName, "controller_0", InstanceType.CONTROLLER, ZK_ADDR);
-    controller.connect();
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
+
     boolean result;
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -73,7 +73,7 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     _gSetupTool.addInstanceToCluster(clusterName, "localhost_12922");
     _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 3);
 
-    participants[nodeNr - 1] = new MockParticipant(clusterName, "localhost_12922", ZK_ADDR);
+    participants[nodeNr - 1] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12922");
     new Thread(participants[nodeNr - 1]).start();
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
@@ -84,11 +84,10 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
-    // controller.disconnect();
-    // for (int i = 0; i < nodeNr; i++)
-    // {
-    // participants[i].syncStop();
-    // }
+    controller.syncStop();
+    for (int i = 0; i < nodeNr; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }
@@ -99,11 +98,13 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
     // setup grand cluster
-    TestHelper.setupCluster("GRAND_" + clusterName, ZK_ADDR, 0, "controller", null, 0, 0, 1, 0,
+    final String grandClusterName = "GRAND_" + clusterName;
+    TestHelper.setupCluster(grandClusterName, ZK_ADDR, 0, "controller", null, 0, 0, 1, 0,
         null, true);
 
-    TestHelper.startController("GRAND_" + clusterName, "controller_0", ZK_ADDR,
-        HelixControllerMain.DISTRIBUTED);
+    ClusterDistributedController distController =
+        new ClusterDistributedController(ZK_ADDR, grandClusterName, "controller_0");
+    distController.syncStart();
 
     // setup cluster
     _gSetupTool.addCluster(clusterName, true);
@@ -125,12 +126,11 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     _gSetupTool.addResourceToCluster(clusterName, "TestDB0", 1, "LeaderStandby");
     _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 1);
 
-    MockParticipant[] participants = new MockParticipant[nodeNr];
+    MockParticipantManager[] participants = new MockParticipantManager[nodeNr];
     for (int i = 0; i < nodeNr - 1; i++) {
       String instanceName = "localhost_" + (12918 + i);
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
-      // new Thread(participants[i]).start();
     }
 
     result =
@@ -148,10 +148,8 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     _gSetupTool.addInstanceToCluster(clusterName, "localhost_12919");
     _gSetupTool.rebalanceStorageCluster(clusterName, "TestDB0", 2);
 
-    participants[nodeNr - 1] = new MockParticipant(clusterName, "localhost_12919", ZK_ADDR);
+    participants[nodeNr - 1] = new MockParticipantManager(ZK_ADDR, clusterName, "localhost_12919");
     participants[nodeNr - 1].syncStart();
-    // new Thread(participants[nodeNr - 1]).start();
-
     result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             ZK_ADDR, clusterName));
@@ -163,10 +161,10 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase {
     Assert.assertEquals(numberOfListeners, 2); // 1 of participant, and 1 of controller
 
     // clean up
-    // for (int i = 0; i < nodeNr; i++)
-    // {
-    // participants[i].syncStop();
-    // }
+    distController.syncStop();
+    for (int i = 0; i < nodeNr; i++) {
+      participants[i].syncStop();
+    }
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 ba4eee2..123ce6e 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
@@ -28,9 +28,9 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
-import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockMSModelFactory;
-import org.apache.helix.mock.participant.MockParticipant;
+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.tools.ClusterSetup;
@@ -50,7 +50,7 @@ public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase
 
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
-    MockParticipant[] participants = new MockParticipant[n];
+    MockParticipantManager[] participants = new MockParticipantManager[n];
 
     TestHelper.setupCluster(clusterName, ZK_ADDR, 12918, // participant port
         "localhost", // participant name prefix
@@ -61,14 +61,15 @@ public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase
         3, // replicas
         "MasterSlave", true); // do rebalance
 
-    ClusterController controller = new ClusterController(clusterName, "controller_0", ZK_ADDR);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
     // start participants
     for (int i = 0; i < n; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -114,7 +115,7 @@ public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase
     // register "TestDB1_Factory" state model factory
     // Logger.getRootLogger().setLevel(Level.INFO);
     for (int i = 0; i < n; i++) {
-      participants[i].getManager().getStateMachineEngine()
+      participants[i].getStateMachineEngine()
           .registerStateModelFactory("MasterSlave", new MockMSModelFactory(), "TestDB1_Factory");
     }
 
@@ -125,7 +126,6 @@ public class TestAddStateModelFactoryAfterConnect extends ZkIntegrationTestBase
 
     // clean up
     // wait for all zk callbacks done
-    Thread.sleep(1000);
     controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 a008814..fe23cb9 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
@@ -27,9 +27,9 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
-import org.apache.helix.mock.participant.MockParticipant;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
@@ -74,15 +74,16 @@ public class TestAutoIsWithEmptyMap extends ZkIntegrationTestBase {
     _gZkClient.writeData(idealPath, curIdealState);
 
     // start controller
-    TestHelper
-        .startController(clusterName, "controller_0", ZK_ADDR, HelixControllerMain.STANDALONE);
+    ClusterControllerManager controller =
+        new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
+    controller.syncStart();
 
     // start participants
-    MockParticipant[] participants = new MockParticipant[5];
+    MockParticipantManager[] participants = new MockParticipantManager[5];
     for (int i = 0; i < 5; i++) {
       String instanceName = "localhost_" + (12918 + i);
 
-      participants[i] = new MockParticipant(clusterName, instanceName, ZK_ADDR, null);
+      participants[i] = new MockParticipantManager(ZK_ADDR, clusterName, instanceName);
       participants[i].syncStart();
     }
 
@@ -92,6 +93,7 @@ public class TestAutoIsWithEmptyMap extends ZkIntegrationTestBase {
     Assert.assertTrue(result);
 
     // clean up
+    controller.syncStop();
     for (int i = 0; i < 5; i++) {
       participants[i].syncStop();
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 0e7f4fa..2aec114 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
@@ -28,13 +28,13 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
 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.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;
@@ -62,13 +62,11 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
     String namespace = "/" + CLUSTER_NAME;
-    if (_zkClient.exists(namespace)) {
-      _zkClient.deleteRecursive(namespace);
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
     }
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
 
     // setup storage cluster
     _setupTool.addCluster(CLUSTER_NAME, true);
@@ -95,24 +93,21 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      if (_startCMResultMap.get(instanceName) != null) {
-        LOG.error("fail to start particpant:" + instanceName
-            + "(participant with same name already exists)");
-      } else {
-        StartCMResult result = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName);
-        _startCMResultMap.put(instanceName, result);
-      }
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      participant.syncStart();
+      _participants[i] = participant;
+
     }
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult startResult =
-        TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    _startCMResultMap.put(controllerName, startResult);
+    _controller =
+        new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
 
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
 
     Assert.assertTrue(result);
@@ -128,7 +123,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB", 1);
 
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, "MyDB"));
     Assert.assertTrue(result);
 
@@ -146,7 +141,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, "MyDB2", 3);
 
     result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, "MyDB2"));
     Assert.assertTrue(result);
 
@@ -160,16 +155,11 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
 
   @Test()
   public void testAutoRebalance() throws Exception {
-
     // kill 1 node
-    String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    _startCMResultMap.get(instanceName)._manager.disconnect();
-    Thread.sleep(1000);
-    _startCMResultMap.get(instanceName)._thread.interrupt();
+    _participants[0].syncStop();
 
-    // verifyBalanceExternalView();
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
 
@@ -178,22 +168,22 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i);
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
-      StartCMResult resultx =
-          TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_'));
-      _startCMResultMap.put(storageNodeName, resultx);
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_'));
+      participant.syncStart();
     }
     Thread.sleep(5000);
     result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
 
     result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, db2));
     Assert.assertTrue(result);
     HelixDataAccessor accessor =
-        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2));
     Set<String> instancesSet = new HashSet<String>();
@@ -246,12 +236,10 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
   }
 
   public static class ExternalViewBalancedVerifier implements ZkVerifier {
-    ZkClient _client;
     String _clusterName;
     String _resourceName;
 
     public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String resourceName) {
-      _client = client;
       _clusterName = clusterName;
       _resourceName = resourceName;
     }
@@ -259,7 +247,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
     @Override
     public boolean verify() {
       HelixDataAccessor accessor =
-          new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_client));
+          new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
       Builder keyBuilder = accessor.keyBuilder();
       IdealState idealState = accessor.getProperty(keyBuilder.idealStates(_resourceName));
       if (idealState == null) {
@@ -298,7 +286,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
 
     @Override
     public ZkClient getZkClient() {
-      return _client;
+      return _gZkClient;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 3523461..5f9f48c 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
@@ -27,12 +27,12 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
-import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
-import org.apache.helix.controller.HelixControllerMain;
+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.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;
@@ -58,13 +58,11 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
     // Logger.getRootLogger().setLevel(Level.INFO);
     System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
 
-    _zkClient = new ZkClient(ZK_ADDR);
-    _zkClient.setZkSerializer(new ZNRecordSerializer());
     String namespace = "/" + CLUSTER_NAME;
-    if (_zkClient.exists(namespace)) {
-      _zkClient.deleteRecursive(namespace);
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
     }
-    _setupTool = new ClusterSetup(ZK_ADDR);
+    _setupTool = new ClusterSetup(_gZkClient);
 
     // setup storage cluster
     _setupTool.addCluster(CLUSTER_NAME, true);
@@ -79,40 +77,33 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
-    StartCMResult startResult =
-        TestHelper.startController(CLUSTER_NAME, controllerName, ZK_ADDR,
-            HelixControllerMain.STANDALONE);
-    _startCMResultMap.put(controllerName, startResult);
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
 
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller; // _startCMResultMap.get(controllerName)._manager;
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     // start dummy participants
     for (int i = 0; i < NODE_NR; i++) {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      if (_startCMResultMap.get(instanceName) != null) {
-        LOG.error("fail to start particpant:" + instanceName
-            + "(participant with same name already exists)");
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].syncStart();
+      Thread.sleep(2000);
+      boolean result =
+          ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
+              CLUSTER_NAME, TEST_DB));
+      Assert.assertTrue(result);
+      ExternalView ev =
+          manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
+      System.out.println(ev.getPartitionSet().size());
+      if (i < 3) {
+        Assert.assertEquals(ev.getPartitionSet().size(), 25 * (i + 1));
       } else {
-        startResult = TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, instanceName);
-        _startCMResultMap.put(instanceName, startResult);
-        Thread.sleep(2000);
-        boolean result =
-            ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
-                CLUSTER_NAME, TEST_DB));
-        Assert.assertTrue(result);
-        ExternalView ev =
-            manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
-        System.out.println(ev.getPartitionSet().size());
-        if (i < 3) {
-          Assert.assertEquals(ev.getPartitionSet().size(), 25 * (i + 1));
-        } else {
-          Assert.assertEquals(ev.getPartitionSet().size(), 100);
-        }
+        Assert.assertEquals(ev.getPartitionSet().size(), 100);
       }
     }
 
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
 
     Assert.assertTrue(result);
@@ -120,49 +111,50 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
 
   @Test()
   public void testAutoRebalanceWithMaxPartitionPerNode() throws Exception {
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    HelixManager manager = _startCMResultMap.get(controllerName)._manager;
+    HelixManager manager = _controller;
     // kill 1 node
-    String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 0);
-    _startCMResultMap.get(instanceName)._manager.disconnect();
-    Thread.sleep(1000);
-    _startCMResultMap.get(instanceName)._thread.interrupt();
+    _participants[0].syncStop();
 
     // verifyBalanceExternalView();
     boolean result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    final HelixDataAccessor accessor = manager.getHelixDataAccessor();
     ExternalView ev =
         manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
     Assert.assertEquals(ev.getPartitionSet().size(), 100);
 
-    instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1);
-    _startCMResultMap.get(instanceName)._manager.disconnect();
-    Thread.sleep(1000);
-    _startCMResultMap.get(instanceName)._thread.interrupt();
+    _participants[1].syncStop();
 
     // verifyBalanceExternalView();
     result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
-    ev = manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
-    Assert.assertEquals(ev.getPartitionSet().size(), 75);
+    result = TestHelper.verify(new TestHelper.Verifier() {
+
+      @Override
+      public boolean verify() throws Exception {
+        ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView(TEST_DB));
+        return ev.getPartitionSet().size() == 75;
+      }
+    }, 3 * 1000);
 
     // add 2 nodes
     for (int i = 0; i < 2; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (1000 + i);
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
 
-      StartCMResult resultx =
-          TestHelper.startDummyProcess(ZK_ADDR, CLUSTER_NAME, storageNodeName.replace(':', '_'));
-      _startCMResultMap.put(storageNodeName, resultx);
+      String newInstanceName = storageNodeName.replace(':', '_');
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, newInstanceName);
+      participant.syncStart();
     }
-    Thread.sleep(1000);
+
+    // Thread.sleep(1000);
     result =
-        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_zkClient,
+        ClusterStateVerifier.verifyByZkCallback(new ExternalViewBalancedVerifier(_gZkClient,
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
   }
@@ -213,12 +205,10 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
   }
 
   public static class ExternalViewBalancedVerifier implements ZkVerifier {
-    ZkClient _client;
     String _clusterName;
     String _resourceName;
 
     public ExternalViewBalancedVerifier(ZkClient client, String clusterName, String resourceName) {
-      _client = client;
       _clusterName = clusterName;
       _resourceName = resourceName;
     }
@@ -226,7 +216,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
     @Override
     public boolean verify() {
       HelixDataAccessor accessor =
-          new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_client));
+          new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_gZkClient));
       Builder keyBuilder = accessor.keyBuilder();
       IdealState idealState = accessor.getProperty(keyBuilder.idealStates(_resourceName));
       int numberOfPartitions = idealState.getRecord().getListFields().size();
@@ -244,7 +234,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
 
     @Override
     public ZkClient getZkClient() {
-      return _client;
+      return _gZkClient;
     }
 
     @Override


[08/10] [HELIX-279] Apply gc handling fixes to ZKHelixManager

Posted by zz...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 933bf78..96b1ac8 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
@@ -22,6 +22,7 @@ package org.apache.helix.controller.stages;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.State;
@@ -40,6 +41,8 @@ import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Sets;
+
 /**
  * For partition compute best possible (instance,state) pair based on
  * IdealState,StateModel,LiveInstance
@@ -86,7 +89,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
   private ResourceAssignment mapDroppedResource(Cluster cluster, ResourceId resourceId,
       ResourceCurrentState currentStateOutput, StateModelDefinition stateModelDef) {
     ResourceAssignment partitionMapping = new ResourceAssignment(resourceId);
-    Set<? extends PartitionId> mappedPartitions =
+    Set<PartitionId> mappedPartitions =
         currentStateOutput.getCurrentStateMappedPartitions(resourceId);
     if (mappedPartitions == null) {
       return partitionMapping;
@@ -106,6 +109,58 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     return partitionMapping;
   }
 
+  /**
+   * Update a ResourceAssignment with dropped and disabled participants for partitions
+   * @param cluster cluster snapshot
+   * @param resourceAssignment current resource assignment
+   * @param currentStateOutput aggregated current state
+   * @param stateModelDef state model definition for the resource
+   */
+  private void mapDroppedAndDisabledPartitions(Cluster cluster,
+      ResourceAssignment resourceAssignment, ResourceCurrentState currentStateOutput,
+      StateModelDefinition stateModelDef) {
+    // get the total partition set: mapped and current state
+    ResourceId resourceId = resourceAssignment.getResourceId();
+    Set<PartitionId> mappedPartitions = Sets.newHashSet();
+    mappedPartitions.addAll(currentStateOutput.getCurrentStateMappedPartitions(resourceId));
+    mappedPartitions.addAll(resourceAssignment.getMappedPartitionIds());
+    for (PartitionId partitionId : mappedPartitions) {
+      // for each partition, get the dropped and disabled mappings
+      Set<ParticipantId> disabledParticipants =
+          ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
+              partitionId);
+
+      // get the error participants
+      Map<ParticipantId, State> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceId, partitionId);
+      Set<ParticipantId> errorParticipants = Sets.newHashSet();
+      for (ParticipantId participantId : currentStateMap.keySet()) {
+        State state = currentStateMap.get(participantId);
+        if (state.equals(State.from(HelixDefinedState.ERROR))) {
+          errorParticipants.add(participantId);
+        }
+      }
+
+      // get the dropped and disabled map
+      State initialState = stateModelDef.getTypedInitialState();
+      Map<ParticipantId, State> participantStateMap = resourceAssignment.getReplicaMap(partitionId);
+      Set<ParticipantId> participants = participantStateMap.keySet();
+      Map<ParticipantId, State> droppedAndDisabledMap =
+          ConstraintBasedAssignment.dropAndDisablePartitions(currentStateMap, participants,
+              disabledParticipants, initialState);
+
+      // don't map error participants
+      for (ParticipantId participantId : errorParticipants) {
+        droppedAndDisabledMap.remove(participantId);
+      }
+      // save the mappings, overwriting as necessary
+      participantStateMap.putAll(droppedAndDisabledMap);
+
+      // include this add step in case the resource assignment did not already map this partition
+      resourceAssignment.addReplicaMap(partitionId, participantStateMap);
+    }
+  }
+
   private BestPossibleStateOutput compute(Cluster cluster, ClusterEvent event,
       Map<ResourceId, ResourceConfig> resourceMap, ResourceCurrentState currentStateOutput) {
     BestPossibleStateOutput output = new BestPossibleStateOutput();
@@ -127,11 +182,14 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
               rebalancer.computeResourceMapping(rebalancerConfig, cluster, currentStateOutput);
         }
       }
+      RebalancerContext context = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
+      StateModelDefinition stateModelDef = stateModelDefs.get(context.getStateModelDefId());
       if (resourceAssignment == null) {
-        RebalancerContext context = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
-        StateModelDefinition stateModelDef = stateModelDefs.get(context.getStateModelDefId());
         resourceAssignment =
             mapDroppedResource(cluster, resourceId, currentStateOutput, stateModelDef);
+      } else {
+        mapDroppedAndDisabledPartitions(cluster, resourceAssignment, currentStateOutput,
+            stateModelDef);
       }
       output.setResourceAssignment(resourceId, resourceAssignment);
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
index 5730289..c036b14 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
@@ -68,7 +68,8 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
           continue;
         }
 
-        if (!liveParticipant.getRunningInstance().getSessionId().equals(message.getTypedTgtSessionId())) {
+        if (!liveParticipant.getRunningInstance().getSessionId()
+            .equals(message.getTypedTgtSessionId())) {
           continue;
         }
 
@@ -126,17 +127,11 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
 
         Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
         for (PartitionId partitionId : partitionStateMap.keySet()) {
-          Partition partition = resource.getSubUnit(partitionId);
-          if (partition != null) {
-            currentStateOutput.setCurrentState(resourceId, partitionId, participantId,
-                curState.getState(partitionId));
-          } else {
-            // log
-          }
+          currentStateOutput.setCurrentState(resourceId, partitionId, participantId,
+              curState.getState(partitionId));
         }
       }
     }
-
     event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
index d6fe8c3..08e6799 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
@@ -82,7 +82,7 @@ public class MessageGenerationStage extends AbstractBaseStage {
 
       ResourceAssignment resourceAssignment =
           bestPossibleStateOutput.getResourceAssignment(resourceId);
-      for (PartitionId subUnitId : resourceConfig.getSubUnitMap().keySet()) {
+      for (PartitionId subUnitId : resourceAssignment.getMappedPartitionIds()) {
         Map<ParticipantId, State> instanceStateMap = resourceAssignment.getReplicaMap(subUnitId);
 
         // we should generate message based on the desired-state priority

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
index 4a3fe28..bbbf5c6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
@@ -95,11 +95,13 @@ public class MessageSelectionStage extends AbstractBaseStage {
         event.getAttribute(AttributeName.RESOURCES.toString());
     ResourceCurrentState currentStateOutput =
         event.getAttribute(AttributeName.CURRENT_STATE.toString());
+    BestPossibleStateOutput bestPossibleStateOutput =
+        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     MessageOutput messageGenOutput = event.getAttribute(AttributeName.MESSAGES_ALL.toString());
     if (cluster == null || resourceMap == null || currentStateOutput == null
-        || messageGenOutput == null) {
+        || messageGenOutput == null || bestPossibleStateOutput == null) {
       throw new StageException("Missing attributes in event:" + event
-          + ". Requires DataCache|RESOURCES|CURRENT_STATE|MESSAGES_ALL");
+          + ". Requires DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE|MESSAGES_ALL");
     }
 
     MessageOutput output = new MessageOutput();
@@ -120,7 +122,8 @@ public class MessageSelectionStage extends AbstractBaseStage {
               configResource == null ? null : configResource.getRebalancerConfig(), cluster);
 
       // TODO fix it
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+      for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId)
+          .getMappedPartitionIds()) {
         List<Message> messages = messageGenOutput.getMessages(resourceId, partitionId);
         List<Message> selectedMessages =
             selectMessages(cluster.getLiveParticipantMap(),

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
index a7b75a3..764b422 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
@@ -121,10 +121,13 @@ public class MessageThrottleStage extends AbstractBaseStage {
         event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     Map<ResourceId, ResourceConfig> resourceMap =
         event.getAttribute(AttributeName.RESOURCES.toString());
+    BestPossibleStateOutput bestPossibleStateOutput =
+        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
 
-    if (cluster == null || resourceMap == null || msgSelectionOutput == null) {
+    if (cluster == null || resourceMap == null || msgSelectionOutput == null
+        || bestPossibleStateOutput == null) {
       throw new StageException("Missing attributes in event: " + event
-          + ". Requires ClusterDataCache|RESOURCES|MESSAGES_SELECTED");
+          + ". Requires ClusterDataCache|RESOURCES|BEST_POSSIBLE_STATE|MESSAGES_SELECTED");
     }
 
     MessageOutput output = new MessageOutput();
@@ -145,9 +148,9 @@ public class MessageThrottleStage extends AbstractBaseStage {
     // go through all new messages, throttle if necessary
     // assume messages should be sorted by state transition priority in messageSelection stage
     for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resource = resourceMap.get(resourceId);
       // TODO fix it
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+      for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId)
+          .getMappedPartitionIds()) {
         List<Message> messages = msgSelectionOutput.getMessages(resourceId, partitionId);
         if (constraint != null && messages != null && messages.size() > 0) {
           messages = throttle(throttleCounterMap, constraint, messages, true);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
index 3dd3b81..2f5ec1d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
@@ -31,6 +31,8 @@ import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.CurrentState;
 
+import com.google.common.collect.Sets;
+
 public class ResourceCurrentState {
   /**
    * map of resource-id to map of partition-id to map of participant-id to state
@@ -225,12 +227,17 @@ public class ResourceCurrentState {
    * @param resourceId resource to look up
    * @return set of mapped partitions, or empty set if there are none
    */
-  public Set<? extends PartitionId> getCurrentStateMappedPartitions(ResourceId resourceId) {
+  public Set<PartitionId> getCurrentStateMappedPartitions(ResourceId resourceId) {
     Map<PartitionId, Map<ParticipantId, State>> currentStateMap = _currentStateMap.get(resourceId);
+    Map<PartitionId, Map<ParticipantId, State>> pendingStateMap = _pendingStateMap.get(resourceId);
+    Set<PartitionId> partitionSet = Sets.newHashSet();
     if (currentStateMap != null) {
-      return currentStateMap.keySet();
+      partitionSet.addAll(currentStateMap.keySet());
+    }
+    if (pendingStateMap != null) {
+      partitionSet.addAll(pendingStateMap.keySet());
     }
-    return Collections.emptySet();
+    return partitionSet;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 02188be..bc2ee50 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
@@ -53,20 +53,24 @@ public class TaskAssignmentStage extends AbstractBaseStage {
     Map<ResourceId, ResourceConfig> resourceMap =
         event.getAttribute(AttributeName.RESOURCES.toString());
     MessageOutput messageOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
+    BestPossibleStateOutput bestPossibleStateOutput =
+        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     Cluster cluster = event.getAttribute("ClusterDataCache");
     Map<ParticipantId, Participant> liveParticipantMap = cluster.getLiveParticipantMap();
 
     if (manager == null || resourceMap == null || messageOutput == null || cluster == null
         || liveParticipantMap == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|DataCache|liveInstanceMap");
+      throw new StageException(
+          "Missing attributes in event:"
+              + event
+              + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|BEST_POSSIBLE_STATE|DataCache|liveInstanceMap");
     }
 
     HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
     List<Message> messagesToSend = new ArrayList<Message>();
     for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resource = resourceMap.get(resourceId);
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+      for (PartitionId partitionId : bestPossibleStateOutput.getResourceAssignment(resourceId)
+          .getMappedPartitionIds()) {
         List<Message> messages = messageOutput.getMessages(resourceId, partitionId);
         messagesToSend.addAll(messages);
       }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
deleted file mode 100644
index f623ca5..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
+++ /dev/null
@@ -1,691 +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 java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-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.HealthStateChangeListener;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixConstants.ChangeType;
-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.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.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.healthcheck.ParticipantHealthReportCollector;
-import org.apache.helix.messaging.DefaultMessagingService;
-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;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper.States;
-
-public abstract class AbstractManager implements HelixManager, IZkStateListener {
-  private static Logger LOG = Logger.getLogger(AbstractManager.class);
-
-  final String _zkAddress;
-  final String _clusterName;
-  final String _instanceName;
-  final InstanceType _instanceType;
-  final int _sessionTimeout;
-  final List<PreConnectCallback> _preConnectCallbacks;
-  protected final List<CallbackHandler> _handlers;
-  final HelixManagerProperties _properties;
-
-  /**
-   * helix version#
-   */
-  final String _version;
-
-  protected ZkClient _zkclient = null;
-  final DefaultMessagingService _messagingService;
-
-  BaseDataAccessor<ZNRecord> _baseDataAccessor;
-  ZKHelixDataAccessor _dataAccessor;
-  final Builder _keyBuilder;
-  ConfigAccessor _configAccessor;
-  ZkHelixPropertyStore<ZNRecord> _helixPropertyStore;
-  LiveInstanceInfoProvider _liveInstanceInfoProvider = null;
-  final List<HelixTimerTask> _timerTasks = new ArrayList<HelixTimerTask>();
-
-  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
-   */
-  final List<Long> _disconnectTimeHistory = new LinkedList<Long>();
-
-  final int _flappingTimeWindowMs;
-  final int _maxDisconnectThreshold;
-
-  public AbstractManager(String zkAddress, String clusterName, String instanceName,
-      InstanceType instanceType) {
-
-    LOG.info("Create a zk-based cluster manager. zkSvr: " + zkAddress + ", clusterName: "
-        + clusterName + ", instanceName: " + instanceName + ", type: " + instanceType);
-
-    _zkAddress = zkAddress;
-    _clusterName = clusterName;
-    _instanceType = instanceType;
-    _instanceName = instanceName;
-    _preConnectCallbacks = new LinkedList<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);
-
-    _sessionTimeout =
-        getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT);
-
-  }
-
-  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);
-    }
-
-    return propertyDefaultValue;
-  }
-
-  /**
-   * different types of helix manager should impl its own handle new session logic
-   */
-  // public abstract void handleNewSession();
-
-  @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;
-    }
-  }
-
-  @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;
-  }
-
-  /**
-   * specific disconnect logic for each helix-manager type
-   */
-  abstract void doDisconnect();
-
-  /**
-   * This function can be called when the connection are in bad state(e.g. flapping),
-   * in which isConnected() could be false and we want to disconnect from cluster.
-   */
-  @Override
-  public void disconnect() {
-    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();
-
-      _dataAccessor.shutdown();
-
-      doDisconnect();
-
-      _zkclient.unsubscribeAll();
-    } finally {
-      _zkclient.close();
-      LOG.info("Cluster manager: " + _instanceName + " disconnected");
-    }
-  }
-
-  @Override
-  public void addIdealStateChangeListener(IdealStateChangeListener listener) throws Exception {
-    addListener(listener, new Builder(_clusterName).idealStates(), ChangeType.IDEAL_STATE,
-        new EventType[] {
-            EventType.NodeDataChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  @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
-        });
-  }
-
-  @Override
-  public void addInstanceConfigChangeListener(InstanceConfigChangeListener listener)
-      throws Exception {
-    addListener(listener, new Builder(_clusterName).instanceConfigs(), ChangeType.INSTANCE_CONFIG,
-        new EventType[] {
-          EventType.NodeChildrenChanged
-        });
-  }
-
-  @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);
-    }
-  }
-
-  @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
-        });
-  }
-
-  @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
-        });
-  }
-
-  @Override
-  public void addHealthStateChangeListener(HealthStateChangeListener listener, String instanceName)
-      throws Exception {
-    addListener(listener, new Builder(_clusterName).healthReports(instanceName), ChangeType.HEALTH,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  @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
-        });
-  }
-
-  @Override
-  public void addControllerListener(ControllerChangeListener listener) {
-    addListener(listener, new Builder(_clusterName).controller(), ChangeType.CONTROLLER,
-        new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  void addControllerMessageListener(MessageListener listener) {
-    addListener(listener, new Builder(_clusterName).controllerMessages(),
-        ChangeType.MESSAGES_CONTROLLER, new EventType[] {
-            EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-        });
-  }
-
-  @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);
-
-      // handler.reset() may modify the handlers list, so do it outside the iteration
-      for (CallbackHandler handler : toRemove) {
-        handler.reset();
-      }
-    }
-
-    return true;
-  }
-
-  @Override
-  public HelixDataAccessor getHelixDataAccessor() {
-    checkConnected();
-    return _dataAccessor;
-  }
-
-  @Override
-  public ConfigAccessor getConfigAccessor() {
-    checkConnected();
-    return _configAccessor;
-  }
-
-  @Override
-  public String getClusterName() {
-    return _clusterName;
-  }
-
-  @Override
-  public String getInstanceName() {
-    return _instanceName;
-  }
-
-  @Override
-  public String getSessionId() {
-    checkConnected();
-    return _sessionId;
-  }
-
-  @Override
-  public long getLastNotificationTime() {
-    // TODO Auto-generated method stub
-    return 0;
-  }
-
-  @Override
-  public HelixAdmin getClusterManagmentTool() {
-    checkConnected();
-    if (_zkclient != null) {
-      return new ZKHelixAdmin(_zkclient);
-    }
-
-    LOG.error("Couldn't get ZKClusterManagementTool because zkclient is null");
-    return null;
-  }
-
-  @Override
-  public synchronized ZkHelixPropertyStore<ZNRecord> getHelixPropertyStore() {
-    checkConnected();
-
-    if (_helixPropertyStore == null) {
-      String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, _clusterName);
-
-      _helixPropertyStore =
-          new ZkHelixPropertyStore<ZNRecord>(new ZkBaseDataAccessor<ZNRecord>(_zkclient), path,
-              null);
-    }
-
-    return _helixPropertyStore;
-  }
-
-  @Override
-  public ClusterMessagingService getMessagingService() {
-    // The caller can register message handler factories on messaging service before the
-    // helix manager is connected. Thus we do not check connected here
-    return _messagingService;
-  }
-
-  @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    // helix-participant will override this
-    return null;
-  }
-
-  @Override
-  public InstanceType getInstanceType() {
-    return _instanceType;
-  }
-
-  @Override
-  public String getVersion() {
-    return _version;
-  }
-
-  @Override
-  public HelixManagerProperties getProperties() {
-    return _properties;
-  }
-
-  @Override
-  public StateMachineEngine getStateMachineEngine() {
-    // helix-participant will override this
-    return null;
-  }
-
-  @Override
-  public abstract boolean isLeader();
-
-  @Override
-  public void startTimerTasks() {
-    for (HelixTimerTask task : _timerTasks) {
-      task.start();
-    }
-
-  }
-
-  @Override
-  public void stopTimerTasks() {
-    for (HelixTimerTask task : _timerTasks) {
-      task.stop();
-    }
-
-  }
-
-  @Override
-  public void addPreConnectCallback(PreConnectCallback callback) {
-    LOG.info("Adding preconnect callback: " + callback);
-    _preConnectCallbacks.add(callback);
-  }
-
-  @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.
-   */
-  protected 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());
-  }
-
-  protected void checkConnected() {
-    if (!isConnected()) {
-      throw new HelixException("ClusterManager not connected. Call clusterManager.connect()");
-    }
-  }
-
-  protected 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;
-        }
-      }
-
-      CallbackHandler newHandler =
-          new CallbackHandler(this, _zkclient, propertyKey, listener, eventType, changeType);
-
-      _handlers.add(newHandler);
-      LOG.info("Added listener: " + listener + " for type: " + type + " to path: "
-          + newHandler.getPath());
-    }
-  }
-
-  protected void initHandlers(List<CallbackHandler> handlers) {
-    synchronized (this) {
-      if (handlers != null) {
-        for (CallbackHandler handler : handlers) {
-          handler.init();
-          LOG.info("init handler: " + handler.getPath() + ", " + handler.getListener());
-        }
-      }
-    }
-  }
-
-  protected 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());
-        }
-      }
-    }
-  }
-
-  /**
-   * different helix-manager may override this to have a cache-enabled based-data-accessor
-   * @param baseDataAccessor
-   * @return
-   */
-  BaseDataAccessor<ZNRecord> createBaseDataAccessor(ZkBaseDataAccessor<ZNRecord> baseDataAccessor) {
-    return baseDataAccessor;
-  }
-
-  void createClient() throws Exception {
-    PathBasedZkSerializer zkSerializer =
-        ChainedPathZkSerializer.builder(new ZNRecordStreamingSerializer()).build();
-
-    _zkclient =
-        new ZkClient(_zkAddress, _sessionTimeout, ZkClient.DEFAULT_CONNECTION_TIMEOUT, zkSerializer);
-
-    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkclient);
-
-    _baseDataAccessor = createBaseDataAccessor(baseDataAccessor);
-
-    _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;
-        }
-      }
-    }
-  }
-
-  // TODO separate out flapping detection code
-  @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();
-      }
-      break;
-    case Expired:
-      LOG.info("KeeperState:" + state + ", expiredSessionId: " + _sessionId + ", instance: "
-          + _instanceName + ", type: " + _instanceType);
-      break;
-    default:
-      break;
-    }
-  }
-
-  /**
-   * 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.
-   */
-  private boolean isFlapping() {
-    if (_disconnectTimeHistory.size() == 0) {
-      return false;
-    }
-    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);
-    }
-    return _disconnectTimeHistory.size() > _maxDisconnectThreshold;
-  }
-
-  /**
-   * controller should override it to return a list of timers that need to start/stop when
-   * leadership changes
-   * @return
-   */
-  protected List<HelixTimerTask> getControllerHelixTimerTasks() {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
deleted file mode 100644
index dd8e9be..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
+++ /dev/null
@@ -1,174 +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 java.util.Arrays;
-import java.util.List;
-import java.util.Timer;
-
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.HelixConstants.ChangeType;
-import org.apache.helix.HelixTimerTask;
-import org.apache.helix.InstanceType;
-import org.apache.helix.PropertyPathConfig;
-import org.apache.helix.PropertyType;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.HealthStatsAggregator;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.monitoring.ZKPathDataDumpTask;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.Watcher.Event.EventType;
-
-public class ControllerManager extends AbstractManager {
-  private static Logger LOG = Logger.getLogger(ControllerManager.class);
-
-  final GenericHelixController _controller = new GenericHelixController();
-
-  // TODO merge into GenericHelixController
-  private CallbackHandler _leaderElectionHandler = null;
-
-  /**
-   * status dump timer-task
-   */
-  static class StatusDumpTask extends HelixTimerTask {
-    Timer _timer = null;
-    final ZkClient zkclient;
-    final AbstractManager helixController;
-
-    public StatusDumpTask(ZkClient zkclient, AbstractManager helixController) {
-      this.zkclient = zkclient;
-      this.helixController = helixController;
-    }
-
-    @Override
-    public void start() {
-      long initialDelay = 30 * 60 * 1000;
-      long period = 120 * 60 * 1000;
-      int timeThresholdNoChange = 180 * 60 * 1000;
-
-      if (_timer == null) {
-        LOG.info("Start StatusDumpTask");
-        _timer = new Timer("StatusDumpTimerTask", true);
-        _timer.scheduleAtFixedRate(new ZKPathDataDumpTask(helixController, zkclient,
-            timeThresholdNoChange), initialDelay, period);
-      }
-
-    }
-
-    @Override
-    public void stop() {
-      if (_timer != null) {
-        LOG.info("Stop StatusDumpTask");
-        _timer.cancel();
-        _timer = null;
-      }
-    }
-  }
-
-  public ControllerManager(String zkAddress, String clusterName, String instanceName) {
-    super(zkAddress, clusterName, instanceName, InstanceType.CONTROLLER);
-
-    _timerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
-    _timerTasks.add(new StatusDumpTask(_zkclient, this));
-  }
-
-  @Override
-  protected List<HelixTimerTask> getControllerHelixTimerTasks() {
-    return _timerTasks;
-  }
-
-  @Override
-  public void handleNewSession() throws Exception {
-    waitUntilConnected();
-
-    /**
-     * reset all handlers, make sure cleanup completed for previous session
-     * disconnect if fail to cleanup
-     */
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.reset();
-    }
-    // TODO reset user defined handlers only
-    resetHandlers();
-
-    /**
-     * from here on, we are dealing with new session
-     */
-
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.init();
-    } else {
-      _leaderElectionHandler =
-          new CallbackHandler(this, _zkclient, _keyBuilder.controller(),
-              new DistributedLeaderElection(this, _controller), new EventType[] {
-                  EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-              }, ChangeType.CONTROLLER);
-    }
-
-    /**
-     * init handlers
-     * ok to init message handler and controller handlers twice
-     * the second init will be skipped (see CallbackHandler)
-     */
-    initHandlers(_handlers);
-  }
-
-  @Override
-  void doDisconnect() {
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.reset();
-    }
-  }
-
-  @Override
-  public boolean isLeader() {
-    if (!isConnected()) {
-      return false;
-    }
-
-    try {
-      LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader());
-      if (leader != null) {
-        String leaderName = leader.getInstanceName();
-        String sessionId = leader.getTypedSessionId().stringify();
-        if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null
-            && sessionId.equals(_sessionId)) {
-          return true;
-        }
-      }
-    } catch (Exception e) {
-      // log
-    }
-    return false;
-  }
-
-  /**
-   * helix-controller uses a write-through cache for external-view
-   */
-  @Override
-  BaseDataAccessor<ZNRecord> createBaseDataAccessor(ZkBaseDataAccessor<ZNRecord> baseDataAccessor) {
-    String extViewPath = PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW, _clusterName);
-    return new ZkCacheBaseDataAccessor<ZNRecord>(baseDataAccessor, Arrays.asList(extViewPath));
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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 df90f6e..d2b520b 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
@@ -22,6 +22,7 @@ package org.apache.helix.manager.zk;
 import java.util.List;
 
 import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.apache.helix.HelixManager;
 import org.apache.helix.HelixTimerTask;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.controller.GenericHelixController;
@@ -35,14 +36,14 @@ import org.apache.log4j.Logger;
 public class ControllerManagerHelper {
   private static Logger LOG = Logger.getLogger(ControllerManagerHelper.class);
 
-  final AbstractManager _manager;
+  final HelixManager _manager;
   final DefaultMessagingService _messagingService;
   final List<HelixTimerTask> _controllerTimerTasks;
 
-  public ControllerManagerHelper(AbstractManager manager) {
+  public ControllerManagerHelper(HelixManager manager, List<HelixTimerTask> controllerTimerTasks) {
     _manager = manager;
     _messagingService = (DefaultMessagingService) manager.getMessagingService();
-    _controllerTimerTasks = manager.getControllerHelixTimerTasks();
+    _controllerTimerTasks = controllerTimerTasks;
   }
 
   public void addListenersToController(GenericHelixController controller) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
index a7d1f25..4fe9164 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
@@ -321,6 +321,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
             _manager.getMessagingService().send(recipientCriteria, messageTemplate, callback,
                 timeOut);
       }
+
       HelixDataAccessor accessor = _manager.getHelixDataAccessor();
       Builder keyBuilder = accessor.keyBuilder();
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
deleted file mode 100644
index f169317..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
+++ /dev/null
@@ -1,190 +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 java.util.ArrayList;
-import java.util.List;
-
-import org.apache.helix.HelixConstants.ChangeType;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixTimerTask;
-import org.apache.helix.InstanceType;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.controller.GenericHelixController;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.HealthStatsAggregator;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.healthcheck.ParticipantHealthReportTask;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.participant.HelixStateMachineEngine;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.Watcher.Event.EventType;
-
-public class DistributedControllerManager extends AbstractManager {
-  private static Logger LOG = Logger.getLogger(DistributedControllerManager.class);
-
-  final StateMachineEngine _stateMachineEngine;
-  final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector;
-
-  CallbackHandler _leaderElectionHandler = null;
-  final GenericHelixController _controller = new GenericHelixController();
-
-  /**
-   * hold timer tasks for controller only
-   * we need to add/remove controller timer tasks during handle new session
-   */
-  final List<HelixTimerTask> _controllerTimerTasks = new ArrayList<HelixTimerTask>();
-
-  public DistributedControllerManager(String zkAddress, String clusterName, String instanceName) {
-    super(zkAddress, clusterName, instanceName, InstanceType.CONTROLLER_PARTICIPANT);
-
-    _stateMachineEngine = new HelixStateMachineEngine(this);
-    _participantHealthInfoCollector = new ParticipantHealthReportCollectorImpl(this, _instanceName);
-
-    _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
-
-    _controllerTimerTasks.add(new HealthStatsAggregationTask(new HealthStatsAggregator(this)));
-    _controllerTimerTasks.add(new ControllerManager.StatusDumpTask(_zkclient, this));
-
-  }
-
-  @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    checkConnected();
-    return _participantHealthInfoCollector;
-  }
-
-  @Override
-  public StateMachineEngine getStateMachineEngine() {
-    return _stateMachineEngine;
-  }
-
-  @Override
-  protected List<HelixTimerTask> getControllerHelixTimerTasks() {
-    return _controllerTimerTasks;
-  }
-
-  @Override
-  public void handleNewSession() throws Exception {
-    waitUntilConnected();
-
-    ParticipantManagerHelper participantHelper =
-        new ParticipantManagerHelper(this, _zkclient, _sessionTimeout);
-
-    /**
-     * 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);
-    }
-
-    /**
-     * auto-join
-     */
-    participantHelper.joinCluster();
-
-    /**
-     * Invoke PreConnectCallbacks
-     */
-    for (PreConnectCallback callback : _preConnectCallbacks) {
-      callback.onPreConnect();
-    }
-
-    participantHelper.createLiveInstance();
-
-    participantHelper.carryOverPreviousCurrentState();
-
-    participantHelper.setupMsgHandler();
-
-    /**
-     * leader election
-     */
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.init();
-    } else {
-      _leaderElectionHandler =
-          new CallbackHandler(this, _zkclient, _keyBuilder.controller(),
-              new DistributedLeaderElection(this, _controller), new EventType[] {
-                  EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
-              }, ChangeType.CONTROLLER);
-    }
-
-    /**
-     * start health-check timer task
-     */
-    participantHelper.createHealthCheckPath();
-    startTimerTasks();
-
-    /**
-     * init handlers
-     * ok to init message handler, data-accessor, and controller handlers twice
-     * the second init will be skipped (see CallbackHandler)
-     */
-    initHandlers(_handlers);
-
-  }
-
-  @Override
-  void doDisconnect() {
-    if (_leaderElectionHandler != null) {
-      _leaderElectionHandler.reset();
-    }
-  }
-
-  @Override
-  public boolean isLeader() {
-    if (!isConnected()) {
-      return false;
-    }
-
-    try {
-      LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader());
-      if (leader != null) {
-        String leaderName = leader.getInstanceName();
-        String sessionId = leader.getTypedSessionId().stringify();
-        if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null
-            && sessionId.equals(_sessionId)) {
-          return true;
-        }
-      }
-    } catch (Exception e) {
-      // log
-    }
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
index caf4dae..9836020 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
@@ -20,10 +20,12 @@ package org.apache.helix.manager.zk;
  */
 
 import java.lang.management.ManagementFactory;
+import java.util.List;
 
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixTimerTask;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
@@ -40,12 +42,15 @@ import org.apache.log4j.Logger;
 public class DistributedLeaderElection implements ControllerChangeListener {
   private static Logger LOG = Logger.getLogger(DistributedLeaderElection.class);
 
-  final AbstractManager _manager;
+  final HelixManager _manager;
   final GenericHelixController _controller;
+  final List<HelixTimerTask> _controllerTimerTasks;
 
-  public DistributedLeaderElection(AbstractManager manager, GenericHelixController controller) {
+  public DistributedLeaderElection(HelixManager manager, GenericHelixController controller,
+      List<HelixTimerTask> controllerTimerTasks) {
     _manager = manager;
     _controller = controller;
+    _controllerTimerTasks = controllerTimerTasks;
   }
 
   /**
@@ -68,7 +73,8 @@ public class DistributedLeaderElection implements ControllerChangeListener {
       return;
     }
 
-    ControllerManagerHelper controllerHelper = new ControllerManagerHelper(_manager);
+    ControllerManagerHelper controllerHelper =
+        new ControllerManagerHelper(_manager, _controllerTimerTasks);
     try {
       if (changeContext.getType().equals(NotificationContext.Type.INIT)
           || changeContext.getType().equals(NotificationContext.Type.CALLBACK)) {
@@ -84,7 +90,7 @@ public class DistributedLeaderElection implements ControllerChangeListener {
                 + _manager.getClusterName());
 
             updateHistory(manager);
-            _manager._baseDataAccessor.reset();
+            _manager.getHelixDataAccessor().getBaseDataAccessor().reset();
             controllerHelper.addListenersToController(_controller);
             controllerHelper.startControllerTimerTasks();
           }
@@ -98,7 +104,7 @@ public class DistributedLeaderElection implements ControllerChangeListener {
         /**
          * clear write-through cache
          */
-        _manager._baseDataAccessor.reset();
+        _manager.getHelixDataAccessor().getBaseDataAccessor().reset();
       }
 
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/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
index b58e4b2..869563a 100644
--- 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
@@ -293,4 +293,10 @@ public class HelixConnectionAdaptor implements HelixManager {
     }
   }
 
+  @Override
+  public void addControllerMessageListener(MessageListener listener) {
+    // TODO Auto-generated method stub
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
deleted file mode 100644
index ab618fe..0000000
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
+++ /dev/null
@@ -1,153 +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 java.util.Arrays;
-
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.InstanceType;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.PropertyPathConfig;
-import org.apache.helix.PropertyType;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
-import org.apache.helix.healthcheck.ParticipantHealthReportTask;
-import org.apache.helix.participant.HelixStateMachineEngine;
-import org.apache.helix.participant.StateMachineEngine;
-import org.apache.log4j.Logger;
-
-public class ParticipantManager extends AbstractManager {
-
-  private static Logger LOG = Logger.getLogger(ParticipantManager.class);
-
-  /**
-   * state-transition message handler factory for helix-participant
-   */
-  final StateMachineEngine _stateMachineEngine;
-
-  final ParticipantHealthReportCollectorImpl _participantHealthInfoCollector;
-
-  public ParticipantManager(String zkAddress, String clusterName, String instanceName) {
-    super(zkAddress, clusterName, instanceName, InstanceType.PARTICIPANT);
-
-    _stateMachineEngine = new HelixStateMachineEngine(this);
-    _participantHealthInfoCollector = new ParticipantHealthReportCollectorImpl(this, _instanceName);
-
-    _timerTasks.add(new ParticipantHealthReportTask(_participantHealthInfoCollector));
-  }
-
-  @Override
-  public ParticipantHealthReportCollector getHealthReportCollector() {
-    checkConnected();
-    return _participantHealthInfoCollector;
-  }
-
-  @Override
-  public StateMachineEngine getStateMachineEngine() {
-    return _stateMachineEngine;
-  }
-
-  @Override
-  public void handleNewSession() {
-    waitUntilConnected();
-
-    /**
-     * stop timer tasks, reset all handlers, make sure cleanup completed for previous session
-     * disconnect if cleanup fails
-     */
-    stopTimerTasks();
-    resetHandlers();
-
-    /**
-     * clear 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);
-    }
-
-    /**
-     * auto-join
-     */
-    ParticipantManagerHelper participantHelper =
-        new ParticipantManagerHelper(this, _zkclient, _sessionTimeout);
-    participantHelper.joinCluster();
-
-    /**
-     * Invoke PreConnectCallbacks
-     */
-    for (PreConnectCallback callback : _preConnectCallbacks) {
-      callback.onPreConnect();
-    }
-
-    participantHelper.createLiveInstance();
-
-    participantHelper.carryOverPreviousCurrentState();
-
-    /**
-     * setup message listener
-     */
-    participantHelper.setupMsgHandler();
-
-    /**
-     * start health check timer task
-     */
-    participantHelper.createHealthCheckPath();
-    startTimerTasks();
-
-    /**
-     * init handlers
-     * ok to init message handler and data-accessor twice
-     * the second init will be skipped (see CallbackHandler)
-     */
-    initHandlers(_handlers);
-
-  }
-
-  /**
-   * helix-participant uses a write-through cache for current-state
-   */
-  @Override
-  BaseDataAccessor<ZNRecord> createBaseDataAccessor(ZkBaseDataAccessor<ZNRecord> baseDataAccessor) {
-    String curStatePath =
-        PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, _clusterName, _instanceName);
-    return new ZkCacheBaseDataAccessor<ZNRecord>(baseDataAccessor, Arrays.asList(curStatePath));
-
-  }
-
-  @Override
-  public boolean isLeader() {
-    return false;
-  }
-
-  /**
-   * disconnect logic for helix-participant
-   */
-  @Override
-  void doDisconnect() {
-    // nothing for participant
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/f8e3b1af/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
index aa84c4d..da266f9 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
@@ -30,6 +30,7 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
+import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.messaging.DefaultMessagingService;
@@ -53,7 +54,7 @@ public class ParticipantManagerHelper {
   private static Logger LOG = Logger.getLogger(ParticipantManagerHelper.class);
 
   final ZkClient _zkclient;
-  final AbstractManager _manager;
+  final HelixManager _manager;
   final PropertyKey.Builder _keyBuilder;
   final String _clusterName;
   final String _instanceName;
@@ -65,8 +66,10 @@ public class ParticipantManagerHelper {
   final ZKHelixDataAccessor _dataAccessor;
   final DefaultMessagingService _messagingService;
   final StateMachineEngine _stateMachineEngine;
+  final LiveInstanceInfoProvider _liveInstanceInfoProvider;
 
-  public ParticipantManagerHelper(AbstractManager manager, ZkClient zkclient, int sessionTimeout) {
+  public ParticipantManagerHelper(HelixManager manager, ZkClient zkclient, int sessionTimeout,
+      LiveInstanceInfoProvider liveInstanceInfoProvider) {
     _zkclient = zkclient;
     _manager = manager;
     _clusterName = manager.getClusterName();
@@ -80,6 +83,7 @@ public class ParticipantManagerHelper {
     _dataAccessor = (ZKHelixDataAccessor) manager.getHelixDataAccessor();
     _messagingService = (DefaultMessagingService) manager.getMessagingService();
     _stateMachineEngine = manager.getStateMachineEngine();
+    _liveInstanceInfoProvider = liveInstanceInfoProvider;
   }
 
   public void joinCluster() {
@@ -90,8 +94,8 @@ public class ParticipantManagerHelper {
           new HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(
               _manager.getClusterName()).build();
       autoJoin =
-          Boolean
-              .parseBoolean(_configAccessor.get(scope, HelixManager.ALLOW_PARTICIPANT_AUTO_JOIN));
+          Boolean.parseBoolean(_configAccessor.get(scope,
+              ZKHelixManager.ALLOW_PARTICIPANT_AUTO_JOIN));
       LOG.info("instance: " + _instanceName + " auto-joining " + _clusterName + " is " + autoJoin);
     } catch (Exception e) {
       // autoJoin is false
@@ -126,6 +130,19 @@ public class ParticipantManagerHelper {
     liveInstance.setHelixVersion(_manager.getVersion());
     liveInstance.setLiveInstance(ManagementFactory.getRuntimeMXBean().getName());
 
+    // LiveInstanceInfoProvider liveInstanceInfoProvider = _manager._liveInstanceInfoProvider;
+    if (_liveInstanceInfoProvider != null) {
+      LOG.info("invoke liveInstanceInfoProvider");
+      ZNRecord additionalLiveInstanceInfo =
+          _liveInstanceInfoProvider.getAdditionalLiveInstanceInfo();
+      if (additionalLiveInstanceInfo != null) {
+        additionalLiveInstanceInfo.merge(liveInstance.getRecord());
+        ZNRecord mergedLiveInstance = new ZNRecord(additionalLiveInstanceInfo, _instanceName);
+        liveInstance = new LiveInstance(mergedLiveInstance);
+        LOG.info("instanceName: " + _instanceName + ", mergedLiveInstance: " + liveInstance);
+      }
+    }
+
     boolean retry;
     do {
       retry = false;
@@ -248,7 +265,7 @@ public class ParticipantManagerHelper {
     }
   }
 
-  public void setupMsgHandler() {
+  public void setupMsgHandler() throws Exception {
     _messagingService.registerMessageHandlerFactory(MessageType.STATE_TRANSITION.toString(),
         _stateMachineEngine);
     _manager.addMessageListener(_messagingService.getExecutor(), _instanceName);


[10/10] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-helix

Posted by zz...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-helix


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

Branch: refs/heads/master
Commit: ebd37fed4b6d1631525e96c6d7301096f4505c58
Parents: f8e3b1a ea8c61d
Author: zzhang <zz...@apache.org>
Authored: Mon Nov 11 13:09:35 2013 -0800
Committer: zzhang <zz...@apache.org>
Committed: Mon Nov 11 13:09:35 2013 -0800

----------------------------------------------------------------------
 helix-admin-webapp/pom.xml |  5 -----
 pom.xml                    | 17 +++++++++++++----
 2 files changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------