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/02/02 00:33:39 UTC

[1/2] git commit: HELIX-38: create a ZkTestManager that exposes additional members such as zkclient for testing

HELIX-38: create a ZkTestManager that exposes additional members such as zkclient for testing


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

Branch: refs/heads/master
Commit: 7d8cf20608423da84f69786a96aafc7690818a79
Parents: a8ee977
Author: zzhang <zz...@uci.edu>
Authored: Fri Feb 1 15:32:52 2013 -0800
Committer: zzhang <zz...@uci.edu>
Committed: Fri Feb 1 15:32:52 2013 -0800

----------------------------------------------------------------------
 .../apache/helix/manager/zk/ZKHelixManager.java    |    7 +-
 .../src/test/java/org/apache/helix/TestHelper.java |   97 +++++-----
 .../test/java/org/apache/helix/ZkTestHelper.java   |  144 +++++++++++++--
 .../TestAddNodeAfterControllerStart.java           |   28 +---
 .../integration/TestSessionExpiryInTransition.java |    8 +-
 .../integration/TestStandAloneCMSessionExpiry.java |   12 +-
 .../TestStartMultipleControllersWithSameName.java  |    3 +-
 .../helix/manager/zk/TestLiveInstanceBounce.java   |    3 +-
 .../manager/zk/TestZkManagerFlappingDetection.java |   14 +-
 .../helix/mock/participant/MockParticipant.java    |   17 +--
 10 files changed, 203 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 eff05a3..1cbdec9 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
@@ -99,7 +99,7 @@ public class ZKHelixManager implements HelixManager
   private ZKHelixDataAccessor                  _helixAccessor;
   private ConfigAccessor                       _configAccessor;
   protected ZkClient                           _zkClient;
-  private final List<CallbackHandler>          _handlers;
+  protected final List<CallbackHandler>         _handlers;
   private final ZkStateChangeListener          _zkStateChangeListener;
   private final InstanceType                   _instanceType;
   volatile String                              _sessionId;
@@ -1069,11 +1069,6 @@ public class ZKHelixManager implements HelixManager
     return _stateMachEngine;
   }
 
-  protected List<CallbackHandler> getHandlers()
-  {
-    return _handlers;
-  }
-
   // TODO: rename this and not expose this function as part of interface
   @Override
   public void startTimerTasks()

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 9670072..8bc12bb 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelper.java
@@ -58,6 +58,8 @@ 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;
@@ -138,12 +140,11 @@ public class TestHelper
                                                 final String instanceName) throws Exception
   {
     StartCMResult result = new StartCMResult();
-    HelixManager manager = null;
-    manager =
-        HelixManagerFactory.getZKHelixManager(clusterName,
-                                              instanceName,
-                                              InstanceType.PARTICIPANT,
-                                              zkAddr);
+    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;
@@ -152,6 +153,39 @@ public class TestHelper
     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,
@@ -159,11 +193,10 @@ public class TestHelper
                                               final String controllerMode) throws Exception
   {
     final StartCMResult result = new StartCMResult();
-    final HelixManager manager =
-        HelixControllerMain.startHelixController(zkConnectString,
-                                                 clusterName,
-                                                 controllerName,
-                                                 controllerMode);
+    final ZkHelixTestManager manager = startHelixController(zkConnectString,
+                                                 	clusterName,
+                                                 	controllerName,
+                                                 	controllerMode);
     result._manager = manager;
 
     Thread thread = new Thread(new Runnable()
@@ -201,7 +234,7 @@ public class TestHelper
   public static class StartCMResult
   {
     public Thread       _thread;
-    public HelixManager _manager;
+    public ZkHelixTestManager _manager;
 
   }
 
@@ -579,46 +612,6 @@ public class TestHelper
     return msg;
   }
 
-  public static int numberOfListeners(String zkAddr, String path) throws Exception
-  {
-    int count = 0;
-    String splits[] = zkAddr.split(":");
-    Socket sock = new Socket(splits[0], Integer.parseInt(splits[1]));
-    PrintWriter out = new PrintWriter(sock.getOutputStream(), true);
-    BufferedReader in = new BufferedReader(new InputStreamReader(sock.getInputStream()));
-
-    out.println("wchp");
-
-    String line = in.readLine();
-    while (line != null)
-    {
-      // System.out.println(line);
-      if (line.equals(path))
-      {
-        // System.out.println("match: " + line);
-
-        String nextLine = in.readLine();
-        if (nextLine == null)
-        {
-          break;
-        }
-        // System.out.println(nextLine);
-        while (nextLine.startsWith("\t0x"))
-        {
-          count++;
-          nextLine = in.readLine();
-          if (nextLine == null)
-          {
-            break;
-          }
-        }
-      }
-      line = in.readLine();
-    }
-    sock.close();
-    return count;
-  }
-
   public static String getTestMethodName()
   {
     StackTraceElement[] calls = Thread.currentThread().getStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
index f9bbc29..1bced7f 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkTestHelper.java
@@ -19,7 +19,14 @@ package org.apache.helix;
  * under the License.
  */
 
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.net.Socket;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 
 import org.I0Itec.zkclient.IZkStateListener;
@@ -48,25 +55,6 @@ public class ZkTestHelper
   {
     // Logger.getRootLogger().setLevel(Level.DEBUG);
   }
-
-  // zkClusterManager that exposes zkclient
-  public static class TestZkHelixManager extends ZKHelixManager
-  {
-
-    public TestZkHelixManager(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 static void disconnectSession(final ZkClient zkClient) throws Exception
   {
@@ -247,4 +235,122 @@ public class ZkTestHelper
     return result;
   }
   
+  /**
+   * return the number of listeners on given zk-path
+   * @param zkAddr
+   * @param path
+   * @return
+   * @throws Exception
+   */
+  public static int numberOfListeners(String zkAddr, String path) throws Exception
+  {
+    int count = 0;
+    String splits[] = zkAddr.split(":");
+    Socket sock = new Socket(splits[0], Integer.parseInt(splits[1]));
+    PrintWriter out = new PrintWriter(sock.getOutputStream(), true);
+    BufferedReader in = new BufferedReader(new InputStreamReader(sock.getInputStream()));
+
+    out.println("wchp");
+
+    String line = in.readLine();
+    while (line != null)
+    {
+      // System.out.println(line);
+      if (line.equals(path))
+      {
+        // System.out.println("match: " + line);
+
+        String nextLine = in.readLine();
+        if (nextLine == null)
+        {
+          break;
+        }
+        // System.out.println(nextLine);
+        while (nextLine.startsWith("\t0x"))
+        {
+          count++;
+          nextLine = in.readLine();
+          if (nextLine == null)
+          {
+            break;
+          }
+        }
+      }
+      line = in.readLine();
+    }
+    sock.close();
+    return count;
+  }
+  
+  /**
+   * return a map from zk-path to a set of zk-session-id that put watches on the zk-path
+   * 
+   * @param zkAddr
+   * @param path
+   * @return
+   * @throws Exception
+   */
+  public static Map<String, Set<String>> getListenersByInstance(String zkAddr) throws Exception
+  {
+    int count = 0;
+    String splits[] = zkAddr.split(":");
+    Socket sock = new Socket(splits[0], Integer.parseInt(splits[1]));
+    PrintWriter out = new PrintWriter(sock.getOutputStream(), true);
+    BufferedReader in = new BufferedReader(new InputStreamReader(sock.getInputStream()));
+
+    out.println("wchp");
+
+    Map<String, Set<String>> listenerMap = new TreeMap<String, Set<String>>();
+    String lastPath = null;
+    String line = in.readLine();
+    while (line != null)
+    {
+    	line = line.trim();
+    	
+    	if (line.startsWith("/")) {
+    		lastPath = line;
+    		if (!listenerMap.containsKey(lastPath)) {
+    			listenerMap.put(lastPath, new TreeSet<String>());
+    		}
+    	} else if (line.startsWith("0x")) {
+    		if (lastPath != null && listenerMap.containsKey(lastPath) ) {
+    			listenerMap.get(lastPath).add(line);
+    		} else
+    		{
+    			LOG.error("Not path associated with listener sessionId: " + line + ", lastPath: " + lastPath);
+    		}
+    	} else
+    	{
+//    		LOG.error("unrecognized line: " + line);
+    	}
+      line = in.readLine();
+    }
+    sock.close();
+    return listenerMap;
+  }
+
+  /**
+   * return a map from session-id to a set of zk-path that the session has watches on
+   * 
+   * @param listenerMap
+   * @return
+   */
+  public static Map<String, Set<String>> getListenersBySession(String zkAddr) throws Exception {
+	  Map<String, Set<String>> listenerMapByInstance = getListenersByInstance(zkAddr);
+	  
+	  // convert to index by sessionId
+	  Map<String, Set<String>> listenerMapBySession = new TreeMap<String, Set<String>>();
+	  for (String path : listenerMapByInstance.keySet()) {
+		  for (String sessionId : listenerMapByInstance.get(path)) {
+			  if (!listenerMapBySession.containsKey(sessionId)) {
+				  listenerMapBySession.put(sessionId, new TreeSet<String>());
+			  }
+			  listenerMapBySession.get(sessionId).add(path);
+		  }
+	  }
+
+	  return listenerMapBySession;
+  }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 1913e57..88cc82c 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
@@ -27,6 +27,8 @@ 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.manager.zk.CallbackHandler;
 import org.apache.helix.manager.zk.ZKHelixManager;
@@ -43,24 +45,6 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase
                                       Logger.getLogger(TestAddNodeAfterControllerStart.class);
   final String          className = getShortClassName();
 
-  class ZkClusterManagerWithGetHandlers extends ZKHelixManager
-  {
-    public ZkClusterManagerWithGetHandlers(String clusterName,
-                                           String instanceName,
-                                           InstanceType instanceType,
-                                           String zkConnectString) throws Exception
-    {
-      super(clusterName, instanceName, instanceType, zkConnectString);
-    }
-
-    @Override
-    public List<CallbackHandler> getHandlers()
-    {
-      return super.getHandlers();
-    }
-
-  }
-
   @Test
   public void testStandalone() throws Exception
   {
@@ -90,8 +74,8 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase
       new Thread(participants[i]).start();
     }
 
-    ZkClusterManagerWithGetHandlers controller =
-        new ZkClusterManagerWithGetHandlers(clusterName,
+    ZkHelixTestManager controller =
+        new ZkHelixTestManager(clusterName,
                                             "controller_0",
                                             InstanceType.CONTROLLER,
                                             ZK_ADDR);
@@ -192,7 +176,7 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase
     
     // check if controller_0 has message listener for localhost_12918
     String msgPath = PropertyPathConfig.getPath(PropertyType.MESSAGES, clusterName, "localhost_12918");
-    int numberOfListeners = TestHelper.numberOfListeners(ZK_ADDR, msgPath);
+    int numberOfListeners = ZkTestHelper.numberOfListeners(ZK_ADDR, msgPath);
     // System.out.println("numberOfListeners(" + msgPath + "): " + numberOfListeners);
     Assert.assertEquals(numberOfListeners, 2);  // 1 of participant, and 1 of controller
 
@@ -210,7 +194,7 @@ public class TestAddNodeAfterControllerStart extends ZkIntegrationTestBase
     Assert.assertTrue(result);
     // check if controller_0 has message listener for localhost_12919
     msgPath = PropertyPathConfig.getPath(PropertyType.MESSAGES, clusterName, "localhost_12919");
-    numberOfListeners = TestHelper.numberOfListeners(ZK_ADDR, msgPath);
+    numberOfListeners = ZkTestHelper.numberOfListeners(ZK_ADDR, msgPath);
     // System.out.println("numberOfListeners(" + msgPath + "): " + numberOfListeners);
     Assert.assertEquals(numberOfListeners, 2);  // 1 of participant, and 1 of controller
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 e405731..e70c3ca 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
@@ -25,8 +25,8 @@ 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.ZkTestHelper.TestZkHelixManager;
 import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;
@@ -48,7 +48,7 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase
     @Override
     public void doTransition(Message message, NotificationContext context)
     {
-      TestZkHelixManager manager = (TestZkHelixManager)context.getManager();
+      ZkHelixTestManager manager = (ZkHelixTestManager)context.getManager();
      
       String instance = message.getTgtName();
       String partition = message.getPartitionName();
@@ -104,8 +104,8 @@ public class TestSessionExpiryInTransition extends ZkIntegrationTestBase
     for (int i = 0; i < 5; i++)
     {
       String instanceName = "localhost_" + (12918 + i);
-      TestZkHelixManager manager =
-          new TestZkHelixManager(clusterName,
+      ZkHelixTestManager manager =
+          new ZkHelixTestManager(clusterName,
                                  instanceName,
                                  InstanceType.PARTICIPANT,
                                  ZK_ADDR);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 a74d7ec..9c64c97 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStandAloneCMSessionExpiry.java
@@ -23,8 +23,8 @@ 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.ZkTestHelper.TestZkHelixManager;
 import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
@@ -64,8 +64,8 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase
     for (int i = 0; i < 5; i++)
     {
       String instanceName = "localhost_" + (12918 + i);
-      TestZkHelixManager manager =
-          new TestZkHelixManager(clusterName,
+      ZkHelixTestManager manager =
+          new ZkHelixTestManager(clusterName,
                                  instanceName,
                                  InstanceType.PARTICIPANT,
                                  ZK_ADDR);
@@ -73,8 +73,8 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase
       participants[i].syncStart();
     }
 
-    TestZkHelixManager controller =
-        new TestZkHelixManager(clusterName,
+    ZkHelixTestManager controller =
+        new ZkHelixTestManager(clusterName,
                                "controller_0",
                                InstanceType.CONTROLLER,
                                ZK_ADDR);
@@ -87,7 +87,7 @@ public class TestStandAloneCMSessionExpiry extends ZkIntegrationTestBase
     Assert.assertTrue(result);
 
     // participant session expiry
-    TestZkHelixManager participantToExpire = (TestZkHelixManager)participants[1].getManager();
+    ZkHelixTestManager participantToExpire = (ZkHelixTestManager)participants[1].getManager();
 
     System.out.println("Expire participant session");
     String oldSessionId = participantToExpire.getSessionId();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 311636d..27e6bdb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStartMultipleControllersWithSameName.java
@@ -24,6 +24,7 @@ import java.util.Date;
 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.model.IdealState.IdealStateModeProperty;
 import org.apache.log4j.Level;
@@ -61,7 +62,7 @@ public class TestStartMultipleControllersWithSameName extends ZkIntegrationTestB
 
 	Thread.sleep(500); // wait leader election finishes
 	String liPath = PropertyPathConfig.getPath(PropertyType.LIVEINSTANCES, clusterName);
-	int listenerNb = TestHelper.numberOfListeners(ZK_ADDR, liPath);
+	int listenerNb = ZkTestHelper.numberOfListeners(ZK_ADDR, liPath);
 	// System.out.println("listenerNb: " + listenerNb);
 	Assert.assertEquals(listenerNb, 1, "Only one controller should succeed in becoming leader");
 	

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 65b5528..a2f0fcd 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
@@ -21,6 +21,7 @@ package org.apache.helix.manager.zk;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
 import org.apache.helix.manager.zk.ZKHelixManager;
@@ -36,7 +37,7 @@ public class TestLiveInstanceBounce extends ZkStandAloneCMTestBaseWithPropertySe
   {
     String controllerName = CONTROLLER_PREFIX + "_0";
     StartCMResult controllerResult = _startCMResultMap.get(controllerName);
-    ZKHelixManager controller = (ZKHelixManager) controllerResult._manager;
+    ZkHelixTestManager controller = (ZkHelixTestManager) controllerResult._manager;
     int handlerSize = controller.getHandlers().size();
 
     for (int i = 0; i < 2; i++)

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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 e7a37f5..d30e63a 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
@@ -23,8 +23,8 @@ 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.ZkTestHelper.TestZkHelixManager;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;
@@ -50,8 +50,8 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase
     
     
       String instanceName = "localhost_" + (12918 + 0);
-      TestZkHelixManager manager =
-          new TestZkHelixManager(clusterName,
+      ZkHelixTestManager manager =
+          new ZkHelixTestManager(clusterName,
                                  instanceName,
                                  InstanceType.PARTICIPANT,
                                  ZK_ADDR);
@@ -117,8 +117,8 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase
       // flapping time window to 5 sec
       System.setProperty("helixmanager.flappingTimeWindow", "10000");
       System.setProperty("helixmanager.maxDisconnectThreshold", "7");
-      TestZkHelixManager manager2 =
-          new TestZkHelixManager(clusterName,
+      ZkHelixTestManager manager2 =
+          new ZkHelixTestManager(clusterName,
                                  instanceName,
                                  type,
                                  ZK_ADDR);
@@ -168,8 +168,8 @@ public class TestZkManagerFlappingDetection extends ZkIntegrationTestBase
       // flapping time window to 5 sec
       System.setProperty("helixmanager.flappingTimeWindow", "5000");
       System.setProperty("helixmanager.maxDisconnectThreshold", "3");
-      TestZkHelixManager manager2 =
-          new TestZkHelixManager(clusterName,
+      ZkHelixTestManager manager2 =
+          new ZkHelixTestManager(clusterName,
                                  null,
                                  InstanceType.CONTROLLER,
                                  ZK_ADDR);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/7d8cf206/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
index 753c254..b3fdeb2 100644
--- 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
@@ -33,6 +33,7 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.ZNRecord;
+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.model.Message;
@@ -57,7 +58,7 @@ public class MockParticipant extends Thread
   private final CountDownLatch    _stopCountDown           = new CountDownLatch(1);
   private final CountDownLatch    _waitStopFinishCountDown = new CountDownLatch(1);
 
-  private final HelixManager      _manager;
+  private final ZkHelixTestManager _manager;
   private final StateModelFactory _msModelFactory;
   private final MockJobIntf       _job;
 
@@ -463,11 +464,7 @@ public class MockParticipant extends Thread
     _instanceName = instanceName;
     _msModelFactory = new MockMSModelFactory(transition);
 
-    _manager =
-        HelixManagerFactory.getZKHelixManager(_clusterName,
-                                              _instanceName,
-                                              InstanceType.PARTICIPANT,
-                                              zkAddr);
+    _manager = new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr);
     _job = job;
   }
 
@@ -481,11 +478,7 @@ public class MockParticipant extends Thread
     _instanceName = instanceName;
     _msModelFactory = factory;
 
-    _manager =
-        HelixManagerFactory.getZKHelixManager(_clusterName,
-                                              _instanceName,
-                                              InstanceType.PARTICIPANT,
-                                              zkAddr);
+    _manager = new ZkHelixTestManager(_clusterName, _instanceName, InstanceType.PARTICIPANT, zkAddr);
     _job = job;
   }
 
@@ -494,7 +487,7 @@ public class MockParticipant extends Thread
     return _msModelFactory;
   }
 
-  public MockParticipant(HelixManager manager, MockTransition transition)
+  public MockParticipant(ZkHelixTestManager manager, MockTransition transition)
   {
     _clusterName = manager.getClusterName();
     _instanceName = manager.getInstanceName();