You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2017/06/22 22:57:40 UTC

[21/50] [abbrv] helix git commit: Add deprecated clusterStateVerifiers classes back to their original packages for back-compatiblilty, marked them all as deprecated.

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
new file mode 100644
index 0000000..447c5ed
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
@@ -0,0 +1,331 @@
+package org.apache.helix.tools.ClusterVerifiers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.PropertyKey;
+import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.task.TaskConstants;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Verifier that verifies whether the ExternalViews of given resources (or all resources in the cluster)
+ * match exactly as its ideal mapping (in idealstate).
+ */
+public class StrictMatchExternalViewVerifier extends ZkHelixClusterVerifier {
+  private static Logger LOG = Logger.getLogger(StrictMatchExternalViewVerifier.class);
+
+  private final Set<String> _resources;
+  private final Set<String> _expectLiveInstances;
+
+  public StrictMatchExternalViewVerifier(String zkAddr, String clusterName, Set<String> resources,
+      Set<String> expectLiveInstances) {
+    super(zkAddr, clusterName);
+    _resources = resources;
+    _expectLiveInstances = expectLiveInstances;
+  }
+
+  public StrictMatchExternalViewVerifier(ZkClient zkClient, String clusterName,
+      Set<String> resources, Set<String> expectLiveInstances) {
+    super(zkClient, clusterName);
+    _resources = resources;
+    _expectLiveInstances = expectLiveInstances;
+  }
+
+  public static class Builder {
+    private String _clusterName;
+    private Set<String> _resources;
+    private Set<String> _expectLiveInstances;
+    private String _zkAddr;
+    private ZkClient _zkClient;
+
+    public StrictMatchExternalViewVerifier build() {
+      if (_clusterName == null || (_zkAddr == null && _zkClient == null)) {
+        throw new IllegalArgumentException("Cluster name or zookeeper info is missing!");
+      }
+
+      if (_zkClient != null) {
+        return new StrictMatchExternalViewVerifier(_zkClient, _clusterName, _resources,
+            _expectLiveInstances);
+      }
+      return new StrictMatchExternalViewVerifier(_zkAddr, _clusterName, _resources,
+          _expectLiveInstances);
+    }
+
+    public Builder(String clusterName) {
+      _clusterName = clusterName;
+    }
+
+    public String getClusterName() {
+      return _clusterName;
+    }
+
+    public Set<String> getResources() {
+      return _resources;
+    }
+
+    public Builder setResources(Set<String> resources) {
+      _resources = resources;
+      return this;
+    }
+
+    public Set<String> getExpectLiveInstances() {
+      return _expectLiveInstances;
+    }
+
+    public Builder setExpectLiveInstances(Set<String> expectLiveInstances) {
+      _expectLiveInstances = expectLiveInstances;
+      return this;
+    }
+
+    public String getZkAddr() {
+      return _zkAddr;
+    }
+
+    public Builder setZkAddr(String zkAddr) {
+      _zkAddr = zkAddr;
+      return this;
+    }
+
+    public ZkClient getZkClient() {
+      return _zkClient;
+    }
+
+    public Builder setZkClient(ZkClient zkClient) {
+      _zkClient = zkClient;
+      return this;
+    }
+  }
+
+  @Override
+  public boolean verify(long timeout) {
+    return verifyByZkCallback(timeout);
+  }
+
+  @Override
+  public boolean verifyByZkCallback(long timeout) {
+    List<ClusterVerifyTrigger> triggers = new ArrayList<ClusterVerifyTrigger>();
+
+    // setup triggers
+    if (_resources != null && !_resources.isEmpty()) {
+      for (String resource : _resources) {
+        triggers
+            .add(new ClusterVerifyTrigger(_keyBuilder.idealStates(resource), true, false, false));
+        triggers
+            .add(new ClusterVerifyTrigger(_keyBuilder.externalView(resource), true, false, false));
+      }
+
+    } else {
+      triggers.add(new ClusterVerifyTrigger(_keyBuilder.idealStates(), false, true, true));
+      triggers.add(new ClusterVerifyTrigger(_keyBuilder.externalViews(), false, true, true));
+    }
+
+    return verifyByCallback(timeout, triggers);
+  }
+
+  @Override
+  protected boolean verifyState() {
+    try {
+      PropertyKey.Builder keyBuilder = _accessor.keyBuilder();
+      // read cluster once and do verification
+      ClusterDataCache cache = new ClusterDataCache();
+      cache.refresh(_accessor);
+
+      Map<String, IdealState> idealStates = cache.getIdealStates();
+      if (idealStates == null) {
+        // ideal state is null because ideal state is dropped
+        idealStates = Collections.emptyMap();
+      }
+
+      // filter out all resources that use Task state model
+      Iterator<Map.Entry<String, IdealState>> it = idealStates.entrySet().iterator();
+      while (it.hasNext()) {
+        Map.Entry<String, IdealState> pair = it.next();
+        if (pair.getValue().getStateModelDefRef().equals(TaskConstants.STATE_MODEL_NAME)) {
+          it.remove();
+        }
+      }
+
+      // verify live instances.
+      if (_expectLiveInstances != null && !_expectLiveInstances.isEmpty()) {
+        Set<String> actualLiveNodes = cache.getLiveInstances().keySet();
+        if (!_expectLiveInstances.equals(actualLiveNodes)) {
+          return false;
+        }
+      }
+
+      Map<String, ExternalView> extViews = _accessor.getChildValuesMap(keyBuilder.externalViews());
+      if (extViews == null) {
+        extViews = Collections.emptyMap();
+      }
+
+      // Filter resources if requested
+      if (_resources != null && !_resources.isEmpty()) {
+        idealStates.keySet().retainAll(_resources);
+        extViews.keySet().retainAll(_resources);
+      }
+
+      // if externalView is not empty and idealState doesn't exist
+      // add empty idealState for the resource
+      for (String resource : extViews.keySet()) {
+        if (!idealStates.containsKey(resource)) {
+          idealStates.put(resource, new IdealState(resource));
+        }
+      }
+
+      for (String resourceName : idealStates.keySet()) {
+        ExternalView extView = extViews.get(resourceName);
+        IdealState idealState = idealStates.get(resourceName);
+        if (extView == null) {
+          if (idealState.isExternalViewDisabled()) {
+            continue;
+          } else {
+            LOG.debug("externalView for " + resourceName + " is not available");
+            return false;
+          }
+        }
+
+        boolean result = verifyExternalView(cache, extView, idealState);
+        if (!result) {
+          return false;
+        }
+      }
+      return true;
+    } catch (Exception e) {
+      LOG.error("exception in verification", e);
+      return false;
+    }
+  }
+
+  private boolean verifyExternalView(ClusterDataCache dataCache, ExternalView externalView,
+      IdealState idealState) {
+    Map<String, Map<String, String>> mappingInExtview = externalView.getRecord().getMapFields();
+    Map<String, Map<String, String>> idealPartitionState;
+
+    switch (idealState.getRebalanceMode()) {
+    case FULL_AUTO:
+    case SEMI_AUTO:
+    case USER_DEFINED:
+      idealPartitionState = computeIdealPartitionState(dataCache, idealState);
+      break;
+    case CUSTOMIZED:
+      idealPartitionState = idealState.getRecord().getMapFields();
+      break;
+    case TASK:
+      // ignore jobs
+    default:
+      return true;
+    }
+
+    return mappingInExtview.equals(idealPartitionState);
+  }
+
+  private Map<String, Map<String, String>> computeIdealPartitionState(ClusterDataCache cache,
+      IdealState idealState) {
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+    Map<String, Map<String, String>> idealPartitionState =
+        new HashMap<String, Map<String, String>>();
+
+    Set<String> liveEnabledInstances = new HashSet<String>(cache.getLiveInstances().keySet());
+    liveEnabledInstances.removeAll(cache.getDisabledInstances());
+
+    for (String partition : idealState.getPartitionSet()) {
+      List<String> preferenceList = ConstraintBasedAssignment
+          .getPreferenceList(new Partition(partition), idealState, liveEnabledInstances);
+      Map<String, String> idealMapping =
+          computeIdealMapping(preferenceList, stateModelDef, liveEnabledInstances);
+      idealPartitionState.put(partition, idealMapping);
+    }
+
+    return idealPartitionState;
+  }
+
+  /**
+   * compute the ideal mapping for resource in SEMI-AUTO based on its preference list
+   */
+  private Map<String, String> computeIdealMapping(List<String> instancePreferenceList,
+      StateModelDefinition stateModelDef, Set<String> liveEnabledInstances) {
+    Map<String, String> instanceStateMap = new HashMap<String, String>();
+
+    if (instancePreferenceList == null) {
+      return instanceStateMap;
+    }
+
+    List<String> statesPriorityList = stateModelDef.getStatesPriorityList();
+    boolean assigned[] = new boolean[instancePreferenceList.size()];
+
+    for (String state : statesPriorityList) {
+      String num = stateModelDef.getNumInstancesPerState(state);
+      int stateCount = -1;
+      if ("N".equals(num)) {
+        stateCount = liveEnabledInstances.size();
+      } else if ("R".equals(num)) {
+        stateCount = instancePreferenceList.size();
+      } else {
+        try {
+          stateCount = Integer.parseInt(num);
+        } catch (Exception e) {
+          LOG.error("Invalid count for state:" + state + " ,count=" + num);
+        }
+      }
+      if (stateCount > 0) {
+        int count = 0;
+        for (int i = 0; i < instancePreferenceList.size(); i++) {
+          String instanceName = instancePreferenceList.get(i);
+
+          if (!assigned[i]) {
+            instanceStateMap.put(instanceName, state);
+            count = count + 1;
+            assigned[i] = true;
+            if (count == stateCount) {
+              break;
+            }
+          }
+        }
+      }
+    }
+
+    return instanceStateMap;
+  }
+
+  @Override
+  public String toString() {
+    String verifierName = getClass().getSimpleName();
+    return verifierName + "(" + _clusterName + "@" + _zkClient.getServers() + "@resources["
+        + _resources != null ? Arrays.toString(_resources.toArray()) : "" + "])";
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/ZkHelixClusterVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/ZkHelixClusterVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/ZkHelixClusterVerifier.java
new file mode 100644
index 0000000..ad5cda2
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/ZkHelixClusterVerifier.java
@@ -0,0 +1,269 @@
+package org.apache.helix.tools.ClusterVerifiers;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.I0Itec.zkclient.IZkChildListener;
+import org.I0Itec.zkclient.IZkDataListener;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.manager.zk.ZkBaseDataAccessor;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.util.ZKClientPool;
+import org.apache.log4j.Logger;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public abstract class ZkHelixClusterVerifier
+    implements IZkChildListener, IZkDataListener, HelixClusterVerifier {
+  private static Logger LOG = Logger.getLogger(ZkHelixClusterVerifier.class);
+  protected static int DEFAULT_TIMEOUT = 30 * 1000;
+  protected static int DEFAULT_PERIOD = 1000;
+
+
+  protected final ZkClient _zkClient;
+  protected final String _clusterName;
+  protected final HelixDataAccessor _accessor;
+  protected final PropertyKey.Builder _keyBuilder;
+  private CountDownLatch _countdown;
+
+  protected static class ClusterVerifyTrigger {
+    final PropertyKey _triggerKey;
+    final boolean _triggerOnDataChange;
+    final boolean _triggerOnChildChange;
+    final boolean _triggerOnChildDataChange;
+
+    public ClusterVerifyTrigger(PropertyKey triggerKey, boolean triggerOnDataChange,
+        boolean triggerOnChildChange, boolean triggerOnChildDataChange) {
+      _triggerKey = triggerKey;
+      _triggerOnDataChange = triggerOnDataChange;
+      _triggerOnChildChange = triggerOnChildChange;
+      _triggerOnChildDataChange = triggerOnChildDataChange;
+    }
+
+    public boolean isTriggerOnDataChange() {
+      return _triggerOnDataChange;
+    }
+
+    public PropertyKey getTriggerKey() {
+      return _triggerKey;
+    }
+
+    public boolean isTriggerOnChildChange() {
+      return _triggerOnChildChange;
+    }
+
+    public boolean isTriggerOnChildDataChange() {
+      return _triggerOnChildDataChange;
+    }
+  }
+
+  public ZkHelixClusterVerifier(ZkClient zkClient, String clusterName) {
+    if (zkClient == null || clusterName == null) {
+      throw new IllegalArgumentException("requires zkClient|clusterName");
+    }
+    _zkClient = zkClient;
+    _clusterName = clusterName;
+    _accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+    _keyBuilder = _accessor.keyBuilder();
+  }
+
+  public ZkHelixClusterVerifier(String zkAddr, String clusterName) {
+    if (zkAddr == null || clusterName == null) {
+      throw new IllegalArgumentException("requires zkAddr|clusterName");
+    }
+    _zkClient = ZKClientPool.getZkClient(zkAddr);
+    _clusterName = clusterName;
+    _accessor = new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+    _keyBuilder = _accessor.keyBuilder();
+  }
+
+  /**
+   *  Verify the cluster.
+   *  The method will be blocked at most {@code timeout}.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @param timeout in milliseconds
+   * @return true if succeed, false if not.
+   */
+  public boolean verify(long timeout) {
+    return verifyByZkCallback(timeout);
+  }
+
+  /**
+   *  Verify the cluster.
+   *  The method will be blocked at most 30 seconds.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @return true if succeed, false if not.
+   */
+  public boolean verify() {
+    return verify(DEFAULT_TIMEOUT);
+  }
+
+  /**
+   *  Verify the cluster by relying on zookeeper callback and verify.
+   *  The method will be blocked at most {@code timeout}.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @param timeout in milliseconds
+   * @return true if succeed, false if not.
+   */
+  public abstract boolean verifyByZkCallback(long timeout);
+
+  /**
+   *  Verify the cluster by relying on zookeeper callback and verify.
+   *  The method will be blocked at most 30 seconds.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @return true if succeed, false if not.
+   */
+  public boolean verifyByZkCallback() {
+    return verifyByZkCallback(DEFAULT_TIMEOUT);
+  }
+
+  /**
+   *  Verify the cluster by periodically polling the cluster status and verify.
+   *  The method will be blocked at most {@code timeout}.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @param timeout
+   * @param period polling interval
+   * @return
+   */
+  public boolean verifyByPolling(long timeout, long period) {
+    try {
+      long start = System.currentTimeMillis();
+      boolean success;
+      do {
+        success = verifyState();
+        if (success) {
+          return true;
+        }
+        TimeUnit.MILLISECONDS.sleep(period);
+      } while ((System.currentTimeMillis() - start) <= timeout);
+    } catch (Exception e) {
+      LOG.error("Exception in verifier", e);
+    }
+    return false;
+  }
+
+  /**
+   *  Verify the cluster by periodically polling the cluster status and verify.
+   *  The method will be blocked at most 30 seconds.
+   *  Return true if the verify succeed, otherwise return false.
+   *
+   * @return true if succeed, false if not.
+   */
+  public boolean verifyByPolling() {
+    return verifyByPolling(DEFAULT_TIMEOUT, DEFAULT_PERIOD);
+  }
+
+  protected boolean verifyByCallback(long timeout, List<ClusterVerifyTrigger> triggers) {
+    _countdown = new CountDownLatch(1);
+
+    for (ClusterVerifyTrigger trigger : triggers) {
+      subscribeTrigger(trigger);
+    }
+
+    boolean success = false;
+    try {
+      success = verifyState();
+      if (!success) {
+
+        success = _countdown.await(timeout, TimeUnit.MILLISECONDS);
+        if (!success) {
+          // make a final try if timeout
+          success = verifyState();
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Exception in verifier", e);
+    }
+
+    // clean up
+    _zkClient.unsubscribeAll();
+
+    return success;
+  }
+
+  private void subscribeTrigger(ClusterVerifyTrigger trigger) {
+    String path = trigger.getTriggerKey().getPath();
+    if (trigger.isTriggerOnDataChange()) {
+      _zkClient.subscribeDataChanges(path, this);
+    }
+
+    if (trigger.isTriggerOnChildChange()) {
+      _zkClient.subscribeChildChanges(path, this);
+    }
+
+    if (trigger.isTriggerOnChildDataChange()) {
+      List<String> childs = _zkClient.getChildren(path);
+      for (String child : childs) {
+        String childPath = String.format("%s/%s", path, child);
+        _zkClient.subscribeDataChanges(childPath, this);
+      }
+    }
+  }
+
+  /**
+   * The method actually performs the required verifications.
+   * @return
+   * @throws Exception
+   */
+  protected abstract boolean verifyState() throws Exception;
+
+  @Override
+  public void handleDataChange(String dataPath, Object data) throws Exception {
+    boolean success = verifyState();
+    if (success) {
+      _countdown.countDown();
+    }
+  }
+
+  @Override
+  public void handleDataDeleted(String dataPath) throws Exception {
+    _zkClient.unsubscribeDataChanges(dataPath, this);
+  }
+
+  @Override
+  public void handleChildChange(String parentPath, List<String> currentChilds) throws Exception {
+    for (String child : currentChilds) {
+      String childPath = String.format("%s/%s", parentPath, child);
+      _zkClient.subscribeDataChanges(childPath, this);
+    }
+
+    boolean success = verifyState();
+    if (success) {
+      _countdown.countDown();
+    }
+  }
+
+  public ZkClient getZkClient() {
+    return _zkClient;
+  }
+
+  public String getClusterName() {
+    return _clusterName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java b/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java
index 2cc145c..55fb652 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/IntegrationTestUtil.java
@@ -36,8 +36,8 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterExternalViewVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterLiveNodesVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterLiveNodesVerifier;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
index 9e150bb..39178a8 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
@@ -48,7 +48,7 @@ import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.Attributes;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.apache.helix.tools.StateModelConfigGenerator;
 import org.apache.helix.util.ZKClientPool;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java b/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
index 2f25acb..576c726 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/SinglePartitionLeaderStandByTest.java
@@ -30,7 +30,7 @@ import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 95adc65..9205629 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
@@ -24,7 +24,7 @@ import java.util.Date;
 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.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 2064224..8875d25 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
@@ -30,7 +30,7 @@ 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.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 acd11c7..dfdc0fe 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
@@ -33,8 +33,8 @@ import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 38361c9..0a773e0 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
@@ -30,9 +30,9 @@ import org.apache.helix.ZNRecord;
 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.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.helix.tools.DefaultIdealStateCalculator;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 d2405ca..f9bbc94 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
@@ -39,8 +39,8 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 5f846e0..477c540 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
@@ -36,8 +36,8 @@ import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalanceWithDisabledInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalanceWithDisabledInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalanceWithDisabledInstance.java
index 5576a8f..dfc7b35 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalanceWithDisabledInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalanceWithDisabledInstance.java
@@ -23,7 +23,7 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java b/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
index 952f73b..78f0c72 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBasicSpectator.java
@@ -29,7 +29,7 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.model.ExternalView;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 b526f2d..167625c 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
@@ -38,8 +38,8 @@ import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 52c03f5..dfafa86 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
@@ -34,8 +34,8 @@ import org.apache.helix.messaging.handling.BatchMessageWrapper;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 cd30afe..b8e6b46 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
@@ -36,9 +36,9 @@ import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.MasterNbInExtViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 b64d2c8..1ee44f6 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
@@ -27,9 +27,9 @@ 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.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.MasterNbInExtViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 e9fa623..f042e66 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
@@ -27,7 +27,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.LiveInstance;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestControllerLiveLock.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestControllerLiveLock.java b/helix-core/src/test/java/org/apache/helix/integration/TestControllerLiveLock.java
index f996a40..dc09d85 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestControllerLiveLock.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestControllerLiveLock.java
@@ -35,7 +35,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java b/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
index 95bc5e3..753baf7 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCorrectnessOnConnectivityLoss.java
@@ -36,8 +36,8 @@ import org.apache.helix.participant.statemachine.StateModelInfo;
 import org.apache.helix.participant.statemachine.StateTransitionError;
 import org.apache.helix.participant.statemachine.Transition;
 import org.apache.helix.spectator.RoutingTableProvider;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestCrushAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCrushAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestCrushAutoRebalance.java
index 8313cca..72df3eb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCrushAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCrushAutoRebalance.java
@@ -28,7 +28,7 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 39aa6f1..13a3a6c 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
@@ -37,8 +37,8 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.IdealStateProperty;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalance.java
index 3eba34b..6342d13 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalance.java
@@ -28,8 +28,8 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.BestPossibleExternalViewVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.HelixClusterVerifier;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalanceWithRackaware.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalanceWithRackaware.java b/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalanceWithRackaware.java
index 94a6c27..af11966 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalanceWithRackaware.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDelayedAutoRebalanceWithRackaware.java
@@ -23,7 +23,7 @@ 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.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.testng.annotations.BeforeClass;
 
 import java.util.Date;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 c66fece..374a767 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
@@ -34,8 +34,8 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
index 92478e0..ce22119 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableCustomCodeRunner.java
@@ -42,7 +42,7 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.participant.CustomCodeCallbackHandler;
 import org.apache.helix.participant.HelixCustomCodeRunner;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
index c2efbfc..9afcf4b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableExternalView.java
@@ -30,7 +30,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 8a78fe6..9171276 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
@@ -21,7 +21,7 @@ package org.apache.helix.integration;
 
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 c12ee39..3b849d5 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
@@ -38,7 +38,7 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
index b5d7188..d3a3f7a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisableResource.java
@@ -38,7 +38,7 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 3268d62..9fa1d87 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
@@ -30,8 +30,8 @@ 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.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 dfc944e..ba5bd4e 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
@@ -29,8 +29,8 @@ 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.model.LiveInstance;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 990c608..ce97088 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
@@ -39,7 +39,7 @@ import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.store.PropertyJsonSerializer;
 import org.apache.helix.store.PropertyStoreException;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.apache.helix.tools.TestCommand;
 import org.apache.helix.tools.TestExecutor;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 143463e..ad9c3a4 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
@@ -42,8 +42,8 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.builder.CustomModeISBuilder;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 d581fb9..82486a8 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
@@ -22,7 +22,7 @@ package org.apache.helix.integration;
 import org.apache.helix.TestHelper;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java b/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
index 21fa120..9bc3376 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnableCompression.java
@@ -14,8 +14,8 @@ import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.builder.CustomModeISBuilder;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.helix.util.GZipCompressionUtil;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 b8eec3b..471c629 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
@@ -29,8 +29,8 @@ 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.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java b/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
index ef274fc..e031d45 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEntropyFreeNodeBounce.java
@@ -44,9 +44,9 @@ import org.apache.helix.participant.statemachine.StateModel;
 import org.apache.helix.participant.statemachine.StateModelFactory;
 import org.apache.helix.participant.statemachine.StateModelInfo;
 import org.apache.helix.participant.statemachine.Transition;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 3ac86fc..d1d2d38 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
@@ -30,7 +30,7 @@ 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.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 2c20076..b4823ef 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
@@ -29,9 +29,9 @@ import org.apache.helix.PropertyKey.Builder;
 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.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.MasterNbInExtViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
index b6e7c31..f2376c0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestFullAutoNodeTagging.java
@@ -40,9 +40,9 @@ import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.ZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.ZkVerifier;
 import org.apache.helix.util.ZKClientPool;
 import org.apache.log4j.Logger;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 7b43441..6b56212 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
@@ -36,7 +36,7 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.participant.CustomCodeCallbackHandler;
 import org.apache.helix.participant.HelixCustomCodeRunner;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 4e0fed6..25a0b92 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
@@ -26,7 +26,7 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 91a7075..47f7e96 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
@@ -36,9 +36,9 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.helix.tools.StateModelConfigGenerator;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 
 // Helix-50: integration test for generate message based on state priority

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
index f99627b..18324f1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidResourceRebalance.java
@@ -30,7 +30,7 @@ import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 c79d61a..615eab6 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
@@ -37,9 +37,9 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.MasterNbInExtViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.MasterNbInExtViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 008bd44..8b8ba81 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
@@ -58,8 +58,8 @@ import org.apache.helix.participant.statemachine.StateModel;
 import org.apache.helix.participant.statemachine.StateModelFactory;
 import org.apache.helix.participant.statemachine.StateModelInfo;
 import org.apache.helix.participant.statemachine.Transition;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 684162d..bd415fd 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
@@ -27,8 +27,8 @@ import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.mock.participant.MockBootstrapModelFactory;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 88c8874..0875a02 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
@@ -28,8 +28,8 @@ import org.apache.helix.ZNRecord;
 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.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 6fa77bb..c18c7f2 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
@@ -29,8 +29,8 @@ import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
index 05b9472..50cff5b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestPartitionLevelTransitionConstraint.java
@@ -43,7 +43,7 @@ import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
 import org.apache.helix.participant.statemachine.StateModel;
 import org.apache.helix.participant.statemachine.StateModelFactory;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 638c4f0..56665d8 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
@@ -32,8 +32,8 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java b/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
index 0474a10..3aec847 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRebalancerPersistAssignments.java
@@ -26,9 +26,9 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.MasterSlaveSMD;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.BestPossibleExternalViewVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.HelixClusterVerifier;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
@@ -164,8 +164,8 @@ public class TestRebalancerPersistAssignments extends ZkStandAloneCMTestBase {
     String testDb = "TestDB1-MasterSlave";
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
-    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.SEMI_AUTO.name());
+    _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5, BuiltInStateModelDefinitions.MasterSlave.name(),
+        RebalanceMode.SEMI_AUTO.name());
     _setupTool.rebalanceStorageCluster(CLUSTER_NAME, testDb, 3);
 
     HelixClusterVerifier verifier =

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java b/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
index 2c0d9d7..123a214 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestReelectedPipelineCorrectness.java
@@ -31,8 +31,8 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 677777f..9f6bf58 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
@@ -34,7 +34,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.apache.helix.tools.DefaultIdealStateCalculator;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 c0199b3..11eae05 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
@@ -29,7 +29,7 @@ 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.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 9522de0..f09a12c 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
@@ -36,7 +36,7 @@ import org.apache.helix.mock.participant.ErrTransition;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/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 ef1aac5..93b039c 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
@@ -29,7 +29,7 @@ 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.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/015a73ce/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java b/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
index f67db93..f0eba23 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResourceGroupEndtoEnd.java
@@ -36,7 +36,7 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.OnlineOfflineSMD;
 import org.apache.helix.spectator.RoutingTableProvider;
-import org.apache.helix.tools.ClusterStateVerifier.ClusterStateVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ClusterStateVerifier;
 import org.testng.Assert;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;