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 2018/01/25 21:48:52 UTC

[01/50] [abbrv] helix git commit: Batch API Implementation

Repository: helix
Updated Branches:
  refs/heads/master 03e8580ec -> 7ba8c5197


Batch API Implementation

This rb include
1. The batch API define and batch API implementation and make old API backward compatible.
2. The server changes the logic to determine which are the disabled instances for both batch API or old API.


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

Branch: refs/heads/master
Commit: 31cec9114b77ee37f79da137ed914c7b330780f3
Parents: 3f34a8e
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Oct 11 18:18:20 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:10 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/helix/HelixAdmin.java  |  29 ++
 .../controller/GenericHelixController.java      |   1 +
 .../rebalancer/DelayedAutoRebalancer.java       |  17 +-
 .../rebalancer/topology/Topology.java           |   6 +-
 .../controller/stages/ClusterDataCache.java     |   7 +-
 .../controller/stages/ReadClusterDataStage.java |   5 +-
 .../manager/zk/ControllerManagerHelper.java     |   1 +
 .../apache/helix/manager/zk/ZKHelixAdmin.java   | 307 ++++++++++++-------
 .../org/apache/helix/model/ClusterConfig.java   |  18 +-
 .../org/apache/helix/model/InstanceConfig.java  |  18 ++
 .../org/apache/helix/tools/ClusterSetup.java    |   9 +-
 .../integration/TestBatchEnableInstances.java   | 110 +++++++
 .../integration/TestZkCallbackHandlerLeak.java  |  16 +-
 .../manager/TestConsecutiveZkSessionExpiry.java |   2 +-
 .../TestDistributedControllerManager.java       |   2 +-
 .../manager/TestZkCallbackHandlerLeak.java      |  10 +-
 .../org/apache/helix/mock/MockHelixAdmin.java   |  14 +
 .../helix/task/TaskSynchronizedTestBase.java    |   1 +
 .../rest/server/resources/InstanceAccessor.java |   6 +-
 19 files changed, 447 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
index 7438ee9..652ab7a 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
@@ -217,6 +217,15 @@ public interface HelixAdmin {
   void enableInstance(String clusterName, String instanceName, boolean enabled);
 
   /**
+   * Batch enable/disable instances in a cluster
+   * By default, all the instances are enabled
+   * @param clusterName
+   * @param instances
+   * @param enabled
+   */
+  void enableInstance(String clusterName, List<String> instances, boolean enabled);
+
+  /**
    * Disable or enable a resource
    * @param clusterName
    * @param resourceName
@@ -479,6 +488,26 @@ public interface HelixAdmin {
    */
   void enableBatchMessageMode(String clusterName, String resourceName, boolean enabled);
 
+
+
+  /**
+   * Get batch disabled instance map (disabled instance -> disabled time) in a cluster. It will
+   * include disabled instances and instances in disabled zones
+   * @param clusterName
+   * @return
+   */
+  Map<String, String> getBatchDisabledInstances(String clusterName);
+
+  /**
+   * Get list of instances by domain for a cluster
+   *
+   * Example : domain could be "helixZoneId=1,rackId=3". All the instances domain contains these
+   * two domains will be selected.
+   * @param clusterName
+   * @return
+   */
+  List<String> getInstancesByDomain(String clusterName, String domain);
+
   /**
    * Release resources
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index 2f7037b..c182ada 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -268,6 +268,7 @@ public class GenericHelixController implements IdealStateChangeListener,
       registry.register(ClusterEventType.CurrentStateChange, dataRefresh, rebalancePipeline, externalViewPipeline);
       registry.register(ClusterEventType.InstanceConfigChange, dataRefresh, rebalancePipeline);
       registry.register(ClusterEventType.ResourceConfigChange, dataRefresh, rebalancePipeline);
+      registry.register(ClusterEventType.ClusterConfigChange, dataRefresh, rebalancePipeline);
       registry.register(ClusterEventType.LiveInstanceChange, dataRefresh, liveInstancePipeline, rebalancePipeline,
           externalViewPipeline);
       registry.register(ClusterEventType.MessageChange, dataRefresh, rebalancePipeline);

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index 2dcad52..a44aa11 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -210,7 +210,7 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
     long currentTime = System.currentTimeMillis();
     for (String ins : offlineOrDisabledInstances) {
       long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
-          instanceConfigMap.get(ins));
+          instanceConfigMap.get(ins), clusterConfig);
       InstanceConfig instanceConfig = instanceConfigMap.get(ins);
       if (inactiveTime > currentTime && instanceConfig != null && instanceConfig
           .isDelayRebalanceEnabled()) {
@@ -237,7 +237,7 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
     // calculate the closest future rebalance time
     for (String ins : offlineOrDisabledInstances) {
       long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
-          instanceConfigMap.get(ins));
+          instanceConfigMap.get(ins), clusterConfig);
       if (inactiveTime != -1 && inactiveTime > currentTime && inactiveTime < nextRebalanceTime) {
         nextRebalanceTime = inactiveTime;
       }
@@ -265,7 +265,7 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
    * @return
    */
   private long getInactiveTime(String instance, Set<String> liveInstances, Long offlineTime,
-      long delay, InstanceConfig instanceConfig) {
+      long delay, InstanceConfig instanceConfig, ClusterConfig clusterConfig) {
     long inactiveTime = Long.MAX_VALUE;
 
     // check the time instance went offline.
@@ -276,8 +276,17 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
     }
 
     // check the time instance got disabled.
-    if (!instanceConfig.getInstanceEnabled()) {
+    if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
+        && clusterConfig.getDisabledInstances().containsKey(instance))) {
       long disabledTime = instanceConfig.getInstanceEnabledTime();
+      if (clusterConfig.getDisabledInstances() != null && clusterConfig.getDisabledInstances()
+          .containsKey(instance)) {
+        // Update batch disable time
+        long batchDisableTime = Long.parseLong(clusterConfig.getDisabledInstances().get(instance));
+        if (disabledTime == -1 || disabledTime > batchDisableTime) {
+          disabledTime = batchDisableTime;
+        }
+      }
       if (disabledTime > 0 && disabledTime + delay < inactiveTime) {
         inactiveTime = disabledTime + delay;
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java
index 8350bd8..bf00e0e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/topology/Topology.java
@@ -229,7 +229,8 @@ public class Topology {
         String zone = config.getZoneId();
         if (zone == null) {
           // we have the hierarchy style of domain id for instance.
-          if (config.getInstanceEnabled()) {
+          if (config.getInstanceEnabled() && (_clusterConfig.getDisabledInstances() == null
+              || !_clusterConfig.getDisabledInstances().containsKey(ins))) {
             // if enabled instance missing ZONE_ID information, fails the rebalance.
             throw new HelixException(String
                 .format("ZONE_ID for instance %s is not set, failed the topology-aware placement!",
@@ -274,7 +275,8 @@ public class Topology {
       }
       String domain = insConfig.getDomain();
       if (domain == null) {
-        if (insConfig.getInstanceEnabled()) {
+        if (insConfig.getInstanceEnabled() && (_clusterConfig.getDisabledInstances() == null
+            || !_clusterConfig.getDisabledInstances().containsKey(ins))) {
           // if enabled instance missing domain information, fails the rebalance.
           throw new HelixException(String
               .format("Domain for instance %s is not set, failed the topology-aware placement!",

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index 97e91f6..8999ed7 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -701,7 +701,8 @@ public class ClusterDataCache {
     Set<String> disabledInstancesSet = new HashSet<String>();
     for (String instance : _instanceConfigMap.keySet()) {
       InstanceConfig config = _instanceConfigMap.get(instance);
-      if (config.getInstanceEnabled() == false
+      if (config.getInstanceEnabled() == false || (_clusterConfig.getDisabledInstances() != null
+          && _clusterConfig.getDisabledInstances().containsKey(instance))
           || config.getInstanceEnabledForPartition(resource, partition) == false) {
         disabledInstancesSet.add(instance);
       }
@@ -718,7 +719,9 @@ public class ClusterDataCache {
     Set<String> disabledInstancesSet = new HashSet<>();
     for (String instance : _instanceConfigMap.keySet()) {
       InstanceConfig config = _instanceConfigMap.get(instance);
-      if (!config.getInstanceEnabled()) {
+      if (!config.getInstanceEnabled()
+          || (_clusterConfig.getDisabledInstances() != null && _clusterConfig.getDisabledInstances()
+          .containsKey(instance))) {
         disabledInstancesSet.add(instance);
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
index 9313157..9361249 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
@@ -29,6 +29,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.controller.GenericHelixController;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
@@ -60,6 +61,7 @@ public class ReadClusterDataStage extends AbstractBaseStage {
 
     HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
     _cache.refresh(dataAccessor);
+    final ClusterConfig clusterConfig = cache.getClusterConfig();
     if (!_cache.isTaskCache()) {
       final ClusterStatusMonitor clusterStatusMonitor =
           event.getAttribute(AttributeName.clusterStatusMonitor.name());
@@ -83,7 +85,8 @@ public class ReadClusterDataStage extends AbstractBaseStage {
               if (liveInstanceMap.containsKey(instanceName)) {
                 liveInstanceSet.add(instanceName);
               }
-              if (!config.getInstanceEnabled()) {
+              if (!config.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
+                  && clusterConfig.getDisabledInstances().containsKey(instanceName))) {
                 disabledInstanceSet.add(instanceName);
               }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
index a29ef78..554b09a 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
@@ -80,6 +80,7 @@ public class ControllerManagerHelper {
        */
       _manager.addInstanceConfigChangeListener(controller);
       _manager.addResourceConfigChangeListener(controller);
+      _manager.addClusterfigChangeListener(controller);
       _manager.addLiveInstanceChangeListener(controller);
       _manager.addIdealStateChangeListener(controller);
       _manager.addControllerListener(controller);

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index f5897af..c3fa9e9 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -33,11 +33,14 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
+import org.apache.commons.math.stat.clustering.Cluster;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.ConfigAccessor;
@@ -74,7 +77,6 @@ import org.apache.helix.util.RebalanceUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 public class ZKHelixAdmin implements HelixAdmin {
   public static final String CONNECTION_TIMEOUT = "helixAdmin.timeOutInSec";
   private final ZkClient _zkClient;
@@ -123,19 +125,20 @@ public class ZKHelixAdmin implements HelixAdmin {
 
     String instanceConfigPath = PropertyPathBuilder.instanceConfig(clusterName, instanceName);
     if (!_zkClient.exists(instanceConfigPath)) {
-      throw new HelixException("Node " + instanceName + " does not exist in config for cluster "
-          + clusterName);
+      throw new HelixException(
+          "Node " + instanceName + " does not exist in config for cluster " + clusterName);
     }
 
     String instancePath = PropertyPathBuilder.instance(clusterName, instanceName);
     if (!_zkClient.exists(instancePath)) {
-      throw new HelixException("Node " + instanceName + " does not exist in instances for cluster "
-          + clusterName);
+      throw new HelixException(
+          "Node " + instanceName + " does not exist in instances for cluster " + clusterName);
     }
 
     String liveInstancePath = PropertyPathBuilder.liveInstance(clusterName, instanceName);
     if (_zkClient.exists(liveInstancePath)) {
-      throw new HelixException("Node " + instanceName + " is still alive for cluster " + clusterName + ", can't drop.");
+      throw new HelixException(
+          "Node " + instanceName + " is still alive for cluster " + clusterName + ", can't drop.");
     }
 
     // delete config path
@@ -150,8 +153,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   public InstanceConfig getInstanceConfig(String clusterName, String instanceName) {
     String instanceConfigPath = PropertyPathBuilder.instanceConfig(clusterName, instanceName);
     if (!_zkClient.exists(instanceConfigPath)) {
-      throw new HelixException("instance" + instanceName + " does not exist in cluster "
-          + clusterName);
+      throw new HelixException(
+          "instance" + instanceName + " does not exist in cluster " + clusterName);
     }
 
     HelixDataAccessor accessor =
@@ -161,7 +164,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     return accessor.getProperty(keyBuilder.instanceConfig(instanceName));
   }
 
-  @Override public boolean setInstanceConfig(String clusterName, String instanceName,
+  @Override
+  public boolean setInstanceConfig(String clusterName, String instanceName,
       InstanceConfig newInstanceConfig) {
     String instanceConfigPath = PropertyPathBuilder.getPath(PropertyType.CONFIGS, clusterName,
         HelixConfigScope.ConfigScopeProperty.PARTICIPANT.toString(), instanceName);
@@ -188,27 +192,22 @@ public class ZKHelixAdmin implements HelixAdmin {
   @Override
   public void enableInstance(final String clusterName, final String instanceName,
       final boolean enabled) {
-    String path = PropertyPathBuilder.instanceConfig(clusterName, instanceName);
-
-    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkClient);
-    if (!baseAccessor.exists(path, 0)) {
-      throw new HelixException("Cluster " + clusterName + ", instance: " + instanceName
-          + ", instance config does not exist");
-    }
-
-    baseAccessor.update(path, new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
-        if (currentData == null) {
-          throw new HelixException("Cluster: " + clusterName + ", instance: " + instanceName
-              + ", participant config is null");
-        }
+    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
+    enableSingleInstance(clusterName, instanceName, enabled, baseAccessor);
+    enableBatchInstances(clusterName, Collections.singletonList(instanceName), enabled,
+        baseAccessor);
+  }
 
-        InstanceConfig config = new InstanceConfig(currentData);
-        config.setInstanceEnabled(enabled);
-        return config.getRecord();
+  @Override
+  public void enableInstance(String clusterName, List<String> instances,
+      boolean enabled) {
+    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
+    if (enabled) {
+      for (String instance : instances) {
+        enableSingleInstance(clusterName, instance, enabled, baseAccessor);
       }
-    }, AccessOption.PERSISTENT);
+    }
+    enableBatchInstances(clusterName, instances, enabled, baseAccessor);
   }
 
   @Override
@@ -221,7 +220,8 @@ public class ZKHelixAdmin implements HelixAdmin {
           + ", ideal-state does not exist");
     }
     baseAccessor.update(path, new DataUpdater<ZNRecord>() {
-      @Override public ZNRecord update(ZNRecord currentData) {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
         if (currentData == null) {
           throw new HelixException(
               "Cluster: " + clusterName + ", resource: " + resourceName + ", ideal-state is null");
@@ -260,22 +260,22 @@ public class ZKHelixAdmin implements HelixAdmin {
     if (idealStateRecord == null) {
       // throw new HelixException("Cluster: " + clusterName + ", resource: " + resourceName
       // + ", ideal state does not exist");
-      logger.warn("Disable partitions: " + partitionNames + " but Cluster: " + clusterName
-          + ", resource: " + resourceName
-          + " does not exists. probably disable it during ERROR->DROPPED transtition");
-
+      logger.warn(
+          "Disable partitions: " + partitionNames + " but Cluster: " + clusterName + ", resource: "
+              + resourceName
+              + " does not exists. probably disable it during ERROR->DROPPED transtition");
     } else {
       // check partitions exist. warn if not
       IdealState idealState = new IdealState(idealStateRecord);
       for (String partitionName : partitionNames) {
-        if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO && idealState
-            .getPreferenceList(partitionName) == null)
-            || (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED && idealState
-                .getPreferenceList(partitionName) == null)
-            || (idealState.getRebalanceMode() == RebalanceMode.TASK && idealState
-                .getPreferenceList(partitionName) == null)
-            || (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED && idealState
-                .getInstanceStateMap(partitionName) == null)) {
+        if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO
+            && idealState.getPreferenceList(partitionName) == null) || (
+            idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED
+                && idealState.getPreferenceList(partitionName) == null) || (
+            idealState.getRebalanceMode() == RebalanceMode.TASK
+                && idealState.getPreferenceList(partitionName) == null) || (
+            idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED
+                && idealState.getInstanceStateMap(partitionName) == null)) {
           logger.warn("Cluster: " + clusterName + ", resource: " + resourceName + ", partition: "
               + partitionName + ", partition does not exist in ideal state");
         }
@@ -285,7 +285,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     // update participantConfig
     // could not use ZNRecordUpdater since it doesn't do listField merge/subtract
     baseAccessor.update(path, new DataUpdater<ZNRecord>() {
-      @Override public ZNRecord update(ZNRecord currentData) {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
         if (currentData == null) {
           throw new HelixException("Cluster: " + clusterName + ", instance: " + instanceName
               + ", participant config is null");
@@ -339,15 +340,17 @@ public class ZKHelixAdmin implements HelixAdmin {
     // check the instance is alive
     LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instanceName));
     if (liveInstance == null) {
-      throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-          + " on " + instanceName + ", because " + instanceName + " is not alive");
+      throw new HelixException(
+          "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+              + ", because " + instanceName + " is not alive");
     }
 
     // check resource group exists
     IdealState idealState = accessor.getProperty(keyBuilder.idealStates(resourceName));
     if (idealState == null) {
-      throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-          + " on " + instanceName + ", because " + resourceName + " is not added");
+      throw new HelixException(
+          "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+              + ", because " + resourceName + " is not added");
     }
 
     // check partition exists in resource group
@@ -355,14 +358,16 @@ public class ZKHelixAdmin implements HelixAdmin {
     if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
       Set<String> partitions = new HashSet<String>(idealState.getRecord().getMapFields().keySet());
       if (!partitions.containsAll(resetPartitionNames)) {
-        throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-            + " on " + instanceName + ", because not all " + partitionNames + " exist");
+        throw new HelixException(
+            "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+                + ", because not all " + partitionNames + " exist");
       }
     } else {
       Set<String> partitions = new HashSet<String>(idealState.getRecord().getListFields().keySet());
       if (!partitions.containsAll(resetPartitionNames)) {
-        throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-            + " on " + instanceName + ", because not all " + partitionNames + " exist");
+        throw new HelixException(
+            "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+                + ", because not all " + partitionNames + " exist");
       }
     }
 
@@ -372,8 +377,9 @@ public class ZKHelixAdmin implements HelixAdmin {
         accessor.getProperty(keyBuilder.currentState(instanceName, sessionId, resourceName));
     for (String partitionName : resetPartitionNames) {
       if (!curState.getState(partitionName).equals(HelixDefinedState.ERROR.toString())) {
-        throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-            + " on " + instanceName + ", because not all " + partitionNames + " are in ERROR state");
+        throw new HelixException(
+            "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+                + ", because not all " + partitionNames + " are in ERROR state");
       }
     }
 
@@ -381,22 +387,23 @@ public class ZKHelixAdmin implements HelixAdmin {
     String stateModelDef = idealState.getStateModelDefRef();
     StateModelDefinition stateModel = accessor.getProperty(keyBuilder.stateModelDef(stateModelDef));
     if (stateModel == null) {
-      throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-          + " on " + instanceName + ", because " + stateModelDef + " is NOT found");
+      throw new HelixException(
+          "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+              + ", because " + stateModelDef + " is NOT found");
     }
 
     // check there is no pending messages for the partitions exist
     List<Message> messages = accessor.getChildValues(keyBuilder.messages(instanceName));
     for (Message message : messages) {
-      if (!MessageType.STATE_TRANSITION.name().equalsIgnoreCase(message.getMsgType())
-          || !sessionId.equals(message.getTgtSessionId())
-          || !resourceName.equals(message.getResourceName())
+      if (!MessageType.STATE_TRANSITION.name().equalsIgnoreCase(message.getMsgType()) || !sessionId
+          .equals(message.getTgtSessionId()) || !resourceName.equals(message.getResourceName())
           || !resetPartitionNames.contains(message.getPartitionName())) {
         continue;
       }
 
-      throw new HelixException("Can't reset state for " + resourceName + "/" + partitionNames
-          + " on " + instanceName + ", because a pending message exists: " + message);
+      throw new HelixException(
+          "Can't reset state for " + resourceName + "/" + partitionNames + " on " + instanceName
+              + ", because a pending message exists: " + message);
     }
 
     String adminName = null;
@@ -455,8 +462,8 @@ public class ZKHelixAdmin implements HelixAdmin {
         for (String partitionName : stateMap.keySet()) {
           Map<String, String> instanceStateMap = stateMap.get(partitionName);
 
-          if (instanceStateMap.containsKey(instanceName)
-              && instanceStateMap.get(instanceName).equals(HelixDefinedState.ERROR.toString())) {
+          if (instanceStateMap.containsKey(instanceName) && instanceStateMap.get(instanceName)
+              .equals(HelixDefinedState.ERROR.toString())) {
             resetPartitionNames.add(partitionName);
           }
         }
@@ -629,12 +636,13 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void addResource(String clusterName, String resourceName, IdealState idealstate) {
+  public void addResource(String clusterName, String resourceName,
+      IdealState idealstate) {
     String stateModelRef = idealstate.getStateModelDefRef();
     String stateModelDefPath = PropertyPathBuilder.stateModelDef(clusterName, stateModelRef);
     if (!_zkClient.exists(stateModelDefPath)) {
-      throw new HelixException("State model " + stateModelRef
-          + " not found in the cluster STATEMODELDEFS path");
+      throw new HelixException(
+          "State model " + stateModelRef + " not found in the cluster STATEMODELDEFS path");
     }
 
     String idealStatePath = PropertyPathBuilder.idealState(clusterName);
@@ -652,7 +660,6 @@ public class ZKHelixAdmin implements HelixAdmin {
       String stateModelRef, String rebalancerMode, int bucketSize) {
     addResource(clusterName, resourceName, partitions, stateModelRef, rebalancerMode, bucketSize,
         -1);
-
   }
 
   @Override
@@ -733,7 +740,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void setResourceIdealState(String clusterName, String resourceName, IdealState idealState) {
+  public void setResourceIdealState(String clusterName, String resourceName,
+      IdealState idealState) {
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
     Builder keyBuilder = accessor.keyBuilder();
@@ -765,8 +773,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     String stateModelPath = stateModelDefPath + "/" + stateModelDef;
     if (_zkClient.exists(stateModelPath)) {
       if (recreateIfExists) {
-        logger.info("Operation.State Model directory exists:" + stateModelPath +
-            ", remove and recreate.");
+        logger.info(
+            "Operation.State Model directory exists:" + stateModelPath + ", remove and recreate.");
         _zkClient.deleteRecursive(stateModelPath);
       } else {
         logger.info("Skip the operation. State Model directory exists:" + stateModelPath);
@@ -796,7 +804,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public StateModelDefinition getStateModelDef(String clusterName, String stateModelName) {
+  public StateModelDefinition getStateModelDef(String clusterName,
+      String stateModelName) {
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
     Builder keyBuilder = accessor.keyBuilder();
@@ -884,8 +893,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void rebalance(String clusterName, String resourceName, int replica, String keyPrefix,
-      String group) {
+  public void rebalance(String clusterName, String resourceName, int replica,
+      String keyPrefix, String group) {
     List<String> instanceNames = new LinkedList<String>();
     if (keyPrefix == null || keyPrefix.length() == 0) {
       keyPrefix = resourceName;
@@ -904,7 +913,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void rebalance(String clusterName, String resourceName, int replica, List<String> instances) {
+  public void rebalance(String clusterName, String resourceName, int replica,
+      List<String> instances) {
     rebalance(clusterName, resourceName, replica, resourceName, instances, "");
   }
 
@@ -966,9 +976,9 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
     if (idealState.getRebalanceMode() != RebalanceMode.FULL_AUTO
         && idealState.getRebalanceMode() != RebalanceMode.USER_DEFINED) {
-      ZNRecord newIdealState =
-          DefaultIdealStateCalculator.calculateIdealState(instanceNames, partitions, replica,
-              keyPrefix, masterStateValue, slaveStateValue);
+      ZNRecord newIdealState = DefaultIdealStateCalculator
+          .calculateIdealState(instanceNames, partitions, replica, keyPrefix, masterStateValue,
+              slaveStateValue);
 
       // for now keep mapField in SEMI_AUTO mode and remove listField in CUSTOMIZED mode
       if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
@@ -990,8 +1000,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void addIdealState(String clusterName, String resourceName, String idealStateFile)
-      throws IOException {
+  public void addIdealState(String clusterName, String resourceName,
+      String idealStateFile) throws IOException {
     ZNRecord idealStateRecord =
         (ZNRecord) (new ZNRecordSerializer().deserialize(readFile(idealStateFile)));
     if (idealStateRecord.getId() == null || !idealStateRecord.getId().equals(resourceName)) {
@@ -1042,10 +1052,11 @@ public class ZKHelixAdmin implements HelixAdmin {
     String path = keyBuilder.constraint(constraintType.toString()).getPath();
 
     baseAccessor.update(path, new DataUpdater<ZNRecord>() {
-      @Override public ZNRecord update(ZNRecord currentData) {
-        ClusterConstraints constraints = currentData == null ?
-            new ClusterConstraints(constraintType) :
-            new ClusterConstraints(currentData);
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        ClusterConstraints constraints = currentData == null
+            ? new ClusterConstraints(constraintType)
+            : new ClusterConstraints(currentData);
 
         constraints.addConstraintItem(constraintId, constraintItem);
         return constraints.getRecord();
@@ -1076,7 +1087,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public ClusterConstraints getConstraints(String clusterName, ConstraintType constraintType) {
+  public ClusterConstraints getConstraints(String clusterName,
+      ConstraintType constraintType) {
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
 
@@ -1085,16 +1097,18 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   /**
-   * Takes the existing idealstate as input and computes newIdealState such that
-   * the partition movement is minimized. The partitions are redistributed among the instances
-   * provided.
+   * Takes the existing idealstate as input and computes newIdealState such that the partition
+   * movement is minimized. The partitions are redistributed among the instances provided.
+   *
    * @param clusterName
    * @param currentIdealState
    * @param instanceNames
+   *
    * @return
    */
   @Override
-  public void rebalance(String clusterName, IdealState currentIdealState, List<String> instanceNames) {
+  public void rebalance(String clusterName, IdealState currentIdealState,
+      List<String> instanceNames) {
     Set<String> activeInstances = new HashSet<String>();
     for (String partition : currentIdealState.getPartitionSet()) {
       activeInstances.addAll(currentIdealState.getRecord().getListField(partition));
@@ -1109,14 +1123,14 @@ public class ZKHelixAdmin implements HelixAdmin {
         this.getStateModelDef(clusterName, currentIdealState.getStateModelDefRef());
 
     if (stateModDef == null) {
-      throw new HelixException("cannot find state model: "
-          + currentIdealState.getStateModelDefRef());
+      throw new HelixException(
+          "cannot find state model: " + currentIdealState.getStateModelDefRef());
     }
     String[] states = RebalanceUtil.parseStates(clusterName, stateModDef);
 
-    ZNRecord newIdealStateRecord =
-        DefaultIdealStateCalculator.convertToZNRecord(balancedRecord,
-            currentIdealState.getResourceName(), states[0], states[1]);
+    ZNRecord newIdealStateRecord = DefaultIdealStateCalculator
+        .convertToZNRecord(balancedRecord, currentIdealState.getResourceName(), states[0],
+            states[1]);
     Set<String> partitionSet = new HashSet<String>();
     partitionSet.addAll(newIdealStateRecord.getMapFields().keySet());
     partitionSet.addAll(newIdealStateRecord.getListFields().keySet());
@@ -1129,12 +1143,12 @@ public class ZKHelixAdmin implements HelixAdmin {
         if (partition.equals(originPartitionName)) {
           continue;
         }
-        newIdealStateRecord.getMapFields().put(originPartitionName,
-            newIdealStateRecord.getMapField(partition));
+        newIdealStateRecord.getMapFields()
+            .put(originPartitionName, newIdealStateRecord.getMapField(partition));
         newIdealStateRecord.getMapFields().remove(partition);
 
-        newIdealStateRecord.getListFields().put(originPartitionName,
-            newIdealStateRecord.getListField(partition));
+        newIdealStateRecord.getListFields()
+            .put(originPartitionName, newIdealStateRecord.getListField(partition));
         newIdealStateRecord.getListFields().remove(partition);
       }
     }
@@ -1151,8 +1165,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
 
     if (!ZKUtil.isInstanceSetup(_zkClient, clusterName, instanceName, InstanceType.PARTICIPANT)) {
-      throw new HelixException("cluster " + clusterName + " instance " + instanceName
-          + " is not setup yet");
+      throw new HelixException(
+          "cluster " + clusterName + " instance " + instanceName + " is not setup yet");
     }
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
@@ -1170,8 +1184,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
 
     if (!ZKUtil.isInstanceSetup(_zkClient, clusterName, instanceName, InstanceType.PARTICIPANT)) {
-      throw new HelixException("cluster " + clusterName + " instance " + instanceName
-          + " is not setup yet");
+      throw new HelixException(
+          "cluster " + clusterName + " instance " + instanceName + " is not setup yet");
     }
     ZKHelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
@@ -1189,8 +1203,8 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
 
     if (!ZKUtil.isInstanceSetup(_zkClient, clusterName, instanceName, InstanceType.PARTICIPANT)) {
-      throw new HelixException("cluster " + clusterName + " instance " + instanceName
-          + " is not setup yet");
+      throw new HelixException(
+          "cluster " + clusterName + " instance " + instanceName + " is not setup yet");
     }
     HelixDataAccessor accessor =
         new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
@@ -1214,7 +1228,8 @@ public class ZKHelixAdmin implements HelixAdmin {
   }
 
   @Override
-  public void enableBatchMessageMode(String clusterName, String resourceName, boolean enabled) {
+  public void enableBatchMessageMode(String clusterName, String resourceName,
+      boolean enabled) {
     // TODO: Change IdealState to ResourceConfig when configs are migrated to ResourceConfig
     IdealState idealState = getResourceIdealState(clusterName, resourceName);
     if (idealState == null) {
@@ -1226,8 +1241,94 @@ public class ZKHelixAdmin implements HelixAdmin {
     setResourceIdealState(clusterName, resourceName, idealState);
   }
 
+  private void enableSingleInstance(final String clusterName, final String instanceName,
+      final boolean enabled, BaseDataAccessor<ZNRecord> baseAccessor) {
+    String path = PropertyPathBuilder.instanceConfig(clusterName, instanceName);
+
+    if (!baseAccessor.exists(path, 0)) {
+      throw new HelixException("Cluster " + clusterName + ", instance: " + instanceName
+          + ", instance config does not exist");
+    }
+
+    baseAccessor.update(path, new DataUpdater<ZNRecord>()
+
+    {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        if (currentData == null) {
+          throw new HelixException("Cluster: " + clusterName + ", instance: " + instanceName
+              + ", participant config is null");
+        }
+
+        InstanceConfig config = new InstanceConfig(currentData);
+        config.setInstanceEnabled(enabled);
+        return config.getRecord();
+      }
+    }, AccessOption.PERSISTENT);
+  }
+
+  private void enableBatchInstances(final String clusterName, final List<String> instances,
+      final boolean enabled, BaseDataAccessor<ZNRecord> baseAccessor) {
+
+    String path = PropertyPathBuilder.clusterConfig(clusterName);
+
+    if (!baseAccessor.exists(path, 0)) {
+      throw new HelixException("Cluster " + clusterName + ": cluster config does not exist");
+    }
+
+    baseAccessor.update(path, new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        if (currentData == null) {
+          throw new HelixException("Cluster: " + clusterName + ": cluster config is null");
+        }
+
+        ClusterConfig clusterConfig = new ClusterConfig(currentData);
+        Map<String, String> disabledInstances = new TreeMap<>();
+        if (clusterConfig.getDisabledInstances() != null) {
+          disabledInstances.putAll(clusterConfig.getDisabledInstances());
+        }
+
+        if (enabled) {
+          disabledInstances.keySet().removeAll(instances);
+        } else {
+          for (String disabledInstance : instances) {
+            if (!disabledInstances.containsKey(disabledInstance)) {
+              disabledInstances.put(disabledInstance, String.valueOf(System.currentTimeMillis()));
+            }
+          }
+        }
+        clusterConfig.setDisabledInstances(disabledInstances);
+
+        return clusterConfig.getRecord();
+      }
+    }, AccessOption.PERSISTENT);
+  }
+
+  @Override
+  public Map<String, String> getBatchDisabledInstances(String clusterName) {
+    ConfigAccessor accessor = new ConfigAccessor(_zkClient);
+    return accessor.getClusterConfig(clusterName).getDisabledInstances();
+  }
+
   @Override
-  public void close() {
+  public List<String> getInstancesByDomain(String clusterName, String domain) {
+    List<String> instances = new ArrayList<>();
+    String path = PropertyPathBuilder.instanceConfig(clusterName);
+    BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
+    List<ZNRecord> znRecords = baseAccessor.getChildren(path, null, 0);
+    for (ZNRecord record : znRecords) {
+      if (record != null) {
+        InstanceConfig instanceConfig = new InstanceConfig(record);
+        if (instanceConfig.isInstanceInDomain(domain)) {
+          instances.add(instanceConfig.getInstanceName());
+        }
+      }
+    }
+    return instances;
+  }
+
+  @Override public void close() {
     if (_zkClient != null) {
       _zkClient.close();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
index 6f74728..2a97145 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
@@ -60,7 +60,8 @@ public class ClusterConfig extends HelixProperty {
     MAX_PARTITIONS_PER_INSTANCE,
     MAX_OFFLINE_INSTANCES_ALLOWED,
     TARGET_EXTERNALVIEW_ENABLED,
-    ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE // Controller won't execute load balance state transition if the number of partitons that need recovery exceeds this limitation
+    ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE, // Controller won't execute load balance state transition if the number of partitons that need recovery exceeds this limitation
+    DISABLED_INSTANCES
   }
   private final static int DEFAULT_MAX_CONCURRENT_TASK_PER_INSTANCE = 40;
   private final static int DEFAULT_ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE = 0; // By default, no load balance if any error partition
@@ -474,6 +475,21 @@ public class ClusterConfig extends HelixProperty {
     _record.setIntField(ClusterConfigProperty.ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE.name(),
         errorPartitionThreshold);
   }
+  /**
+   * Set the disabled instance list
+   * @param disabledInstances
+   */
+  public void setDisabledInstances(Map<String, String> disabledInstances) {
+    _record.setMapField(ClusterConfigProperty.DISABLED_INSTANCES.name(), disabledInstances);
+  }
+
+  /**
+   * Get current disabled instance map of <instance, disabledTimeStamp>
+   * @return
+   */
+  public Map<String, String> getDisabledInstances() {
+    return _record.getMapField(ClusterConfigProperty.DISABLED_INSTANCES.name());
+  }
 
   /**
    * Get IdealState rules defined in the cluster config.

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
index e88f37d..4343006 100644
--- a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
@@ -20,6 +20,7 @@ package org.apache.helix.model;
  */
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -27,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.HelixException;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.util.HelixUtil;
@@ -426,6 +428,22 @@ public class InstanceConfig extends HelixProperty {
     }
   }
 
+  public boolean isInstanceInDomain(String domain) {
+    if (domain == null) {
+      throw new HelixException("Invalid input for domain.");
+    }
+
+    if (_record.getSimpleField(InstanceConfigProperty.DOMAIN.name()) == null) {
+      return false;
+    }
+
+    Set<String> domainSet = new HashSet<>(Arrays.asList(domain.split(",")));
+    Set<String> instanceDomains = new HashSet<>(
+        Arrays.asList(_record.getSimpleField(InstanceConfigProperty.DOMAIN.name()).split(",")));
+    domainSet.removeAll(instanceDomains);
+    return domainSet.size() == 0;
+  }
+
   /**
    * Whether the delay rebalance is enabled for this instance.
    * By default, it is enable if the field is not set.

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
index 30483f6..5736169 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
@@ -46,6 +46,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
@@ -217,8 +218,10 @@ public class ClusterSetup {
       throw new HelixException(error);
     }
 
+    ClusterConfig clusterConfig = accessor.getProperty(keyBuilder.clusterConfig());
     // ensure node is disabled, otherwise fail
-    if (config.getInstanceEnabled()) {
+    if (config.getInstanceEnabled() && (clusterConfig.getDisabledInstances() == null
+        || !clusterConfig.getDisabledInstances().containsKey(instanceId))) {
       String error = "Node " + instanceId + " is enabled, cannot drop";
       _logger.warn(error);
       throw new HelixException(error);
@@ -245,8 +248,10 @@ public class ClusterSetup {
       throw new HelixException(error);
     }
 
+    ClusterConfig clusterConfig = accessor.getProperty(keyBuilder.clusterConfig());
     // ensure old instance is disabled, otherwise fail
-    if (oldConfig.getInstanceEnabled()) {
+    if (oldConfig.getInstanceEnabled() && (clusterConfig.getDisabledInstances() == null
+        || !clusterConfig.getDisabledInstances().containsKey(oldInstanceName))) {
       String error =
           "Old instance " + oldInstanceName + " is enabled, it need to be disabled and turned off";
       _logger.warn(error);

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
new file mode 100644
index 0000000..eab5c32
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
@@ -0,0 +1,110 @@
+package org.apache.helix.integration;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.integration.task.TaskTestBase;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ExternalView;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestBatchEnableInstances extends TaskTestBase {
+  private ConfigAccessor _accessor;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 1;
+    _numReplicas = 3;
+    _numNodes = 5;
+    _numParitions = 4;
+    super.beforeClass();
+    _accessor = new ConfigAccessor(_gZkClient);
+  }
+
+  @Test
+  public void testOldEnableDisable() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), false);
+    Thread.sleep(2000);
+
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    Assert.assertEquals(externalView.getRecord().getMapFields().size(), _numParitions);
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      Assert.assertTrue(!stateMap.keySet().contains(_participants[0].getInstanceName()));
+    }
+    _gSetupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), true);
+  }
+
+  @Test
+  public void testBatchEnableDisable() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
+        Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
+        false);
+    Thread.sleep(2000);
+
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    Assert.assertEquals(externalView.getRecord().getMapFields().size(), _numParitions);
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      Assert.assertTrue(!stateMap.keySet().contains(_participants[0].getInstanceName()));
+      Assert.assertTrue(!stateMap.keySet().contains(_participants[1].getInstanceName()));
+    }
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
+        Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
+        true);
+  }
+
+  @Test
+  public void testOldDisableBatchEnable() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), false);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
+        Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
+        true);
+    Thread.sleep(2000);
+
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    Assert.assertEquals(externalView.getRecord().getMapFields().size(), _numParitions);
+    int numOfFirstHost = 0;
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      if (stateMap.keySet().contains(_participants[0].getInstanceName())) {
+        numOfFirstHost++;
+      }
+    }
+    Assert.assertTrue(numOfFirstHost > 0);
+    _gSetupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), true);
+  }
+
+  @Test
+  public void testBatchDisableOldEnable() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
+        Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
+        false);
+    _gSetupTool.getClusterManagementTool()
+        .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), true);
+    Thread.sleep(2000);
+
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    Assert.assertEquals(externalView.getRecord().getMapFields().size(), _numParitions);
+    int numOfFirstHost = 0;
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      if (stateMap.keySet().contains(_participants[0].getInstanceName())) {
+        numOfFirstHost++;
+      }
+      Assert.assertTrue(!stateMap.keySet().contains(_participants[1].getInstanceName()));
+    }
+    Assert.assertTrue(numOfFirstHost > 0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
+        Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
+        true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
index 2de3a10..04c3ed4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkCallbackHandlerLeak.java
@@ -90,10 +90,10 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 6 + 5*n zk-watchers.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers.");
 
     // check participant zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -115,7 +115,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
     // printHandlers(participantManagerToExpire);
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManagerToExpire.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, 10,
+    Assert.assertEquals(controllerHandlerNb, 11,
         "HelixController should have 10 (5+2n) callback handlers for 2 (n) participant");
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handlers");
@@ -145,10 +145,10 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 6 + 5*n zk-watchers after session expiry.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers after session expiry.");
 
     // check participant zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -241,7 +241,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManager.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, 10,
+    Assert.assertEquals(controllerHandlerNb, 11,
         "HelixController should have 10 (6+2n) callback handlers for 2 participant, but was "
             + controllerHandlerNb + ", " + printHandlers(controller));
     Assert.assertEquals(particHandlerNb, 1,
@@ -273,10 +273,10 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 6 + 5*n zk-watchers after session expiry.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers after session expiry.");
 
     // check participant zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
index 176fe33..a30de78 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestConsecutiveZkSessionExpiry.java
@@ -244,7 +244,7 @@ public class TestConsecutiveZkSessionExpiry extends ZkUnitTestBase {
     Assert
         .assertEquals(
             handlers.size(),
-            1,
+            2,
             "Distributed controller should have 2 handler (message) after lose leadership, but was "
                 + handlers.size());
 

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
index ea00888..142cd1f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestDistributedControllerManager.java
@@ -147,7 +147,7 @@ public class TestDistributedControllerManager extends ZkIntegrationTestBase {
     Assert
         .assertEquals(
             handlers.size(),
-            1,
+            2,
             "Distributed controller should have 1 handler (message) after lose leadership, but was "
                 + handlers.size());
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
index 00f95b0..ec2dd1b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
@@ -96,7 +96,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
         return watchPaths.size() == (7 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 7 + 5*n zk-watchers.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers.");
 
     // check participant zk-watchers
     final MockParticipantManager participantManagerToExpire = participants[0];
@@ -152,7 +152,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
         return watchPaths.size() == (7 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 7 + 5*n zk-watchers after session expiry.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers after session expiry.");
 
     // check participant zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {
@@ -237,7 +237,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManager.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, (6 + 2 * n),
+    Assert.assertEquals(controllerHandlerNb, (7 + 2 * n),
         "HelixController should have 9 (5+2n) callback handlers for 2 participant, but was "
             + controllerHandlerNb + ", " + TestHelper.printHandlers(controller));
     Assert.assertEquals(particHandlerNb, 1,
@@ -270,10 +270,10 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
-    Assert.assertTrue(result, "Controller should have 7 + 5*n zk-watchers after session expiry.");
+    Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers after session expiry.");
 
     // check participant zk-watchers
     result = TestHelper.verify(new TestHelper.Verifier() {

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
index beeb3cf..8679007 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
@@ -219,6 +219,11 @@ public class MockHelixAdmin implements HelixAdmin {
 
   }
 
+  @Override public void enableInstance(String clusterName, List<String> instances,
+      boolean enabled) {
+
+  }
+
   @Override public void enableResource(String clusterName, String resourceName, boolean enabled) {
 
   }
@@ -366,6 +371,15 @@ public class MockHelixAdmin implements HelixAdmin {
 
   }
 
+  @Override
+  public Map<String, String> getBatchDisabledInstances(String clusterName) {
+    return null;
+  }
+
+  @Override public List<String> getInstancesByDomain(String clusterName, String domain) {
+    return null;
+  }
+
   @Override public void close() {
 
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java b/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
index 4109fa3..e8c7b0d 100644
--- a/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/task/TaskSynchronizedTestBase.java
@@ -110,6 +110,7 @@ public class TaskSynchronizedTestBase extends ZkIntegrationTestBase {
   }
 
   protected void setupParticipants() {
+    _participants = new MockParticipantManager[_numNodes];
     for (int i = 0; i < _numNodes; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (_startPort + i);
       _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);

http://git-wip-us.apache.org/repos/asf/helix/blob/31cec911/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
index 4fe8060..5b25c3e 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
@@ -36,6 +36,7 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Error;
 import org.apache.helix.model.HealthStat;
@@ -91,13 +92,14 @@ public class InstanceAccessor extends AbstractResource {
     }
 
     List<String> liveInstances = accessor.getChildNames(accessor.keyBuilder().liveInstances());
-
+    ClusterConfig clusterConfig = accessor.getProperty(accessor.keyBuilder().clusterConfig());
 
     for (String instanceName : instances) {
       InstanceConfig instanceConfig =
           accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
       if (instanceConfig != null) {
-        if (!instanceConfig.getInstanceEnabled()) {
+        if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
+            && clusterConfig.getDisabledInstances().containsKey(instanceName))) {
           disabledNode.add(JsonNodeFactory.instance.textNode(instanceName));
         }
 


[02/50] [abbrv] helix git commit: Fix a minor issue when updating a workflowConfig with empty workflowId.

Posted by jx...@apache.org.
Fix a minor issue when updating a workflowConfig with empty workflowId.


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

Branch: refs/heads/master
Commit: 00f8efea0daf0418d4a85dcfe6c6c2a009114c47
Parents: 31cec91
Author: Lei Xia <lx...@linkedin.com>
Authored: Thu Nov 2 13:26:38 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:17 2018 -0800

----------------------------------------------------------------------
 helix-core/src/main/java/org/apache/helix/task/TaskDriver.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/00f8efea/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index 90abc59..732a717 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -167,7 +167,7 @@ public class TaskDriver {
    * @param newWorkflowConfig
    */
   public void updateWorkflow(String workflow, WorkflowConfig newWorkflowConfig) {
-    if (newWorkflowConfig.getWorkflowId() == null) {
+    if (newWorkflowConfig.getWorkflowId() == null || newWorkflowConfig.getWorkflowId().isEmpty()) {
       newWorkflowConfig.getRecord()
           .setSimpleField(WorkflowConfig.WorkflowConfigProperty.WorkflowID.name(), workflow);
     }


[50/50] [abbrv] helix git commit: Fix duplicated functions

Posted by jx...@apache.org.
Fix duplicated functions


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

Branch: refs/heads/master
Commit: 7ba8c5197ca3b1f82553b3d6029c857afa7becee
Parents: 4e7eca5
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Jan 24 18:07:09 2018 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:38 2018 -0800

----------------------------------------------------------------------
 helix-core/src/main/java/org/apache/helix/PropertyKey.java | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7ba8c519/helix-core/src/main/java/org/apache/helix/PropertyKey.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
index 4129671..4e58a89 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -322,14 +322,6 @@ public class PropertyKey {
     }
 
     /**
-     * Get a property key associated with specified instance
-     * @return {@link PropertyKey}
-     */
-    public PropertyKey instance(String instanceName) {
-      return new PropertyKey(PropertyType.INSTANCES, null, _clusterName, instanceName);
-    }
-
-    /**
      * Get a property key associated with {@link Message} for an instance
      * @param instanceName
      * @return {@link PropertyKey}


[03/50] [abbrv] helix git commit: Add test for GenericHelixController thread leak

Posted by jx...@apache.org.
Add test for GenericHelixController thread leak


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

Branch: refs/heads/master
Commit: ac39b3984f16ce593b99c8482f7d16522dcae935
Parents: 00f8efe
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Oct 31 17:12:04 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:21 2018 -0800

----------------------------------------------------------------------
 .../TestGenericHelixControllerThreading.java    | 73 ++++++++++++++++++++
 1 file changed, 73 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ac39b398/helix-core/src/test/java/org/apache/helix/integration/controller/TestGenericHelixControllerThreading.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestGenericHelixControllerThreading.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestGenericHelixControllerThreading.java
new file mode 100644
index 0000000..822e060
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestGenericHelixControllerThreading.java
@@ -0,0 +1,73 @@
+package org.apache.helix.integration.controller;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Set;
+import org.apache.helix.controller.GenericHelixController;
+import org.apache.helix.ZkUnitTestBase;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestGenericHelixControllerThreading extends ZkUnitTestBase {
+
+  private static final String EVENT_PROCESS_THREAD_NAME_PREFIX =
+      "GerenricHelixController-event_process";
+
+  //    Temporarily disabling the test as it's not stable when running under the entire mvn test suite.
+  //    Some other crashed tests might cause this one to fail as controllers might not be gracefully
+  //    shutdown
+  @Test(enabled = false)
+  public void testGenericHelixControllerThreadCount() throws Exception {
+    System.out.println("testGenericHelixControllerThreadCount STARTs");
+    final int numControllers = 100;
+    ArrayList<GenericHelixController> controllers = createHelixControllers(numControllers);
+    Assert.assertEquals(getThreadCountByNamePrefix(EVENT_PROCESS_THREAD_NAME_PREFIX), numControllers * 2);
+
+    int remainingExpectedEventProcessThreadsCount = numControllers * 2;
+    for (GenericHelixController ctrl : controllers) {
+      ctrl.shutdown();
+      remainingExpectedEventProcessThreadsCount -= 2;
+      Assert.assertEquals(getThreadCountByNamePrefix(EVENT_PROCESS_THREAD_NAME_PREFIX),
+          remainingExpectedEventProcessThreadsCount);
+    }
+    System.out.println("testGenericHelixControllerThreadCount ENDs");
+  }
+
+  private ArrayList<GenericHelixController> createHelixControllers(int count) {
+    ArrayList<GenericHelixController> ret = new ArrayList<>();
+    for (int i = 0; i < count; i++) {
+      ret.add(new GenericHelixController());
+    }
+    return ret;
+  }
+
+  private int getThreadCountByNamePrefix(String threadNamePrefix) {
+    Set<Thread> threadSet = Thread.getAllStackTraces().keySet();
+    int eventThreadCount = 0;
+    for (Thread t : threadSet) {
+      if (t.getName().startsWith(threadNamePrefix)) {
+        eventThreadCount += 1;
+      }
+    }
+    return eventThreadCount;
+  }
+
+}


[28/50] [abbrv] helix git commit: Fix NPE if rebalance strategy is not specified in IS.

Posted by jx...@apache.org.
Fix NPE if rebalance strategy is not specified in IS.


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

Branch: refs/heads/master
Commit: 019d6f4ddb21f0c9371fcf6f4e99504fa961c551
Parents: d9696cc
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Dec 11 11:41:18 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:16 2018 -0800

----------------------------------------------------------------------
 .../helix/controller/rebalancer/DelayedAutoRebalancer.java  | 2 +-
 .../java/org/apache/helix/messaging/handling/HelixTask.java | 9 ++++++++-
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/019d6f4d/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index 7ad2eb2..f943abf 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -66,7 +66,7 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
     ZNRecord znRecord = clusterData.getCachedIdealMapping(resourceName);
     if (znRecord != null) {
       // TODO: only apply to legacy Auto-RebalanceStrategy at this time, need to apply to any strategy in future.
-      if (currentIdealState.getRebalanceStrategy().equals(AutoRebalanceStrategy.class.getName())) {
+      if (AutoRebalanceStrategy.class.getName().equals(currentIdealState.getRebalanceStrategy())) {
         LOG.info("Use cached idealstate for " + resourceName);
         IdealState idealState = new IdealState(znRecord);
         return idealState;

http://git-wip-us.apache.org/repos/asf/helix/blob/019d6f4d/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
index 7b1853f..2543d81 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
@@ -214,6 +214,8 @@ public class HelixTask implements MessageTask {
     boolean success = accessor.removeProperty(msgKey);
     if (!success) {
       logger.warn("Failed to delete message " + message.getId() + " from zk!");
+    } else {
+      logger.info("Delete message " + message.getId() + " from zk!");
     }
   }
 
@@ -225,6 +227,9 @@ public class HelixTask implements MessageTask {
 
       // Ignore all relay messages if participant's session has changed.
       if (!_manager.getSessionId().equals(message.getTgtSessionId())) {
+        logger.info(
+            "Session id has been changed, ignore all relay messages attached with " + message
+                .getId());
         return;
       }
 
@@ -234,7 +239,7 @@ public class HelixTask implements MessageTask {
           msg.setRelayTime(taskCompletionTime);
           if (msg.isExpired()) {
             logger.info(
-                "Relay message expired, ignore it! " + msg.getId() + " to instance " + instance);
+                "Relay message expired, ignore " + msg.getId() + " to instance " + instance);
             continue;
           }
           PropertyKey msgKey = keyBuilder.message(instance, msg.getId());
@@ -242,6 +247,8 @@ public class HelixTask implements MessageTask {
               .create(msgKey.getPath(), msg.getRecord(), AccessOption.PERSISTENT);
           if (!success) {
             logger.warn("Failed to send relay message " + msg.getId() + " to " + instance);
+          } else {
+            logger.info("Send relay message " + message.getId() + " to " + instance);
           }
         }
       }


[10/50] [abbrv] helix git commit: Use SlidingTimeWindowArrayReservoir to reduce memory consumption

Posted by jx...@apache.org.
Use SlidingTimeWindowArrayReservoir to reduce memory consumption


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

Branch: refs/heads/master
Commit: 143420c4f460636a82d5f53a6a9a22bdb6bc9ff6
Parents: 3a73b0f
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Nov 14 14:49:47 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:05 2018 -0800

----------------------------------------------------------------------
 helix-core/pom.xml                                        |  2 +-
 .../helix/monitoring/mbeans/ClusterEventMonitor.java      |  6 +++---
 .../helix/monitoring/mbeans/HelixCallbackMonitor.java     |  4 ++--
 .../helix/monitoring/mbeans/MessageLatencyMonitor.java    |  4 ++--
 .../apache/helix/monitoring/mbeans/ResourceMonitor.java   |  4 ++--
 .../helix/monitoring/mbeans/ZkClientPathMonitor.java      | 10 +++++-----
 6 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/pom.xml
----------------------------------------------------------------------
diff --git a/helix-core/pom.xml b/helix-core/pom.xml
index d833627..4855d7c 100644
--- a/helix-core/pom.xml
+++ b/helix-core/pom.xml
@@ -157,7 +157,7 @@ under the License.
     <dependency>
       <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
-      <version>3.1.2</version>
+      <version>3.2.3</version>
     </dependency>
   </dependencies>
   <build>

http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
index 8c77466..25b7894 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
@@ -20,7 +20,7 @@ package org.apache.helix.monitoring.mbeans;
  */
 
 import com.codahale.metrics.Histogram;
-import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
@@ -57,7 +57,7 @@ public class ClusterEventMonitor extends DynamicMBeanProvider {
     _clusterStatusMonitor = clusterStatusMonitor;
 
     _duration = new HistogramDynamicMetric("DurationGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _count = new SimpleDynamicMetric("EventCounter", 0l);
     _maxDuration = new SimpleDynamicMetric("MaxSingleDurationGauge", 0l);
     _totalDuration = new SimpleDynamicMetric("TotalDurationCounter", 0l);
@@ -69,7 +69,7 @@ public class ClusterEventMonitor extends DynamicMBeanProvider {
     _clusterStatusMonitor = clusterStatusMonitor;
 
     _duration = new HistogramDynamicMetric("DurationGauge", new Histogram(
-        new SlidingTimeWindowReservoir(histogramTimeWindowMs, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(histogramTimeWindowMs, TimeUnit.MILLISECONDS)));
     _count = new SimpleDynamicMetric("EventCounter", 0l);
     _maxDuration = new SimpleDynamicMetric("MaxSingleDurationGauge", 0l);
     _totalDuration = new SimpleDynamicMetric("TotalDurationCounter", 0l);

http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
index 0fc2001..4c595da 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
@@ -20,7 +20,7 @@ package org.apache.helix.monitoring.mbeans;
  */
 
 import com.codahale.metrics.Histogram;
-import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.InstanceType;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
@@ -64,7 +64,7 @@ public class HelixCallbackMonitor extends DynamicMBeanProvider {
             changeType.name());
 
     _latencyGauge = new HistogramDynamicMetric("LatencyGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _totalLatencyCounter = new SimpleDynamicMetric("LatencyCounter", 0l);
     _unbatchedCounter = new SimpleDynamicMetric("UnbatchedCounter", 0l);
     _counter = new SimpleDynamicMetric("Counter", 0l);

http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
index dac5826..a496eab 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
@@ -20,7 +20,7 @@ package org.apache.helix.monitoring.mbeans;
  */
 
 import com.codahale.metrics.Histogram;
-import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.model.Message;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
@@ -49,7 +49,7 @@ public class MessageLatencyMonitor extends DynamicMBeanProvider {
         "MessageLatency");
 
     _messageLatencyGauge = new HistogramDynamicMetric("MessagelatencyGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _totalMessageLatency = new SimpleDynamicMetric("TotalMessageLatency", 0l);
     _totalMessageCount = new SimpleDynamicMetric("TotalMessageCount", 0l);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
index 662f323..7c8382e 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
@@ -20,7 +20,7 @@ package org.apache.helix.monitoring.mbeans;
  */
 
 import com.codahale.metrics.Histogram;
-import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.model.ExternalView;
@@ -118,7 +118,7 @@ public class ResourceMonitor extends DynamicMBeanProvider {
 
     _partitionTopStateHandoffDurationGauge =
         new HistogramDynamicMetric("PartitionTopStateHandoffDurationGauge", new Histogram(
-            new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+            new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _totalMessageReceived = new SimpleDynamicMetric("TotalMessageReceived", 0l);
     _maxSinglePartitionTopStateHandoffDuration =
         new SimpleDynamicMetric("MaxSinglePartitionTopStateHandoffDurationGauge", 0l);

http://git-wip-us.apache.org/repos/asf/helix/blob/143420c4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
index bc6a36b..2c06eb0 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
@@ -20,7 +20,7 @@ package org.apache.helix.monitoring.mbeans;
  */
 
 import com.codahale.metrics.Histogram;
-import com.codahale.metrics.SlidingTimeWindowReservoir;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
 import java.util.concurrent.TimeUnit;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
@@ -104,13 +104,13 @@ public class ZkClientPathMonitor extends DynamicMBeanProvider {
     _readCounter = new SimpleDynamicMetric("ReadCounter", 0l);
 
     _readLatencyGauge = new HistogramDynamicMetric("ReadLatencyGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _writeLatencyGauge = new HistogramDynamicMetric("WriteLatencyGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _readBytesGauge = new HistogramDynamicMetric("ReadBytesGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
     _writeBytesGauge = new HistogramDynamicMetric("WriteBytesGauge", new Histogram(
-        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+        new SlidingTimeWindowArrayReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
   }
 
   public ZkClientPathMonitor register() throws JMException {


[04/50] [abbrv] helix git commit: Add throttling to prevent too many workflows/jobs created.

Posted by jx...@apache.org.
Add throttling to prevent too many workflows/jobs created.

ZK has issue that a large amount of nodes in one path will prevent getChildNames to be return successfully.
This change is a workaround to minimize the problem before ZK service side is ready.


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

Branch: refs/heads/master
Commit: 51170220b6be851d0bdc4c3977f0420259c4a8e9
Parents: ac39b39
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Fri Nov 3 15:54:04 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:25 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/helix/task/TaskDriver.java  | 48 +++++++----
 .../integration/task/WorkflowGenerator.java     | 16 ++++
 .../helix/task/TestTaskCreateThrottling.java    | 88 ++++++++++++++++++++
 3 files changed, 136 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/51170220/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
index 732a717..99fa761 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java
@@ -19,23 +19,8 @@ package org.apache.helix.task;
  * under the License.
  */
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
 import org.I0Itec.zkclient.DataUpdater;
-import org.apache.helix.AccessOption;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyPathBuilder;
-import org.apache.helix.ZNRecord;
+import org.apache.helix.*;
 import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -49,6 +34,8 @@ import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.*;
+
 /**
  * CLI for scheduling/canceling workflows
  */
@@ -70,11 +57,23 @@ public class TaskDriver {
   /** Default time out for monitoring workflow or job state */
   private final static int _defaultTimeout = 3 * 60 * 1000; /* 3 mins */
 
+  // HELIX-619 This is a temporary solution for too many ZK nodes issue.
+  // Limit workflows/jobs creation to prevent the problem.
+  //
+  // Note this limitation should be smaller than ZK capacity. If current nodes count already exceeds
+  // the CAP, the verification method will not throw exception since the getChildNames() call will
+  // return empty list.
+  //
+  // TODO Implement or configure the limitation in ZK server.
+  private final static int DEFAULT_CONFIGS_LIMITATION = 10000;
+  protected int _configsLimitation = DEFAULT_CONFIGS_LIMITATION;
+
   private final HelixDataAccessor _accessor;
   private final HelixPropertyStore<ZNRecord> _propertyStore;
   private final HelixAdmin _admin;
   private final String _clusterName;
 
+
   public TaskDriver(HelixManager manager) {
     this(manager.getClusterManagmentTool(), manager.getHelixDataAccessor(),
         manager.getHelixPropertyStore(), manager.getClusterName());
@@ -114,6 +113,8 @@ public class TaskDriver {
     LOG.info("Starting workflow " + flow.getName());
     flow.validate();
 
+    validateZKNodeLimitation(flow.getJobConfigs().keySet().size() + 1);
+
     WorkflowConfig newWorkflowConfig =
         new WorkflowConfig.Builder(flow.getWorkflowConfig()).setWorkflowId(flow.getName()).build();
 
@@ -324,6 +325,8 @@ public class TaskDriver {
       }
     }
 
+    validateZKNodeLimitation(1);
+
     // Create the job to ensure that it validates
     JobConfig jobConfig = jobBuilder.setWorkflow(queue).build();
     final String namespacedJobName = TaskUtil.getNamespacedJobName(queue, job);
@@ -786,4 +789,17 @@ public class TaskDriver {
       throws InterruptedException {
     return pollForJobState(workflowName, jobName, _defaultTimeout, states);
   }
+
+  /**
+   * Throw Exception if children nodes will exceed limitation after adding newNodesCount children.
+   * @param newConfigNodeCount
+   */
+  private void validateZKNodeLimitation(int newConfigNodeCount) {
+    List<String> resourceConfigs =
+        _accessor.getChildNames(_accessor.keyBuilder().resourceConfigs());
+    if (resourceConfigs.size() + newConfigNodeCount > _configsLimitation) {
+      throw new HelixException(
+          "Cannot create more workflows or jobs because there are already too many items created in the path CONFIGS.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/51170220/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
index 38797d3..5db0431 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/task/WorkflowGenerator.java
@@ -76,4 +76,20 @@ public class WorkflowGenerator {
 
     return builder;
   }
+
+  public static Workflow.Builder generateDefaultRepeatedJobWorkflowBuilder(String workflowName, int jobCount) {
+    Workflow.Builder builder = new Workflow.Builder(workflowName);
+    JobConfig.Builder jobBuilder = JobConfig.Builder.fromMap(DEFAULT_JOB_CONFIG);
+    jobBuilder.setJobCommandConfigMap(DEFAULT_COMMAND_CONFIG);
+
+    builder.addJob(JOB_NAME_1, jobBuilder);
+
+    for (int i = 0; i < jobCount - 1; i++) {
+      String jobName = JOB_NAME_2 + "-" + i;
+      builder.addParentChildDependency(JOB_NAME_1, jobName);
+      builder.addJob(jobName, jobBuilder);
+    }
+
+    return builder;
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/51170220/helix-core/src/test/java/org/apache/helix/task/TestTaskCreateThrottling.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/task/TestTaskCreateThrottling.java b/helix-core/src/test/java/org/apache/helix/task/TestTaskCreateThrottling.java
new file mode 100644
index 0000000..3cb1605
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/task/TestTaskCreateThrottling.java
@@ -0,0 +1,88 @@
+package org.apache.helix.task;
+
+/*
+ * 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.HelixException;
+import org.apache.helix.integration.task.MockTask;
+import org.apache.helix.integration.task.TaskTestBase;
+import org.apache.helix.integration.task.TaskTestUtil;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestTaskCreateThrottling extends TaskTestBase {
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    setSingleTestEnvironment();
+    super.beforeClass();
+    _driver._configsLimitation = 10;
+  }
+
+  @Test
+  public void testTaskCreatingThrottle() {
+    Workflow flow = WorkflowGenerator
+        .generateDefaultRepeatedJobWorkflowBuilder("hugeWorkflow", _driver._configsLimitation + 1)
+        .build();
+    try {
+      _driver.start(flow);
+      Assert.fail("Creating a huge workflow contains more jobs than expected should fail.");
+    } catch (HelixException e) {
+      // expected
+    }
+  }
+
+  @Test(dependsOnMethods = { "testTaskCreatingThrottle" })
+  public void testEnqueueJobsThrottle() throws InterruptedException {
+    List<String> jobs = new ArrayList<>();
+    // Use a short name for testing
+    JobQueue.Builder builder = TaskTestUtil.buildJobQueue("Q");
+    builder.setCapacity(Integer.MAX_VALUE);
+    JobConfig.Builder jobBuilder =
+        new JobConfig.Builder().setTargetResource(WorkflowGenerator.DEFAULT_TGT_DB)
+            .setCommand(MockTask.TASK_COMMAND).setMaxAttemptsPerTask(2)
+            .setJobCommandConfigMap(WorkflowGenerator.DEFAULT_COMMAND_CONFIG).setExpiry(1L);
+    for (int i = 0; i < _driver._configsLimitation - 5; i++) {
+      builder.enqueueJob("J" + i, jobBuilder);
+      jobs.add("J" + i);
+    }
+    JobQueue jobQueue = builder.build();
+    // check if large number of jobs smaller than the threshold is OK.
+    _driver.start(jobQueue);
+    _driver.stop(jobQueue.getName());
+    try {
+      for (int i = 0; i < _driver._configsLimitation; i++) {
+        _driver.enqueueJob(jobQueue.getName(), "EJ" + i, jobBuilder);
+        jobs.add("EJ" + i);
+      }
+      Assert.fail("Enqueuing a huge number of jobs should fail.");
+    } catch (HelixException e) {
+      // expected
+    }
+
+    for (String job : jobs) {
+      _driver.deleteJob(jobQueue.getName(), job);
+    }
+    _driver.delete(jobQueue.getName());
+  }
+}


[49/50] [abbrv] helix git commit: Add instance support for CriteriaEvalutator

Posted by jx...@apache.org.
Add instance support for CriteriaEvalutator


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

Branch: refs/heads/master
Commit: 4e7eca5171f509bc3669ad018beef1e149eca6a5
Parents: 77b09c3
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Jan 11 10:38:16 2018 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:34 2018 -0800

----------------------------------------------------------------------
 helix-core/src/main/java/org/apache/helix/PropertyKey.java   | 8 ++++++++
 .../java/org/apache/helix/messaging/CriteriaEvaluator.java   | 2 +-
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4e7eca51/helix-core/src/main/java/org/apache/helix/PropertyKey.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
index 4e58a89..4129671 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -322,6 +322,14 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with specified instance
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey instance(String instanceName) {
+      return new PropertyKey(PropertyType.INSTANCES, null, _clusterName, instanceName);
+    }
+
+    /**
      * Get a property key associated with {@link Message} for an instance
      * @param instanceName
      * @return {@link PropertyKey}

http://git-wip-us.apache.org/repos/asf/helix/blob/4e7eca51/helix-core/src/main/java/org/apache/helix/messaging/CriteriaEvaluator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/CriteriaEvaluator.java b/helix-core/src/main/java/org/apache/helix/messaging/CriteriaEvaluator.java
index cd6aed6..ec7b0fd 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/CriteriaEvaluator.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/CriteriaEvaluator.java
@@ -41,7 +41,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public class CriteriaEvaluator {
-  private static Logger logger = Logger.getLogger(CriteriaEvaluator.class);
+  private static Logger logger = LoggerFactory.getLogger(CriteriaEvaluator.class);
   public static final String MATCH_ALL_SYM = "%";
 
   /**


[42/50] [abbrv] helix git commit: [helix-front] Upgrade Angular to 5; Material to 5; ngx-datatable to 11

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/package.json
----------------------------------------------------------------------
diff --git a/helix-front/package.json b/helix-front/package.json
index 567d39c..cde017f 100644
--- a/helix-front/package.json
+++ b/helix-front/package.json
@@ -1,31 +1,32 @@
 {
   "name": "helix-front",
-  "version": "1.0.0",
+  "version": "1.1.0",
   "author": "Vivo Xu <vx...@linkedin.com>",
   "license": "Apache-2.0",
   "private": true,
   "scripts": {
     "ng": "ng",
     "build": "rm -rf dist && mkdir dist && ng build -aot -prod && tsc -p server",
-    "start": "concurrently -r \"ng serve -pc proxy.conf.json\" \"tsc -w -p server\" \"nodemon dist/server/app.js\"",
+    "start": "tsc -p server && concurrently -r \"ng serve -pc proxy.conf.json\" \"tsc -w -p server\" \"nodemon dist/server/app.js\"",
     "prod": "npm run build && node dist/server/app.js",
     "test": "ng test",
     "lint": "ng lint",
     "e2e": "ng e2e"
   },
   "dependencies": {
-    "@angular/animations": "^4.3.2",
-    "@angular/common": "^4.3.2",
-    "@angular/compiler": "^4.3.2",
-    "@angular/core": "^4.3.2",
-    "@angular/flex-layout": "2.0.0-beta.8",
-    "@angular/forms": "^4.3.2",
-    "@angular/http": "^4.3.2",
-    "@angular/material": "2.0.0-beta.6",
-    "@angular/platform-browser": "^4.3.2",
-    "@angular/platform-browser-dynamic": "^4.3.2",
-    "@angular/router": "^4.3.2",
-    "@swimlane/ngx-datatable": "^9.3.0",
+    "@angular/animations": "^5.1.1",
+    "@angular/cdk": "^5.0.1",
+    "@angular/common": "^5.1.1",
+    "@angular/compiler": "^5.1.1",
+    "@angular/core": "^5.1.1",
+    "@angular/flex-layout": "^2.0.0-beta.12",
+    "@angular/forms": "^5.1.1",
+    "@angular/http": "^5.1.1",
+    "@angular/material": "^5.0.1",
+    "@angular/platform-browser": "^5.1.1",
+    "@angular/platform-browser-dynamic": "^5.1.1",
+    "@angular/router": "^5.1.1",
+    "@swimlane/ngx-datatable": "^11.1.7",
     "angulartics2": "^2.2.1",
     "body-parser": "^1.17.2",
     "core-js": "^2.4.1",
@@ -35,22 +36,22 @@
     "hammerjs": "^2.0.8",
     "lodash": "^4.17.4",
     "morgan": "^1.8.2",
-    "ngx-clipboard": "^8.0.2",
+    "ngx-clipboard": "^9.0.0",
     "ngx-json-viewer": "^1.0.0",
     "node-sass": "4.5.0",
     "request": "^2.81.0",
-    "rxjs": "^5.1.0",
+    "rxjs": "^5.5.5",
     "zone.js": "^0.8.4"
   },
   "devDependencies": {
-    "@angular/cli": "^1.2.0",
-    "@angular/compiler-cli": "^4.3.2",
+    "@angular/cli": "^1.6.1",
+    "@angular/compiler-cli": "^5.1.1",
     "@types/hammerjs": "^2.0.34",
     "@types/jasmine": "2.5.38",
     "@types/lodash": "^4.14.71",
     "@types/node": "~6.0.60",
     "@types/request": "^2.0.0",
-    "codelyzer": "~2.0.0",
+    "codelyzer": "^4.0.2",
     "concurrently": "^3.5.0",
     "jasmine-core": "~2.5.2",
     "jasmine-spec-reporter": "~3.2.0",
@@ -64,7 +65,7 @@
     "nodemon": "1.11.0",
     "protractor": "~5.1.0",
     "ts-node": "~2.0.0",
-    "tslint": "~4.5.0",
-    "typescript": "~2.4.0"
+    "tslint": "^5.8.0",
+    "typescript": "2.4.2"
   }
 }


[47/50] [abbrv] helix git commit: Implement of CRUSH-ed algorithm.

Posted by jx...@apache.org.
Implement of CRUSH-ed algorithm.

The algorithm is based on CRUSH. The new implementation trade-off between uniform distribution and partition movements during major cluster changes.
Please refer to https://iwww.corp.linkedin.com/wiki/cf/display/ENGS/CRUSH-ed+for+even+distribution


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

Branch: refs/heads/master
Commit: 9dfb098ed6083609f6bbd17cd202b65ce26e8c7c
Parents: fde1a6a
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Mon Nov 20 14:06:49 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:24 2018 -0800

----------------------------------------------------------------------
 ...stractEvenDistributionRebalanceStrategy.java | 208 +++++
 .../strategy/CrushEdRebalanceStrategy.java      |  32 +
 .../CardDealingAdjustmentAlgorithm.java         | 212 ++++++
 .../ConsistentHashingAdjustmentAlgorithm.java   | 136 ++++
 .../java/org/apache/helix/DistributionTest.java | 753 +++++++++++++++++++
 .../TestCrushAutoRebalance.java                 |   7 +-
 .../TestCrushAutoRebalanceNonRack.java          |  95 ++-
 7 files changed, 1400 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
new file mode 100644
index 0000000..9012f73
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
@@ -0,0 +1,208 @@
+package org.apache.helix.controller.rebalancer.strategy;
+
+/*
+ * 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.HelixException;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.rebalancer.strategy.crushMapping.CardDealingAdjustmentAlgorithm;
+import org.apache.helix.controller.rebalancer.strategy.crushMapping.ConsistentHashingAdjustmentAlgorithm;
+import org.apache.helix.controller.rebalancer.topology.Topology;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * Abstract class of Forced Even Assignment Patched Algorithm.
+ * This class contains common logic that re-calculate assignment based on a result calculated by the base algorithm.
+ * The target of this patching step is more even partition distribution, but number of partitions to be reshuffled during node outage could be higher than the base algorithm.
+ */
+public abstract class AbstractEvenDistributionRebalanceStrategy implements RebalanceStrategy {
+  private static final Logger _logger =
+      LoggerFactory.getLogger(AbstractEvenDistributionRebalanceStrategy.class);
+  private String _resourceName;
+  private int _replica;
+
+  protected abstract RebalanceStrategy getBaseRebalanceStrategy();
+
+  @Override
+  public void init(String resourceName, final List<String> partitions,
+      final LinkedHashMap<String, Integer> states, int maximumPerNode) {
+    _resourceName = resourceName;
+    getBaseRebalanceStrategy().init(resourceName, partitions, states, maximumPerNode);
+    _replica = countStateReplicas(states);
+  }
+
+  /**
+   * Force uniform distribution based on the parent strategy class's calculation result.
+   *
+   * @param allNodes       All instances
+   * @param liveNodes      List of live instances
+   * @param currentMapping current replica mapping
+   * @param clusterData    cluster data
+   * @return
+   * @throws HelixException
+   */
+  @Override
+  public ZNRecord computePartitionAssignment(final List<String> allNodes,
+      final List<String> liveNodes, final Map<String, Map<String, String>> currentMapping,
+      ClusterDataCache clusterData) throws HelixException {
+    boolean continueNextStep = true;
+    // Round 1: Calculate mapping using the base strategy.
+    // Note to use all nodes for minimizing the influence of live node changes to mapping.
+    ZNRecord origAssignment = getBaseRebalanceStrategy()
+        .computePartitionAssignment(allNodes, allNodes, currentMapping, clusterData);
+    Map<String, List<String>> origPartitionMap = origAssignment.getListFields();
+    // If the original calculation contains no assignment, skip patching
+    if (origPartitionMap.isEmpty()) {
+      continueNextStep = false;
+    }
+
+    // Transform current assignment to instance->partitions map, and get total partitions
+    Map<String, List<String>> nodeToPartitionMap = convertMap(origPartitionMap);
+
+    if (continueNextStep) {
+      // Round 2: Rebalance mapping using card dealing algorithm. For ensuring evenness distribution.
+      Topology allNodeTopo = new Topology(allNodes, allNodes, clusterData.getInstanceConfigMap(),
+          clusterData.getClusterConfig());
+      CardDealingAdjustmentAlgorithm cardDealer =
+          new CardDealingAdjustmentAlgorithm(allNodeTopo, _replica);
+      continueNextStep = cardDealer.computeMapping(nodeToPartitionMap, _resourceName.hashCode());
+    }
+
+    // Round 3: Reorder preference Lists to ensure participants' orders (so as the states) are uniform.
+    Map<String, List<String>> partitionMap = shufflePreferenceList(nodeToPartitionMap);
+
+    // Round 4: Re-mapping the partitions on non-live nodes using consistent hashing for reducing movement.
+    if (continueNextStep && !liveNodes.containsAll(allNodes)) {
+      Topology liveNodeTopo = new Topology(allNodes, liveNodes, clusterData.getInstanceConfigMap(),
+          clusterData.getClusterConfig());
+      ConsistentHashingAdjustmentAlgorithm hashPlacement =
+          new ConsistentHashingAdjustmentAlgorithm(liveNodeTopo);
+      if (hashPlacement.computeMapping(nodeToPartitionMap, _resourceName.hashCode())) {
+        // Since mapping is changed by hashPlacement, need to adjust nodes order.
+        Map<String, List<String>> adjustedPartitionMap = convertMap(nodeToPartitionMap);
+        for (String partition : adjustedPartitionMap.keySet()) {
+          List<String> preSelectedList = partitionMap.get(partition);
+          Set<String> adjustedNodeList = new HashSet<>(adjustedPartitionMap.get(partition));
+          List<String> finalNodeList = adjustedPartitionMap.get(partition);
+          int index = 0;
+          // 1. Add the ones in pre-selected node list first, in order
+          for (String node : preSelectedList) {
+            if (adjustedNodeList.remove(node)) {
+              finalNodeList.set(index++, node);
+            }
+          }
+          // 2. Add the rest of nodes to the map
+          for (String node : adjustedNodeList) {
+            finalNodeList.set(index++, node);
+          }
+        }
+        partitionMap = adjustedPartitionMap;
+      } else {
+        continueNextStep = false;
+      }
+    }
+
+    if (continueNextStep) {
+      ZNRecord result = new ZNRecord(_resourceName);
+      result.setListFields(partitionMap);
+      return result;
+    } else {
+      if (_logger.isDebugEnabled()) {
+        _logger.debug("Force even distribution is not possible, using the default strategy: "
+            + getBaseRebalanceStrategy().getClass().getSimpleName());
+      }
+      // Force even is not possible, fallback to use default strategy
+      if (liveNodes.equals(allNodes)) {
+        return origAssignment;
+      } else {
+        // need to re-calculate since node list is different.
+        return getBaseRebalanceStrategy()
+            .computePartitionAssignment(allNodes, liveNodes, currentMapping, clusterData);
+      }
+    }
+  }
+
+  // Best effort to shuffle preference lists for all partitions for uniform distribution regarding the top state.
+  private Map<String, List<String>> shufflePreferenceList(
+      Map<String, List<String>> nodeToPartitionMap) {
+    final Map<String, List<String>> partitionMap = convertMap(nodeToPartitionMap);
+    // evaluate node's order according to:
+    // 1. their potential top state replicas count (less count, higher priority)
+    // 2. their assigned top state replicas (less top state replica, higher priority)
+    final Map<String, Integer> nodeScores = new HashMap<>();
+    for (String node : nodeToPartitionMap.keySet()) {
+      // Init with the potential replicas count
+      nodeScores.put(node, nodeToPartitionMap.get(node).size());
+    }
+    for (final String partition : partitionMap.keySet()) {
+      List<String> nodes = partitionMap.get(partition);
+      // order according to score
+      Collections.sort(nodes, new Comparator<String>() {
+        @Override
+        public int compare(String o1, String o2) {
+          int o1Score = nodeScores.get(o1);
+          int o2Score = nodeScores.get(o2);
+          if (o1Score == o2Score) {
+            return new Integer((partition + o1).hashCode()).compareTo((partition + o2).hashCode());
+          } else {
+            return o1Score - o2Score;
+          }
+        }
+      });
+      // After assignment, the nodes has less potential top states
+      for (int i = 0; i < nodes.size(); i++) {
+        String nodeName = nodes.get(i);
+        nodeScores.put(nodeName,
+            nodeScores.get(nodeName) - 1 + (i == 0 ? (int) Math.pow(_replica, 2) : 0));
+      }
+    }
+    return partitionMap;
+  }
+
+  // Convert the map from <key, list of values> to a new map <original value, list of related keys>
+  private Map<String, List<String>> convertMap(Map<String, List<String>> originalMap) {
+    Map<String, List<String>> resultMap = new HashMap<>();
+    for (String originalKey : originalMap.keySet()) {
+      for (String originalValue : originalMap.get(originalKey)) {
+        if (!resultMap.containsKey(originalValue)) {
+          resultMap.put(originalValue, new ArrayList<String>());
+        }
+        resultMap.get(originalValue).add(originalKey);
+      }
+    }
+    return resultMap;
+  }
+
+  /**
+   * Counts the total number of replicas given a state-count mapping
+   *
+   * @return
+   */
+  private int countStateReplicas(Map<String, Integer> stateCountMap) {
+    int total = 0;
+    for (Integer count : stateCountMap.values()) {
+      total += count;
+    }
+    return total;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/CrushEdRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/CrushEdRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/CrushEdRebalanceStrategy.java
new file mode 100644
index 0000000..a7d0c4f
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/CrushEdRebalanceStrategy.java
@@ -0,0 +1,32 @@
+package org.apache.helix.controller.rebalancer.strategy;
+
+/*
+ * 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.
+ */
+
+/**
+ * CRUSH-ed, CRUSH with even distribution. This is an Auto rebalance strategy based on CRUSH algorithm.
+ * This gives even partition distribution, but number of partitions to be reshuffled during node outage could be high.
+ */
+public class CrushEdRebalanceStrategy extends AbstractEvenDistributionRebalanceStrategy {
+  private final RebalanceStrategy _baseStrategy = new CrushRebalanceStrategy();
+
+  protected RebalanceStrategy getBaseRebalanceStrategy() {
+    return _baseStrategy;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/CardDealingAdjustmentAlgorithm.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/CardDealingAdjustmentAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/CardDealingAdjustmentAlgorithm.java
new file mode 100644
index 0000000..4470094
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/CardDealingAdjustmentAlgorithm.java
@@ -0,0 +1,212 @@
+package org.apache.helix.controller.rebalancer.strategy.crushMapping;
+
+import org.apache.helix.controller.rebalancer.topology.Node;
+import org.apache.helix.controller.rebalancer.topology.Topology;
+
+import java.util.*;
+
+public class CardDealingAdjustmentAlgorithm {
+  private static int MAX_ADJUSTMENT = 2;
+
+  private int _replica;
+  // Instance -> FaultZone Tag
+  private Map<String, String> _instanceFaultZone = new HashMap<>();
+  private Map<String, Long> _instanceWeight = new HashMap<>();
+  private long _totalWeight = 0;
+  private Map<String, Long> _faultZoneWeight = new HashMap<>();
+  // Record existing partitions that are assigned to a fault zone
+  private Map<String, Set<String>> _faultZonePartitionMap = new HashMap<>();
+
+  public CardDealingAdjustmentAlgorithm(Topology topology, int replica) {
+    _replica = replica;
+    // Get all instance related information.
+    for (Node zone : topology.getFaultZones()) {
+      _faultZoneWeight.put(zone.getName(), zone.getWeight());
+      if (!_faultZonePartitionMap.containsKey(zone.getName())) {
+        _faultZonePartitionMap.put(zone.getName(), new HashSet<String>());
+      }
+      for (Node instance : Topology.getAllLeafNodes(zone)) {
+        if (!instance.isFailed()) {
+          _instanceWeight.put(instance.getName(), instance.getWeight());
+          _totalWeight += instance.getWeight();
+          _instanceFaultZone.put(instance.getName(), zone.getName());
+        }
+      }
+    }
+  }
+
+  public boolean computeMapping(Map<String, List<String>> nodeToPartitionMap, int randomSeed) {
+    // Records exceed partitions
+    TreeMap<String, Integer> toBeReassigned = new TreeMap<>();
+
+    // Calculate total partitions that need to be calculated
+    long totalReplicaCount = 0;
+    for (List<String> partitions : nodeToPartitionMap.values()) {
+      totalReplicaCount += partitions.size();
+    }
+    if (totalReplicaCount == 0 || _replica > _faultZoneWeight.size()) {
+      return false;
+    }
+
+    // instance -> target (ideal) partition count
+    Map<String, Float> targetPartitionCount = new HashMap<>();
+    for (String liveInstance : _instanceFaultZone.keySet()) {
+      long zoneWeight = _faultZoneWeight.get(_instanceFaultZone.get(liveInstance));
+      float instanceRatioInZone = ((float) _instanceWeight.get(liveInstance)) / zoneWeight;
+      // 1. if replica = fault zone, fault zone weight does not count, so calculate according to fault zone count.
+      // 2. else, should consider fault zone weight to calculate expected threshold.
+      float zonePartitions;
+      if (_replica == _faultZoneWeight.size()) {
+        zonePartitions = ((float) totalReplicaCount) / _faultZoneWeight.size();
+      } else {
+        zonePartitions = ((float) totalReplicaCount) * zoneWeight / _totalWeight;
+      }
+      targetPartitionCount.put(liveInstance, instanceRatioInZone * zonePartitions);
+    }
+
+    // Calculate the expected spikes
+    // Assign spikes to each zone according to zone weight
+    int totalOverflows = (int) totalReplicaCount % _instanceFaultZone.size();
+    Map<String, Integer> maxZoneOverflows = new HashMap<>();
+    for (String faultZoneName : _faultZoneWeight.keySet()) {
+      float zoneWeight = _faultZoneWeight.get(faultZoneName);
+      maxZoneOverflows.put(faultZoneName,
+          (int) Math.ceil(((float) totalOverflows) * zoneWeight / _totalWeight));
+    }
+
+    Iterator<String> nodeIter = nodeToPartitionMap.keySet().iterator();
+    while (nodeIter.hasNext()) {
+      String instance = nodeIter.next();
+      // Cleanup the existing mapping. Remove all non-active nodes from the mapping
+      if (!_instanceFaultZone.containsKey(instance)) {
+        List<String> partitions = nodeToPartitionMap.get(instance);
+        addToReAssignPartition(toBeReassigned, partitions);
+        partitions.clear();
+        nodeIter.remove();
+      }
+    }
+
+    List<String> orderedInstances = new ArrayList<>(_instanceFaultZone.keySet());
+    // Different resource should shuffle nodes in different ways.
+    Collections.shuffle(orderedInstances, new Random(randomSeed));
+    for (String instance : orderedInstances) {
+      if (!nodeToPartitionMap.containsKey(instance)) {
+        continue;
+      }
+      // Cut off the exceed partitions compared with target partition count.
+      List<String> partitions = nodeToPartitionMap.get(instance);
+      int target = (int) (Math.floor(targetPartitionCount.get(instance)));
+      if (partitions.size() > target) {
+        int maxZoneOverflow = maxZoneOverflows.get(_instanceFaultZone.get(instance));
+        if (maxZoneOverflow > 0 && totalOverflows > 0) {
+          // When fault zone has overflow capacity AND there are still remaining overflow partitions
+          target = (int) (Math.ceil(targetPartitionCount.get(instance)));
+          maxZoneOverflows.put(_instanceFaultZone.get(instance), maxZoneOverflow - 1);
+          totalOverflows--;
+        }
+
+        // Shuffle partitions to randomly pickup exceed ones. Ensure the algorithm generates consistent results when the inputs are the same.
+        Collections.shuffle(partitions, new Random(instance.hashCode() * 31 + randomSeed));
+        addToReAssignPartition(toBeReassigned, partitions.subList(target, partitions.size()));
+
+        // Put the remaining partitions to the assignment, and record in fault zone partition list
+        List<String> remainingPartitions = new ArrayList<>(partitions.subList(0, target));
+        partitions.clear();
+        nodeToPartitionMap.put(instance, remainingPartitions);
+      }
+      _faultZonePartitionMap.get(_instanceFaultZone.get(instance))
+          .addAll(nodeToPartitionMap.get(instance));
+    }
+
+    // Reassign if any instances have space left.
+    // Assign partition according to the target capacity, CAP at "Math.floor(target) + adjustment"
+    int adjustment = 0;
+    while (!toBeReassigned.isEmpty() && adjustment <= MAX_ADJUSTMENT) {
+      partitionDealing(_instanceFaultZone.keySet(), toBeReassigned, _faultZonePartitionMap,
+          _instanceFaultZone, nodeToPartitionMap, targetPartitionCount, randomSeed, adjustment++);
+    }
+    return toBeReassigned.isEmpty();
+  }
+
+  private void partitionDealing(Collection<String> instances,
+      TreeMap<String, Integer> toBeReassigned, Map<String, Set<String>> faultZonePartitionMap,
+      Map<String, String> faultZoneMap, final Map<String, List<String>> assignmentMap,
+      Map<String, Float> targetPartitionCount, final int randomSeed, int targetAdjustment) {
+    PriorityQueue<String> instanceQueue =
+        new PriorityQueue<>(instances.size(), new Comparator<String>() {
+          @Override
+          public int compare(String node1, String node2) {
+            int node1Load = assignmentMap.containsKey(node1) ? assignmentMap.get(node1).size() : 0;
+            int node2Load = assignmentMap.containsKey(node2) ? assignmentMap.get(node2).size() : 0;
+            if (node1Load == node2Load) {
+              return new Integer((node1 + randomSeed).hashCode())
+                  .compareTo((node2 + randomSeed).hashCode());
+            } else {
+              return node1Load - node2Load;
+            }
+          }
+        });
+    instanceQueue.addAll(instances);
+
+    while (!toBeReassigned.isEmpty()) {
+      boolean anyPartitionAssigned = false;
+      Iterator<String> instanceIter = instanceQueue.iterator();
+      while (instanceIter.hasNext()) {
+        String instance = instanceIter.next();
+        // Temporary remove the node from queue.
+        // If any partition assigned to the instance, add it back to reset priority.
+        instanceIter.remove();
+        boolean partitionAssignedToInstance = false;
+        String faultZoneStr = faultZoneMap.get(instance);
+        List<String> partitions = assignmentMap.containsKey(instance) ?
+            assignmentMap.get(instance) :
+            new ArrayList<String>();
+        int space =
+            (int) (Math.floor(targetPartitionCount.get(instance))) + targetAdjustment - partitions
+                .size();
+        if (space > 0) {
+          // Find a pending partition to locate
+          for (String pendingPartition : toBeReassigned.navigableKeySet()) {
+            if (!faultZonePartitionMap.get(faultZoneStr).contains(pendingPartition)) {
+              if (!assignmentMap.containsKey(instance)) {
+                assignmentMap.put(instance, partitions);
+              }
+              partitions.add(pendingPartition);
+              faultZonePartitionMap.get(faultZoneStr).add(pendingPartition);
+              if (toBeReassigned.get(pendingPartition) == 1) {
+                toBeReassigned.remove(pendingPartition);
+              } else {
+                toBeReassigned.put(pendingPartition, toBeReassigned.get(pendingPartition) - 1);
+              }
+              // if any assignment is made:
+              // this instance can hold more partitions in the future
+              partitionAssignedToInstance = true;
+              break;
+            }
+          }
+        }
+        if (partitionAssignedToInstance) {
+          // Reset priority in the queue
+          instanceQueue.add(instance);
+          anyPartitionAssigned = true;
+          break;
+        }
+      }
+      if (!anyPartitionAssigned) {
+        // if no pending partition is assigned to any instances in this loop, new assignment is not possible
+        break;
+      }
+    }
+  }
+
+  private void addToReAssignPartition(TreeMap<String, Integer> toBeReassigned,
+      List<String> partitions) {
+    for (String partition : partitions) {
+      if (!toBeReassigned.containsKey(partition)) {
+        toBeReassigned.put(partition, 1);
+      } else {
+        toBeReassigned.put(partition, toBeReassigned.get(partition) + 1);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
new file mode 100644
index 0000000..e594cd1
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
@@ -0,0 +1,136 @@
+package org.apache.helix.controller.rebalancer.strategy.crushMapping;
+
+import org.apache.helix.controller.rebalancer.topology.Node;
+import org.apache.helix.controller.rebalancer.topology.Topology;
+import org.apache.helix.util.JenkinsHash;
+
+import java.util.*;
+
+public class ConsistentHashingAdjustmentAlgorithm {
+  private JenkinsHash _hashFunction;
+  private ConsistentHashSelector _selector;
+  Set<String> _liveInstances = new HashSet<>();
+  // Instance -> FaultZone Tag
+  private Map<String, String> _faultZoneMap = new HashMap<>();
+  // Record existing partitions that are assigned to a fault zone
+  private Map<String, Set<String>> _faultZonePartitionMap = new HashMap<>();
+
+  public ConsistentHashingAdjustmentAlgorithm(Topology topology) {
+    _hashFunction = new JenkinsHash();
+    List<String> allInstances = new ArrayList<>();
+    // Get all instance related information.
+    for (Node zone : topology.getFaultZones()) {
+      for (Node instance : Topology.getAllLeafNodes(zone)) {
+        if (!instance.isFailed()) {
+          _liveInstances.add(instance.getName());
+        }
+        allInstances.add(instance.getName());
+        _faultZoneMap.put(instance.getName(), zone.getName());
+        if (!_faultZonePartitionMap.containsKey(zone.getName())) {
+          _faultZonePartitionMap.put(zone.getName(), new HashSet<String>());
+        }
+      }
+    }
+    _selector = new ConsistentHashSelector(allInstances);
+  }
+
+  public boolean computeMapping(Map<String, List<String>> nodeToPartitionMap, int randomSeed) {
+    if (_liveInstances.isEmpty()) {
+      return false;
+    }
+
+    Set<String> inactiveInstances = new HashSet<>();
+    Map<String, Integer> toBeReassigned = new HashMap<>();
+    // Remove all partition assignment to a non-live instance
+    Iterator<String> nodeIter = nodeToPartitionMap.keySet().iterator();
+    while (nodeIter.hasNext()) {
+      String instance = nodeIter.next();
+      List<String> partitions = nodeToPartitionMap.get(instance);
+      if (!_liveInstances.contains(instance)) {
+        inactiveInstances.add(instance);
+        addToReAssignPartition(toBeReassigned, partitions);
+        partitions.clear();
+        nodeIter.remove();
+      } else {
+        _faultZonePartitionMap.get(_faultZoneMap.get(instance)).addAll(partitions);
+      }
+    }
+
+    for (String partition : new ArrayList<>(toBeReassigned.keySet())) {
+      int remainReplicas = toBeReassigned.get(partition);
+      Set<String> conflictInstance = new HashSet<>();
+      for (int index = 0; index < toBeReassigned.get(partition); index++) {
+        Iterable<String> sortedInstances = _selector.getCircle(_hashFunction.hash(randomSeed, partition.hashCode(), index));
+        Iterator<String> instanceItr = sortedInstances.iterator();
+        while (instanceItr.hasNext() && conflictInstance.size() + inactiveInstances.size() != _selector.instanceSize) {
+          String instance = instanceItr.next();
+          if (!_liveInstances.contains(instance)) {
+            inactiveInstances.add(instance);
+          }
+          if (inactiveInstances.contains(instance) || conflictInstance.contains(instance)) {
+            continue;
+          }
+          Set<String> faultZonePartitions = _faultZonePartitionMap.get(_faultZoneMap.get(instance));
+          if (faultZonePartitions.contains(partition)) {
+            conflictInstance.add(instance);
+            continue;
+          }
+          // insert this assignment
+          if (!nodeToPartitionMap.containsKey(instance)) {
+            nodeToPartitionMap.put(instance, new ArrayList<String>());
+          }
+          nodeToPartitionMap.get(instance).add(partition);
+          faultZonePartitions.add(partition);
+          remainReplicas--;
+          break;
+        }
+      }
+      if (remainReplicas == 0) {
+        toBeReassigned.remove(partition);
+      } else {
+        toBeReassigned.put(partition, remainReplicas);
+      }
+    }
+
+    return toBeReassigned.isEmpty();
+  }
+
+  private void addToReAssignPartition(Map<String, Integer> toBeReassigned,
+      List<String> partitions) {
+    for (String partition : partitions) {
+      if (!toBeReassigned.containsKey(partition)) {
+        toBeReassigned.put(partition, 1);
+      } else {
+        toBeReassigned.put(partition, toBeReassigned.get(partition) + 1);
+      }
+    }
+  }
+
+  private class ConsistentHashSelector {
+    private final static int DEFAULT_TOKENS_PER_INSTANCE = 1000;
+    private final SortedMap<Long, String> circle = new TreeMap<Long, String>();
+    protected int instanceSize = 0;
+
+    public ConsistentHashSelector(List<String> instances) {
+      for (String instance : instances) {
+        long tokenCount = DEFAULT_TOKENS_PER_INSTANCE;
+        add(instance, tokenCount);
+        instanceSize++;
+      }
+    }
+
+    private void add(String instance, long numberOfReplicas) {
+      for (int i = 0; i < numberOfReplicas; i++) {
+        circle.put(_hashFunction.hash(instance.hashCode(), i), instance);
+      }
+    }
+
+    public Iterable<String> getCircle(long data) {
+      if (circle.isEmpty()) {
+        return null;
+      }
+      long hash = _hashFunction.hash(data);
+      return circle.tailMap(hash).values();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/test/java/org/apache/helix/DistributionTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/DistributionTest.java b/helix-core/src/test/java/org/apache/helix/DistributionTest.java
new file mode 100644
index 0000000..bf580d1
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/DistributionTest.java
@@ -0,0 +1,753 @@
+package org.apache.helix;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
+import org.apache.helix.controller.rebalancer.topology.Node;
+import org.apache.helix.controller.rebalancer.topology.Topology;
+import org.apache.helix.manager.zk.ZNRecordSerializer;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.util.HelixUtil;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.*;
+
+public class DistributionTest {
+  private static String instanceFolderPath;
+  private static String instanceList;
+  private static String idealStateFolderPath;
+  private static String idealStateList;
+
+  String Path = "/home/jjwang/Desktop/FEAP-test";
+  //String Path = "/Users/jjwang/Desktop/FEAP-test";
+
+  @DataProvider(name = "rebalanceStrategies")
+  public static String[][] rebalanceStrategies() {
+    return new String[][] {
+        //{AutoRebalanceStrategy.class.getName()},
+        { CrushRebalanceStrategy.class.getName() },
+        //{ MultiRoundCrushRebalanceStrategy.class.getName() },
+        //{ CrushEdRebalanceStrategy.class.getName() }
+    };
+  }
+
+  String[] fabrics = { "lor1", "lva1", "ltx1", "lsg1",
+  };
+  String[] clusters = { "ESPRESSO_IDENTITY", "ESPRESSO_MT-MD-1", "ESPRESSO_TSCP", "ESPRESSO_MT_PHASE1",
+          "ESPRESSO_MT-MD-3", "ESPRESSO_USCP", "ESPRESSO_MT-LEGACY", /* "venice-0" */
+  };
+  String topState = "master";
+  float[] nodeAdjustSimulator =
+      { /*-0.5f, -0.2f, -0.1f, -0.01f, */ 0.01f, 0.1f, 0.2f, 0.5f, 1f};
+
+  @Test(dataProvider = "rebalanceStrategies")
+  public void testNodeChange(String rebalanceStrategyClass) throws Exception {
+    for (String cluster : clusters) {
+      System.out.println(cluster
+          + "\tChangeType\tNumOfNodeChange\tDiffRate\tTotalMv\tTotalMvRate\tExtraMvRate\tExtraMvRateComparedWithAvgDist\tTopStateChange\tTopStateChangeRate\tTopStateChangeWithNewDeployRate\tExtraTopStateChangeRate");
+      for (String fabric : fabrics) {
+        String path = Path + "/" + cluster + "/" + fabric;
+        if (new File(path).exists()) {
+          System.out.print(fabric);
+          for (float adjustRate : nodeAdjustSimulator) {
+            Set<String> deltaNode = new HashSet<>();
+            List<String> liveInstances = new ArrayList<>();
+            Map<String, Map<String, String>> resultA =
+                calculate(path, rebalanceStrategyClass, adjustRate, deltaNode, liveInstances);
+            double[] distEval = checkEvenness(liveInstances, resultA, false);
+            if (adjustRate != 0) {
+              Map<String, Map<String, String>> result =
+                  calculate(path, rebalanceStrategyClass, 0, deltaNode, new ArrayList<String>());
+              double[] diff = checkMovement(result, resultA, deltaNode, false);
+              System.out.println(
+                  "\t" + (adjustRate > 0 ? "Adding\t" : "Disabling\t") + diff[0] + "\t"
+                      + distEval[3] + "\t" + diff[1] + "\t" + diff[2] + "\t" + diff[3] + "\t"
+                      + diff[8] + "\t" + diff[4] + "\t" + diff[5] + "\t" + diff[10] + "\t"
+                      + diff[6]);
+            }
+          }
+        }
+      }
+      System.out.println();
+    }
+  }
+
+  @Test(dataProvider = "rebalanceStrategies")
+  public void testDist(String rebalanceStrategyClass) throws Exception {
+    for (String cluster : clusters) {
+      System.out.println(cluster
+          + "\tTotalReplica\tMinReplica\tMaxReplica\tDiffRate\tSTDEV\tMinTopState\tMaxTopState\ttopStateDiffRate\ttopStateSTDEV");
+      for (String fabric : fabrics) {
+        String path = Path + "/" + cluster + "/" + fabric;
+        if (new File(path).exists()) {
+          Set<String> deltaNode = new HashSet<>();
+          List<String> liveInstances = new ArrayList<>();
+          Map<String, Map<String, String>> result =
+              calculate(path, rebalanceStrategyClass, 0, deltaNode, liveInstances);
+          double[] distEval = checkEvenness(liveInstances, result, false);
+          System.out.println(
+              fabric + "\t" + distEval[0] + "\t" + distEval[1] + "\t" + distEval[2] + "\t"
+                  + distEval[3] + "\t" + distEval[4] + "\t" + distEval[5] + "\t" + distEval[6]
+                  + "\t" + distEval[7] + "\t" + distEval[8]);
+        }
+      }
+      System.out.println();
+    }
+  }
+
+  int _replica = 1;
+  int partitionCount = 101;
+  int faultZone = 10;
+  int[] resourceCounts = new int[] { 100 };
+  int[] nodeCounts = new int[] { 100, /*100, 200, 500, 1000*/ };
+
+  @Test(dataProvider = "rebalanceStrategies")
+  public void testDistUsingRandomTopo(String rebalanceStrategyClass) throws Exception {
+    for (int nodeCount : nodeCounts) {
+      for (int resourceCount : resourceCounts) {
+        System.out.println(
+            "NodeCount\tResourceCount\tTotalReplica\tMinReplica\tMaxReplica\tDiffRate\tSTDEV\tMinTopState\tMaxTopState\tTopStateDiffRate\tTopStateSTDEV");
+        List<String> liveInstances = new ArrayList<>();
+        Map<String, Map<String, String>> result =
+            calculateUsingRandomTopo(rebalanceStrategyClass, _replica, partitionCount,
+                resourceCount, nodeCount, faultZone, liveInstances);
+        double[] distEval = checkEvenness(liveInstances, result, false);
+        System.out.println(
+            nodeCount + "\t" + resourceCount + "\t" + distEval[0] + "\t" + distEval[1] + "\t"
+                + distEval[2] + "\t" + distEval[3] + "\t" + distEval[4] + "\t" + distEval[5] + "\t"
+                + distEval[6] + "\t" + distEval[7] + "\t" + distEval[8]);
+      }
+    }
+
+    System.out.println();
+  }
+
+  @Test(dataProvider = "rebalanceStrategies")
+  public void testRollingUpgrade(String rebalanceStrategyClass) throws Exception {
+    for (String cluster : clusters) {
+      System.out.println(cluster
+          + "\tTotalMv\tTotalMvRate\tExtraMvRate\tExtraMvRateComparedWithAvgDist\tTopStateChange\tTopStateChangeRate\tTopStateChangeWithNewDeployRate\tExtraTopStateChange");
+      for (String fabric : fabrics) {
+        String path = Path + "/" + cluster + "/" + fabric;
+        if (new File(path).exists()) {
+
+          List<List<String>> deltaNodesHistory = new ArrayList<>();
+          List<List<String>> liveInstancesHistory = new ArrayList<>();
+
+          List<Map<String, Map<String, String>>> mappingHistory =
+              calculateRollingUpgrade(path, rebalanceStrategyClass, deltaNodesHistory,
+                  liveInstancesHistory, true);
+
+          Map<String, Map<String, String>> basicMapping =
+              calculate(path, rebalanceStrategyClass, 0, new HashSet<String>(),
+                  new ArrayList<String>());
+
+          double[] maxDiff = new double[8];
+          for (int i = 0; i < mappingHistory.size(); i++) {
+            List<String> deltaNode = deltaNodesHistory.get(i);
+            Map<String, Map<String, String>> mapA = mappingHistory.get(i);
+
+            Map<String, Map<String, String>> mapB = basicMapping;
+            if (i != 0) {
+              deltaNode.addAll(deltaNodesHistory.get(i - 1));
+              mapB = mappingHistory.get(i - 1);
+            }
+            double[] diff = checkMovement(mapB, mapA, deltaNode, false);
+
+            maxDiff[0] = Math.max(diff[1], maxDiff[0]);
+            maxDiff[1] = Math.max(diff[2], maxDiff[1]);
+            maxDiff[2] = Math.max(diff[3], maxDiff[2]);
+            maxDiff[3] = Math.max(diff[4], maxDiff[3]);
+            maxDiff[4] = Math.max(diff[5], maxDiff[4]);
+            maxDiff[5] = Math.max(diff[6], maxDiff[5]);
+            maxDiff[6] = Math.max(diff[8], maxDiff[6]);
+            maxDiff[7] = Math.max(diff[10], maxDiff[7]);
+          }
+          System.out.println(
+              fabric + "\t" + maxDiff[0] + "\t" + maxDiff[1] + "\t" + maxDiff[2] + "\t" + maxDiff[6]
+                  + "\t" + maxDiff[3] + "\t" + maxDiff[4] + "\t" + maxDiff[7] + "\t" + maxDiff[5]);
+        }
+      }
+      System.out.println();
+    }
+  }
+
+  public List<Map<String, Map<String, String>>> calculateRollingUpgrade(String Path,
+      String rebalanceStrategyClass, List<List<String>> deltaNodesHistory,
+      List<List<String>> liveInstancesHistory, boolean recoverNode) throws Exception {
+    instanceFolderPath = Path + "/instanceConfigs/";
+    instanceList = Path + "/instance";
+    idealStateFolderPath = Path + "/idealStates/";
+    idealStateList = Path + "/idealstate";
+    Path path = Paths.get(Path + "/clusterConfig");
+    ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
+    ClusterConfig clusterConfig = new ClusterConfig(record);
+    List<String> allNodes = new ArrayList<>();
+    List<InstanceConfig> instanceConfigs =
+        getInstanceConfigs(instanceFolderPath, instanceList, allNodes);
+    List<IdealState> idealStates = getIdealStates(idealStateFolderPath, idealStateList);
+
+    List<String> deltaNodes = new ArrayList<>();
+
+    List<Map<String, Map<String, String>>> totalMapHistory = new ArrayList<>();
+    for (String downNode : allNodes) {
+      deltaNodes.add(downNode);
+
+      List<String> liveInstances = new ArrayList<>(allNodes);
+      liveInstances.removeAll(deltaNodes);
+      Map<String, Map<String, String>> totalMaps = new HashMap<>();
+
+      totalMapHistory.add(totalMaps);
+      liveInstancesHistory.add(liveInstances);
+      deltaNodesHistory.add(new ArrayList<>(deltaNodes));
+
+      Map<String, Integer> partitions = new HashMap<>();
+      for (int i = 0; i < idealStates.size(); i++) {
+        Map<String, Map<String, String>> maps = HelixUtil
+            .getIdealAssignmentForFullAuto(clusterConfig, instanceConfigs, liveInstances,
+                idealStates.get(i), new ArrayList<>(idealStates.get(i).getPartitionSet()),
+                rebalanceStrategyClass);
+        for (String partitionName : idealStates.get(i).getPartitionSet()) {
+          partitions.put(partitionName, idealStates.get(i).getReplicaCount(liveInstances.size()));
+        }
+        totalMaps.putAll(maps);
+      }
+      Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+      for (InstanceConfig config : instanceConfigs) {
+        instanceConfigMap.put(config.getInstanceName(), config);
+      }
+      verifyDistribution(totalMaps, liveInstances, partitions,
+          new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances,
+              instanceConfigMap, clusterConfig));
+      if (recoverNode) {
+        deltaNodes.remove(downNode);
+      }
+    }
+    return totalMapHistory;
+  }
+
+  public Map<String, Map<String, String>> calculateUsingRandomTopo(String rebalanceStrategyClass,
+      int replica, int partitionCount, int nodeCount, int resourceCount, int faultZone,
+      List<String> liveInstances) throws Exception {
+    String[] className = rebalanceStrategyClass.split("\\.");
+    String PARTICIPANT_PREFIX =
+        className[className.length - 1] + "_node_" + nodeCount + resourceCount;
+    String RESOURCE_PREFIX =
+        className[className.length - 1] + "_resource_" + nodeCount + resourceCount;
+    String CLUSTER_NAME =
+        className[className.length - 1] + nodeCount + resourceCount + "TestingCluster";
+
+    ClusterConfig clusterConfig = new ClusterConfig(CLUSTER_NAME);
+    clusterConfig.setTopologyAwareEnabled(true);
+    clusterConfig.setFaultZoneType("zone");
+    clusterConfig.setTopology("/zone/rack/instance");
+
+    List<InstanceConfig> newInstanceConfigs = new ArrayList<>();
+    Random rand = new Random();
+    for (int i = 0; i < nodeCount; i++) {
+      String nodeName = PARTICIPANT_PREFIX + Math.abs(rand.nextInt()) + "_" + i;
+      String zone = "zone-" + i % faultZone;
+      InstanceConfig newConfig = new InstanceConfig(nodeName);
+      liveInstances.add(nodeName);
+      newConfig.setInstanceEnabled(true);
+      newConfig.setHostName(nodeName);
+      newConfig.setPort(new Integer(i).toString());
+      newConfig.setDomain(String
+          .format("cluster=%s,zone=%s,rack=myRack,instance=%s", CLUSTER_NAME, zone, nodeName));
+      newConfig.setWeight(1000);
+      newConfig.setDelayRebalanceEnabled(false);
+      newConfig.setMaxConcurrentTask(1000);
+      newInstanceConfigs.add(newConfig);
+    }
+
+    Map<String, Map<String, String>> totalMaps = new HashMap<>();
+    Map<String, Integer> partitions = new HashMap<>();
+    List<IdealState> idealStates = new ArrayList<>();
+
+    for (int i = 0; i < resourceCount; i++) {
+      String resourceName = RESOURCE_PREFIX + "_" + i;
+      IdealState idealState = new IdealState(resourceName);
+      idealState.setStateModelDefRef(BuiltInStateModelDefinitions.MasterSlave.name());
+      idealState.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO);
+      idealState.setReplicas(new Integer(replica).toString());
+      idealState.setNumPartitions(partitionCount);
+      idealState.setRebalancerClassName(rebalanceStrategyClass);
+      for (int p = 0; p < partitionCount; p++) {
+        String partitionName = resourceName + "_" + p;
+        idealState.setPreferenceList(partitionName, new ArrayList<String>());
+      }
+      idealStates.add(idealState);
+    }
+
+    long duration = 0;
+    for (IdealState idealState : idealStates) {
+      long startTime = System.currentTimeMillis();
+      Map<String, Map<String, String>> maps = HelixUtil
+          .getIdealAssignmentForFullAuto(clusterConfig, newInstanceConfigs, liveInstances,
+              idealState, new ArrayList<>(idealState.getPartitionSet()), rebalanceStrategyClass);
+      duration += System.currentTimeMillis() - startTime;
+
+      for (String partitionName : idealState.getPartitionSet()) {
+        partitions.put(partitionName, idealState.getReplicaCount(liveInstances.size()));
+      }
+      totalMaps.putAll(maps);
+    }
+
+    //System.out.println("Total running time:\t" + duration);
+
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    for (InstanceConfig config : newInstanceConfigs) {
+      instanceConfigMap.put(config.getInstanceName(), config);
+    }
+    verifyDistribution(totalMaps, liveInstances, partitions,
+        new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances, instanceConfigMap,
+            clusterConfig));
+    return totalMaps;
+  }
+
+  public Map<String, Map<String, String>> calculate(String Path, String rebalanceStrategyClass,
+      float instanceAdjustRate, Set<String> deltaNode, List<String> liveInstances)
+      throws Exception {
+    instanceFolderPath = Path + "/instanceConfigs/";
+    instanceList = Path + "/instance";
+    idealStateFolderPath = Path + "/idealStates/";
+    idealStateList = Path + "/idealstate";
+    Path path = Paths.get(Path + "/clusterConfig");
+    ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
+    ClusterConfig clusterConfig = new ClusterConfig(record);
+    List<InstanceConfig> instanceConfigs =
+        getInstanceConfigs(instanceFolderPath, instanceList, liveInstances);
+
+    int adjustNodeCount = (int) (instanceAdjustRate > 0 ?
+        Math.ceil(instanceAdjustRate * liveInstances.size()) :
+        Math.floor(instanceAdjustRate * liveInstances.size()));
+
+    if (adjustNodeCount > 0) {
+      for (int i = 0; i < adjustNodeCount; i++) {
+        int cloneIndex = i % (liveInstances.size() - 1);
+        String nodeName = instanceConfigs.get(cloneIndex).getInstanceName() + "_random" + i;
+        liveInstances.add(nodeName);
+        InstanceConfig cloneConfig = new InstanceConfig(nodeName);
+        cloneConfig.setHostName(nodeName);
+        cloneConfig.setInstanceEnabled(true);
+        cloneConfig.setPort(instanceConfigs.get(cloneIndex).getPort());
+        cloneConfig.setDomain(instanceConfigs.get(cloneIndex).getDomain() + "_random" + i);
+        if (instanceConfigs.get(cloneIndex).getWeight() > 0) {
+          cloneConfig.setWeight(instanceConfigs.get(cloneIndex).getWeight());
+        }
+        cloneConfig
+            .setDelayRebalanceEnabled(instanceConfigs.get(cloneIndex).isDelayRebalanceEnabled());
+        if (instanceConfigs.get(cloneIndex).getMaxConcurrentTask() > 0) {
+          cloneConfig.setMaxConcurrentTask(instanceConfigs.get(cloneIndex).getMaxConcurrentTask());
+        }
+        instanceConfigs.add(cloneConfig);
+        deltaNode.add(nodeName);
+      }
+    } else {
+      if (adjustNodeCount > liveInstances.size()) {
+        throw new Exception("All nodes are removed, no assignment possible.");
+      }
+      for (int i = 0; i < Math.abs(adjustNodeCount); i++) {
+        String nodeName = liveInstances.remove(i);
+        deltaNode.add(nodeName);
+      }
+    }
+
+    List<IdealState> idealStates = getIdealStates(idealStateFolderPath, idealStateList);
+    Map<String, Map<String, String>> totalMaps = new HashMap<>();
+    Map<String, Integer> partitions = new HashMap<>();
+
+    long duration = 0;
+    for (int i = 0; i < idealStates.size(); i++) {
+      long startTime = System.currentTimeMillis();
+      int partitionCount = idealStates.get(i).getNumPartitions();
+      List<String> partitionList =
+          new ArrayList<>(idealStates.get(i).getPartitionSet()).subList(0, partitionCount);
+      Map<String, Map<String, String>> maps = HelixUtil
+          .getIdealAssignmentForFullAuto(clusterConfig, instanceConfigs, liveInstances,
+              idealStates.get(i), partitionList, rebalanceStrategyClass);
+      for (String partitionName : partitionList) {
+        partitions.put(partitionName, idealStates.get(i).getReplicaCount(liveInstances.size()));
+      }
+      duration += System.currentTimeMillis() - startTime;
+
+      // print resource details
+/*      Map<String, Set<String>> nodeMapping = convertMapping(maps);
+      String partitionCountsStr = idealStates.get(i).getResourceName();
+      List<String> sortedInstances = new ArrayList<>(liveInstances);
+      Collections.sort(sortedInstances);
+      for (String node : sortedInstances) {
+        partitionCountsStr += "\t" + (nodeMapping.containsKey(node) ? nodeMapping.get(node).size() : 0);
+      }
+      System.out.println(partitionCountsStr);*/
+
+      totalMaps.putAll(maps);
+    }
+    //System.out.println("Takes " + duration + "ms");
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    for (InstanceConfig config : instanceConfigs) {
+      instanceConfigMap.put(config.getInstanceName(), config);
+    }
+    verifyDistribution(totalMaps, liveInstances, partitions,
+        new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances, instanceConfigMap,
+            clusterConfig));
+    return totalMaps;
+  }
+
+  private void verifyDistribution(Map<String, Map<String, String>> map, List<String> liveInstances,
+      Map<String, Integer> partitionExp, Topology topology) throws Exception {
+    Map<String, Set<String>> faultZonePartition = new HashMap<>();
+    Map<String, String> instanceFaultZone = new HashMap<>();
+    for (Node node : topology.getFaultZones()) {
+      faultZonePartition.put(node.getName(), new HashSet<String>());
+      for (Node instance : Topology.getAllLeafNodes(node)) {
+        instanceFaultZone.put(instance.getName(), node.getName());
+      }
+    }
+    for (String partition : map.keySet()) {
+      // no partition missing, no partition duplicate
+      if (!partitionExp.containsKey(partition) || map.get(partition).size() != partitionExp
+          .get(partition)) {
+        throw new Exception("partition replica in mapping is not as expected");
+      }
+      partitionExp.remove(partition);
+      // no partition on non-live node
+      for (String instance : map.get(partition).keySet()) {
+        if (!liveInstances.contains(instance)) {
+          throw new Exception("assignment is not on a live node!");
+        }
+        // no fault zone conflict
+        String faultZone = instanceFaultZone.get(instance);
+        if (faultZonePartition.get(faultZone).contains(partition)) {
+          throw new Exception("faultzone conflict!");
+        }
+        faultZonePartition.get(faultZone).add(partition);
+      }
+    }
+    if (!partitionExp.isEmpty()) {
+      throw new Exception("partition is not assigned");
+    }
+  }
+
+  private double[] checkEvenness(List<String> liveInstances,
+      Map<String, Map<String, String>> totalMaps, boolean verbose) {
+    StringBuilder output = new StringBuilder();
+    Map<String, List<String>> detailMap = new HashMap<>();
+    Map<String, Integer> distributionMap = new TreeMap<>();
+    Map<String, Integer> topStateDistributionMap = new HashMap<>();
+    for (String instance : liveInstances) {
+      distributionMap.put(instance, 0);
+      topStateDistributionMap.put(instance, 0);
+      detailMap.put(instance, new ArrayList<String>());
+    }
+
+    for (String partition : totalMaps.keySet()) {
+      Map<String, String> instanceMap = totalMaps.get(partition);
+      for (String instance : instanceMap.keySet()) {
+        detailMap.get(instance).add(partition + "-" + totalMaps.get(partition).get(instance));
+        distributionMap.put(instance, distributionMap.get(instance) + 1);
+        if (instanceMap.get(instance).equalsIgnoreCase(topState)) {
+          topStateDistributionMap.put(instance, topStateDistributionMap.get(instance) + 1);
+        }
+      }
+    }
+
+    int totalReplicas = 0;
+    int minR = Integer.MAX_VALUE;
+    int maxR = 0;
+    int mminR = Integer.MAX_VALUE;
+    int mmaxR = 0;
+    for (String instance : distributionMap.keySet()) {
+      output.append(instance + "\t" + distributionMap.get(instance) + "\tpartitions\t"
+          + topStateDistributionMap.get(instance) + "\ttopStates\n");
+      //output.append(instance + "\t:\t" + distributionMap.get(instance) + "\tpartitions\t" + topStateDistributionMap.get(instance) + "\ttopStates\t" + detailMap.get(instance) + "\n");
+      totalReplicas += distributionMap.get(instance);
+      minR = Math.min(minR, distributionMap.get(instance));
+      maxR = Math.max(maxR, distributionMap.get(instance));
+    }
+    for (String instance : topStateDistributionMap.keySet()) {
+      mminR = Math.min(mminR, topStateDistributionMap.get(instance));
+      mmaxR = Math.max(mmaxR, topStateDistributionMap.get(instance));
+    }
+
+    output.append("Maximum holds " + maxR + " replicas and minimum holds " + minR
+        + " replicas, differentiation is " + (double) (maxR - minR) / maxR * 100 + "%\n");
+    output.append("Maximum holds " + mmaxR + " topStates and minimum holds " + mminR
+        + " topStates, differentiation is " + (double) (mmaxR - mminR) / mmaxR * 100 + "%\n ");
+
+    if (verbose) {
+      System.out.println(output.toString());
+    }
+    return new double[] { totalReplicas, minR, maxR, (double) (maxR - minR) / maxR * 100,
+        STDEV(new ArrayList<>(distributionMap.values())), mminR, mmaxR,
+        (double) (mmaxR - mminR) / mmaxR * 100,
+        STDEV(new ArrayList<>(topStateDistributionMap.values()))
+    };
+  }
+
+  private double STDEV(List<Integer> data) {
+    if (data.isEmpty() || data.size() == 1) {
+      return 0;
+    }
+    double totalDiff = 0;
+    double average = 0;
+    for (int num : data) {
+      average += num;
+    }
+    average /= data.size();
+    for (int i = 0; i < data.size(); i++) {
+      totalDiff += Math.pow(data.get(i) - average, 2);
+    }
+    return Math.sqrt(totalDiff) / (data.size() - 1);
+  }
+
+  private static List<InstanceConfig> getInstanceConfigs(String instanceFolderPath,
+      String instanceList, List<String> liveInstances) throws IOException {
+    List<InstanceConfig> instanceConfigs = new ArrayList<>();
+    for (ZNRecord record : getRecords(instanceFolderPath, instanceList)) {
+      instanceConfigs.add(new InstanceConfig(record));
+      liveInstances.add(record.getId());
+    }
+    return instanceConfigs;
+  }
+
+  private static List<IdealState> getIdealStates(String idealStateFolderPath, String idealStateList)
+      throws IOException {
+    List<IdealState> idealStates = new ArrayList<>();
+    for (ZNRecord record : getRecords(idealStateFolderPath, idealStateList)) {
+      IdealState idealState = new IdealState(record);
+      try {
+        BuiltInStateModelDefinitions.valueOf(idealState.getStateModelDefRef());
+      } catch (IllegalArgumentException ex) {
+        idealState.setStateModelDefRef("OnlineOffline");
+      }
+      idealStates.add(idealState);
+    }
+    return idealStates;
+  }
+
+  private static List<ZNRecord> getRecords(String folderPath, String list) throws IOException {
+    List<ZNRecord> records = new ArrayList<>();
+    List<String> names = new ArrayList<>();
+    try (BufferedReader br = new BufferedReader(new FileReader(list))) {
+      String sCurrentLine = br.readLine();
+      names.addAll(Arrays.asList(sCurrentLine.split(" ")));
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    for (String name : names) {
+      Path path = Paths.get(folderPath + name);
+      ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
+      records.add(record);
+    }
+    return records;
+  }
+
+  private Map<String, Map<String, Integer>> getStateCount(Map<String, Map<String, String>> map) {
+    Map<String, Map<String, Integer>> mapStateCount = new HashMap<>();
+    for (String partition : map.keySet()) {
+      Map<String, Integer> stateCount = new HashMap<>();
+      mapStateCount.put(partition, stateCount);
+      for (String node : map.get(partition).keySet()) {
+        String state = map.get(partition).get(node);
+        if (!stateCount.containsKey(state)) {
+          stateCount.put(state, 1);
+        } else {
+          stateCount.put(state, stateCount.get(state) + 1);
+        }
+      }
+    }
+    return mapStateCount;
+  }
+
+  private void verifyMaps(Map<String, Map<String, String>> map,
+      Map<String, Map<String, String>> newMap) throws Exception {
+    // check no partition lose
+    Map<String, Map<String, Integer>> mapStateCount = getStateCount(map);
+    Map<String, Map<String, Integer>> newMapStateCount = getStateCount(newMap);
+    for (String partition : mapStateCount.keySet()) {
+      if (!newMapStateCount.containsKey(partition)) {
+        throw new Exception("mapping does not match");
+      }
+      for (String state : mapStateCount.get(partition).keySet()) {
+        if (!newMapStateCount.get(partition).containsKey(state)
+            || mapStateCount.get(partition).get(state) != newMapStateCount.get(partition)
+            .get(state)) {
+          throw new Exception("state does not match");
+        }
+      }
+      for (String state : newMapStateCount.get(partition).keySet()) {
+        if (!mapStateCount.get(partition).containsKey(state)) {
+          throw new Exception("state does not match");
+        }
+      }
+    }
+    for (String partition : newMapStateCount.keySet()) {
+      if (!mapStateCount.containsKey(partition)) {
+        throw new Exception("mapping does not match");
+      }
+    }
+  }
+
+  private double[] checkMovement(Map<String, Map<String, String>> map,
+      Map<String, Map<String, String>> newMap, Collection<String> deltaNodes, boolean verbose)
+      throws Exception {
+    verifyMaps(map, newMap);
+    int totalChange = 0;
+    int totalTopStateChange = 0;
+    int totalTopStateChangeWithNewDeployment = 0;
+    int totalPartition = 0;
+    int totalTopState = 0;
+
+    for (String partition : map.keySet()) {
+      Map<String, String> origStates = map.get(partition);
+      Map<String, String> newStates = newMap.get(partition);
+      String topStateNode = "", newtopStateNode = "";
+      for (String node : origStates.keySet()) {
+        if (origStates.get(node).equalsIgnoreCase(topState)) {
+          topStateNode = node;
+        }
+      }
+      for (String node : newStates.keySet()) {
+        if (newStates.get(node).equalsIgnoreCase(topState)) {
+          newtopStateNode = node;
+          totalTopState++;
+        }
+      }
+      if (!topStateNode.equalsIgnoreCase(newtopStateNode)) {
+        totalTopStateChange++;
+        if (!origStates.containsKey(newtopStateNode)) {
+          totalTopStateChangeWithNewDeployment++;
+        }
+      }
+    }
+
+    Map<String, Set<String>> list = convertMapping(map);
+    Map<String, Set<String>> newList = convertMapping(newMap);
+
+    Map<String, Integer> addition = new HashMap<>();
+    Map<String, Integer> subtraction = new HashMap<>();
+    for (String instance : newList.keySet()) {
+      Set<String> oldPartitions = list.get(instance);
+      Set<String> newPartitions = newList.get(instance);
+      totalPartition += newPartitions.size();
+      if (oldPartitions == null) {
+        addition.put(instance, newPartitions.size());
+      } else {
+        Set<String> commonPartitions = new HashSet<>(newPartitions);
+        commonPartitions.retainAll(oldPartitions);
+
+        newPartitions.removeAll(commonPartitions);
+
+        addition.put(instance, newPartitions.size());
+
+        oldPartitions.removeAll(commonPartitions);
+        subtraction.put(instance, oldPartitions.size());
+      }
+      totalChange += newPartitions.size();
+      //System.out.println("Changed partition on node: \t" + instance + "\t: \t" + newPartitions.toString());
+    }
+    /*
+      List<String> instances = new ArrayList<>(newList.keySet());
+      Collections.sort(instances);
+      System.out.println("Addition partition count: ");
+      for (String instance : instances) {
+        System.out.println(addition.containsKey(instance) ? addition.get(instance) : 0);
+      }
+
+      System.out.println("Subtraction partition count: ");
+      for (String instance : instances) {
+        System.out.println(subtraction.containsKey(instance) ? subtraction.get(instance) : 0);
+      }
+    */
+
+    int nodeChanged = 0;
+    int necessaryChange = 0;
+    int necessarytopStateChange = 0;
+    for (String instance : deltaNodes) {
+      nodeChanged++;
+      if (list.containsKey(instance)) {
+        necessaryChange += list.get(instance).size();
+        for (Map<String, String> nodeState : map.values()) {
+          if (nodeState.containsKey(instance)) {
+            if (nodeState.get(instance).equalsIgnoreCase(topState)) {
+              necessarytopStateChange++;
+            }
+          }
+        }
+      }
+      if (newList.containsKey(instance)) {
+        necessaryChange += newList.get(instance).size();
+        for (Map<String, String> nodeState : newMap.values()) {
+          if (nodeState.containsKey(instance)) {
+            if (nodeState.get(instance).equalsIgnoreCase(topState)) {
+              necessarytopStateChange++;
+            }
+          }
+        }
+      }
+    }
+
+    if (verbose) {
+      System.out.println(
+          "\t\t\t" + "Total partition change count: \t" + totalChange + "\t/\t" + totalPartition
+              + "\t, rate: \t" + (((float) totalChange) / totalPartition * 100) + "%\t"
+              + "Diff nodes have partition \t" + necessaryChange + "\t, unnecessary change rate: \t"
+              + (((float) totalChange - necessaryChange) / totalPartition * 100)
+              + "%\t, which is \t" + (((float) totalChange - necessaryChange) / totalChange * 100)
+              + "%\t of the movement.");
+    }
+
+    double expectedAverageMv =
+        (double) totalPartition / Math.max(list.size(), newList.size()) * deltaNodes.size();
+
+    return new double[] { nodeChanged, totalChange, (((double) totalChange) / totalPartition * 100),
+        (((double) totalChange - necessaryChange) / totalPartition * 100), totalTopStateChange,
+        (((double) totalTopStateChange) / totalTopState * 100),
+        (((double) totalTopStateChange - necessarytopStateChange) / totalTopState * 100),
+        expectedAverageMv, (((double) totalChange - expectedAverageMv) / totalPartition * 100),
+        totalTopStateChangeWithNewDeployment,
+        (((double) totalTopStateChangeWithNewDeployment) / totalTopState * 100)
+    };
+  }
+
+  private Map<String, Set<String>> convertMapping(Map<String, Map<String, String>> map) {
+    Map<String, Set<String>> list = new HashMap<>();
+    for (String partition : map.keySet()) {
+      for (String instance : map.get(partition).keySet()) {
+        if (!list.containsKey(instance)) {
+          list.put(instance, new HashSet<String>());
+        }
+        list.get(instance).add(partition);
+      }
+    }
+    return list;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
index d886e44..08608f3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
@@ -26,8 +26,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
-import org.apache.helix.controller.rebalancer.strategy.MultiRoundCrushRebalanceStrategy;
+
+import org.apache.helix.controller.rebalancer.strategy.*;
 import org.apache.helix.integration.common.ZkIntegrationTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
@@ -111,7 +111,8 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
   @DataProvider(name = "rebalanceStrategies")
   public static Object [][] rebalanceStrategies() {
     return new String[][] { {"CrushRebalanceStrategy", CrushRebalanceStrategy.class.getName()},
-        {"MultiRoundCrushRebalanceStrategy", MultiRoundCrushRebalanceStrategy.class.getName()}
+        {"MultiRoundCrushRebalanceStrategy", MultiRoundCrushRebalanceStrategy.class.getName()},
+        {"CrushEdRebalanceStrategy", CrushEdRebalanceStrategy.class.getName()}
     };
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/9dfb098e/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
index c11ee87..85059b6 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalanceNonRack.java
@@ -19,33 +19,25 @@ package org.apache.helix.integration.rebalancer.CrushRebalancers;
  * under the License.
  */
 
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import org.apache.helix.ConfigAccessor;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
 import org.apache.helix.integration.common.ZkStandAloneCMTestBase;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.model.BuiltInStateModelDefinitions;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
+import org.apache.helix.model.*;
 import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.tools.ClusterSetup;
-import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
+import java.util.*;
+
 public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
   final int NUM_NODE = 6;
   protected static final int START_PORT = 12918;
@@ -59,7 +51,7 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
   List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
   Map<String, String> _nodeToTagMap = new HashMap<String, String>();
   List<String> _nodes = new ArrayList<String>();
-  Set<String> _allDBs = new HashSet<String>();
+  Set<String> _allDBs = new HashSet<>();
   int _replica = 3;
 
   private static String[] _testModels = { BuiltInStateModelDefinitions.OnlineOffline.name(),
@@ -91,7 +83,8 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
       String tag = "tag-" + i % 2;
       _setupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
       _nodeToTagMap.put(storageNodeName, tag);
-      InstanceConfig instanceConfig = configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
+      InstanceConfig instanceConfig =
+          configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
       instanceConfig.setDomain("instance=" + storageNodeName);
       configAccessor.setInstanceConfig(CLUSTER_NAME, storageNodeName, instanceConfig);
     }
@@ -112,13 +105,15 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     //enableTopologyAwareRebalance(_gZkClient, CLUSTER_NAME, true);
   }
 
-  @DataProvider(name = "rebalanceStrategies") public static String[][] rebalanceStrategies() {
-    return new String[][] { { "CrushRebalanceStrategy", CrushRebalanceStrategy.class.getName() } };
+  @DataProvider(name = "rebalanceStrategies")
+  public static String[][] rebalanceStrategies() {
+    return new String[][] { { "CrushRebalanceStrategy", CrushRebalanceStrategy.class.getName() },
+        {"CrushEdRebalanceStrategy", CrushEdRebalanceStrategy.class.getName()}
+    };
   }
 
   @Test(dataProvider = "rebalanceStrategies", enabled = true)
-  public void test(String rebalanceStrategyName, String rebalanceStrategyClass)
-      throws Exception {
+  public void test(String rebalanceStrategyName, String rebalanceStrategyClass) throws Exception {
     System.out.println("Test " + rebalanceStrategyName);
     int i = 0;
     for (String stateModel : _testModels) {
@@ -131,10 +126,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     Thread.sleep(300);
 
     HelixClusterVerifier _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
-
     for (String db : _allDBs) {
       IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
@@ -161,8 +155,8 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     }
     Thread.sleep(300);
 
-      HelixClusterVerifier _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+    HelixClusterVerifier _clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
     for (String db : _allDBs) {
@@ -173,11 +167,12 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     }
   }
 
-  @Test(dataProvider = "rebalanceStrategies", enabled = true, dependsOnMethods = { "test",
-      "testWithInstanceTag"})
+  @Test(dataProvider = "rebalanceStrategies", enabled = true, dependsOnMethods = {
+      "testWithInstanceTag"
+  })
   public void testLackEnoughLiveInstances(String rebalanceStrategyName,
       String rebalanceStrategyClass) throws Exception {
-    System.out.println("TestLackEnoughInstances " + rebalanceStrategyName);
+    System.out.println("TestLackEnoughLiveInstances " + rebalanceStrategyName);
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
     // shutdown participants, keep only two left
@@ -185,9 +180,9 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
       _participants.get(i).syncStop();
     }
 
-    int i = 0;
+    int j = 0;
     for (String stateModel : _testModels) {
-      String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
+      String db = "Test-DB-" + rebalanceStrategyName + "-" + j++;
       _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
       _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
@@ -196,22 +191,26 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     Thread.sleep(300);
 
     HelixClusterVerifier _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify(5000));
-
     for (String db : _allDBs) {
       IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
           _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, 2);
     }
+
+    for (int i = 2; i < _participants.size(); i++) {
+      _participants.get(i).syncStart();
+    }
   }
 
-  @Test(dataProvider = "rebalanceStrategies", enabled = true, dependsOnMethods = { "test",
-      "testWithInstanceTag"})
-  public void testLackEnoughInstances(String rebalanceStrategyName,
-      String rebalanceStrategyClass) throws Exception {
+  @Test(dataProvider = "rebalanceStrategies", enabled = true, dependsOnMethods = {
+      "testLackEnoughLiveInstances"
+  })
+  public void testLackEnoughInstances(String rebalanceStrategyName, String rebalanceStrategyClass)
+      throws Exception {
     System.out.println("TestLackEnoughInstances " + rebalanceStrategyName);
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
@@ -225,27 +224,42 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
       _setupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
     }
 
-    int i = 0;
+    int j = 0;
     for (String stateModel : _testModels) {
-      String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
+      String db = "Test-DB-" + rebalanceStrategyName + "-" + j++;
       _setupTool.addResourceToCluster(CLUSTER_NAME, db, _PARTITIONS, stateModel,
           RebalanceMode.FULL_AUTO + "", rebalanceStrategyClass);
       _setupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
     Thread.sleep(300);
-
     HelixClusterVerifier _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
             .setResources(_allDBs).build();
     Assert.assertTrue(_clusterVerifier.verify());
-
     for (String db : _allDBs) {
       IdealState is = _setupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       ExternalView ev =
           _setupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
       validateIsolation(is, ev, 2);
     }
+
+    // recover test environment
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    for (int i = 2; i < _participants.size(); i++) {
+      String storageNodeName = _participants.get(i).getInstanceName();
+      _setupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+
+      InstanceConfig instanceConfig =
+          configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
+      instanceConfig.setDomain("instance=" + storageNodeName);
+      configAccessor.setInstanceConfig(CLUSTER_NAME, storageNodeName, instanceConfig);
+
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName);
+      participant.syncStart();
+      _participants.set(i, participant);
+    }
   }
 
   /**
@@ -267,7 +281,8 @@ public class TestCrushAutoRebalanceNonRack extends ZkStandAloneCMTestBase {
     }
   }
 
-  @AfterMethod public void afterMethod() throws Exception {
+  @AfterMethod
+  public void afterMethod() throws Exception {
     for (String db : _allDBs) {
       _setupTool.dropResourceFromCluster(CLUSTER_NAME, db);
     }


[36/50] [abbrv] helix git commit: HELIX-661: fix accidentally deleted import

Posted by jx...@apache.org.
HELIX-661: fix accidentally deleted import


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

Branch: refs/heads/master
Commit: ec7eaaae93b47f3cb5a5a3aea5513335b7349910
Parents: 40710b2
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Dec 19 09:45:44 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:42 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/helix/rest/server/TestInstanceAccessor.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ec7eaaae/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 763f95b..ce2bec9 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import javax.ws.rs.client.Entity;
@@ -135,7 +136,7 @@ public class TestInstanceAccessor extends AbstractTestClass {
         Response.Status.OK.getStatusCode());
     Assert.assertEquals(_configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME).getTags(),
         ImmutableList.of("tag2"));
-    
+
     // TODO: Reenable the test after storage node fix the problem
     // Batch disable instances
     /*


[18/50] [abbrv] helix git commit: REST API support cluster maintenance mode enable disable

Posted by jx...@apache.org.
REST API support cluster maintenance mode enable disable


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

Branch: refs/heads/master
Commit: 3cfe7850dee324a531caeb2e5a8852ce13879aeb
Parents: ae53705
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Nov 15 13:09:39 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:37 2018 -0800

----------------------------------------------------------------------
 .../rest/server/resources/AbstractResource.java |  2 ++
 .../rest/server/resources/ClusterAccessor.java  | 24 ++++++++++++++++++--
 .../helix/rest/server/TestClusterAccessor.java  | 21 +++++++++++++++++
 3 files changed, 45 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/3cfe7850/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index d800eb0..50f6f08 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -65,6 +65,8 @@ public class AbstractResource {
     expand,
     enable,
     disable,
+    enableMaintenanceMode,
+    disableMaintenanceMode,
     update,
     delete,
     rebalance,

http://git-wip-us.apache.org/repos/asf/helix/blob/3cfe7850/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
index 6e90400..1c998b7 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
@@ -62,6 +62,7 @@ public class ClusterAccessor extends AbstractResource {
     liveInstances,
     resources,
     paused,
+    maintenance,
     messages,
     stateModelDefinitions,
     clusters
@@ -100,6 +101,9 @@ public class ClusterAccessor extends AbstractResource {
 
     boolean paused = (dataAccessor.getProperty(keyBuilder.pause()) == null ? false : true);
     clusterInfo.put(ClusterProperties.paused.name(), paused);
+    boolean maintenance =
+        (dataAccessor.getProperty(keyBuilder.maintenance()) == null ? false : true);
+    clusterInfo.put(ClusterProperties.maintenance.name(), maintenance);
 
     List<String> idealStates = dataAccessor.getChildNames(keyBuilder.idealStates());
     clusterInfo.put(ClusterProperties.resources.name(), idealStates);
@@ -151,7 +155,8 @@ public class ClusterAccessor extends AbstractResource {
   @POST
   @Path("{clusterId}")
   public Response updateCluster(@PathParam("clusterId") String clusterId,
-      @QueryParam("command") String commandStr, @QueryParam("superCluster") String superCluster) {
+      @QueryParam("command") String commandStr, @QueryParam("superCluster") String superCluster,
+      String content) {
     Command command;
     try {
       command = getCommand(commandStr);
@@ -201,7 +206,22 @@ public class ClusterAccessor extends AbstractResource {
         return serverError(ex);
       }
       break;
-
+    case enableMaintenanceMode:
+      try {
+        helixAdmin.enableMaintenanceMode(clusterId, true, content);
+      } catch (Exception ex) {
+        _logger.error("Failed to enable maintenance mode " + clusterId);
+        return serverError(ex);
+      }
+      break;
+    case disableMaintenanceMode:
+      try {
+        helixAdmin.enableMaintenanceMode(clusterId, false);
+      } catch (Exception ex) {
+        _logger.error("Failed to disable maintenance mode " + clusterId);
+        return serverError(ex);
+      }
+      break;
     default:
       return badRequest("Unsupported command " + command);
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/3cfe7850/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
index 95d1951..b48c01b 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
@@ -31,11 +31,14 @@ import java.util.Set;
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.resources.AbstractResource.Command;
 import org.apache.helix.rest.server.resources.ClusterAccessor;
@@ -236,6 +239,24 @@ public class TestClusterAccessor extends AbstractTestClass {
     getClusterConfigFromRest(cluster);
   }
 
+  @Test(dependsOnMethods = "testGetClusterConfig")
+  public void testEnableDisableMaintenanceMode() {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String cluster = _clusters.iterator().next();
+    String reason = "Test reason";
+    HelixDataAccessor accessor = new ZKHelixDataAccessor(cluster, _baseAccessor);
+    post("clusters/" + cluster, ImmutableMap.of("command", "enableMaintenanceMode"),
+        Entity.entity(reason, MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+    MaintenanceSignal signal = accessor.getProperty(accessor.keyBuilder().maintenance());
+    Assert.assertNotNull(signal);
+    Assert.assertEquals(reason, signal.getReason());
+    post("clusters/" + cluster, ImmutableMap.of("command", "disableMaintenanceMode"),
+        Entity.entity(new String(), MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.OK.getStatusCode());
+    Assert.assertNull(accessor.getProperty(accessor.keyBuilder().maintenance()));
+  }
+
   private ClusterConfig getClusterConfigFromRest(String cluster) throws IOException {
     String body = get("clusters/" + cluster + "/configs", Response.Status.OK.getStatusCode(), true);
 


[32/50] [abbrv] helix git commit: Move zkclient from I0ITec to Helix codebase.

Posted by jx...@apache.org.
Move zkclient from I0ITec to Helix codebase.


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

Branch: refs/heads/master
Commit: 7fc03f4c3f66625ed347a130be4205ce9d9419b4
Parents: 9b9da19
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Dec 13 15:57:28 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:32 2018 -0800

----------------------------------------------------------------------
 .../org/apache/helix/manager/zk/ZkClient.java   |    3 +-
 .../helix/manager/zk/zookeeper/ZkClient.java    | 1226 ++++++++++++++++++
 .../manager/zk/zookeeper/ZkEventThread.java     |   85 ++
 3 files changed, 1313 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/7fc03f4c/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
index c9f7ccf..182c77e 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
@@ -48,8 +48,9 @@ import java.util.concurrent.Callable;
  * ZkClient jar Ideally we should commit the changes we do here to ZKClient.
  */
 
-public class ZkClient extends org.I0Itec.zkclient.ZkClient {
+public class ZkClient extends org.apache.helix.manager.zk.zookeeper.ZkClient {
   private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
+
   public static final int DEFAULT_CONNECTION_TIMEOUT = 60 * 1000;
   public static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/7fc03f4c/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
new file mode 100644
index 0000000..d26a274
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
@@ -0,0 +1,1226 @@
+/**
+ * Copyright 2010 the original author or authors.
+ * Licensed 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.
+ */
+package org.apache.helix.manager.zk.zookeeper;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.TimeUnit;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.I0Itec.zkclient.IZkChildListener;
+import org.I0Itec.zkclient.IZkConnection;
+import org.I0Itec.zkclient.IZkDataListener;
+import org.I0Itec.zkclient.IZkStateListener;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.ZkLock;
+import org.I0Itec.zkclient.exception.ZkBadVersionException;
+import org.I0Itec.zkclient.exception.ZkException;
+import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.I0Itec.zkclient.exception.ZkNoNodeException;
+import org.I0Itec.zkclient.exception.ZkNodeExistsException;
+import org.I0Itec.zkclient.exception.ZkTimeoutException;
+import org.I0Itec.zkclient.ExceptionUtil;
+import org.I0Itec.zkclient.serialize.SerializableSerializer;
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.ConnectionLossException;
+import org.apache.zookeeper.KeeperException.SessionExpiredException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.helix.manager.zk.zookeeper.ZkEventThread.ZkEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Abstracts the interaction with zookeeper and allows permanent (not just one time) watches on nodes in ZooKeeper
+ */
+public class ZkClient implements Watcher {
+  private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
+
+  protected final IZkConnection _connection;
+  protected final long operationRetryTimeoutInMillis;
+  private final Map<String, Set<IZkChildListener>> _childListener =
+      new ConcurrentHashMap<String, Set<IZkChildListener>>();
+  private final ConcurrentHashMap<String, Set<IZkDataListener>> _dataListener =
+      new ConcurrentHashMap<String, Set<IZkDataListener>>();
+  private final Set<IZkStateListener> _stateListener = new CopyOnWriteArraySet<IZkStateListener>();
+  private KeeperState _currentState;
+  private final ZkLock _zkEventLock = new ZkLock();
+  private boolean _shutdownTriggered;
+  private ZkEventThread _eventThread;
+  // TODO PVo remove this later
+  private Thread _zookeeperEventThread;
+  private ZkSerializer _zkSerializer;
+  private volatile boolean _closed;
+
+  public ZkClient(String serverstring) {
+    this(serverstring, Integer.MAX_VALUE);
+  }
+
+  public ZkClient(String zkServers, int connectionTimeout) {
+    this(new ZkConnection(zkServers), connectionTimeout);
+  }
+
+  public ZkClient(String zkServers, int sessionTimeout, int connectionTimeout) {
+    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout);
+  }
+
+  public ZkClient(String zkServers, int sessionTimeout, int connectionTimeout,
+      ZkSerializer zkSerializer) {
+    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout, zkSerializer);
+  }
+
+  /**
+   *
+   * @param zkServers
+   *            The Zookeeper servers
+   * @param sessionTimeout
+   *            The session timeout in milli seconds
+   * @param connectionTimeout
+   *            The connection timeout in milli seconds
+   * @param zkSerializer
+   *            The Zookeeper data serializer
+   * @param operationRetryTimeout
+   *            Most operations done through this {@link org.I0Itec.zkclient.ZkClient} are retried in cases like
+   *            connection loss with the Zookeeper servers. During such failures, this
+   *            <code>operationRetryTimeout</code> decides the maximum amount of time, in milli seconds, each
+   *            operation is retried. A value lesser than 0 is considered as
+   *            "retry forever until a connection has been reestablished".
+   */
+  public ZkClient(final String zkServers, final int sessionTimeout, final int connectionTimeout,
+      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
+    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout, zkSerializer,
+        operationRetryTimeout);
+  }
+
+  public ZkClient(IZkConnection connection) {
+    this(connection, Integer.MAX_VALUE);
+  }
+
+  public ZkClient(IZkConnection connection, int connectionTimeout) {
+    this(connection, connectionTimeout, new SerializableSerializer());
+  }
+
+  public ZkClient(IZkConnection zkConnection, int connectionTimeout, ZkSerializer zkSerializer) {
+    this(zkConnection, connectionTimeout, zkSerializer, -1);
+  }
+
+  /**
+   *
+   * @param zkConnection
+   *            The Zookeeper servers
+   * @param connectionTimeout
+   *            The connection timeout in milli seconds
+   * @param zkSerializer
+   *            The Zookeeper data serializer
+   * @param operationRetryTimeout
+   *            Most operations done through this {@link org.I0Itec.zkclient.ZkClient} are retried in cases like
+   *            connection loss with the Zookeeper servers. During such failures, this
+   *            <code>operationRetryTimeout</code> decides the maximum amount of time, in milli seconds, each
+   *            operation is retried. A value lesser than 0 is considered as
+   *            "retry forever until a connection has been reestablished".
+   */
+  public ZkClient(final IZkConnection zkConnection, final int connectionTimeout,
+      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
+    if (zkConnection == null) {
+      throw new NullPointerException("Zookeeper connection is null!");
+    }
+    _connection = zkConnection;
+    _zkSerializer = zkSerializer;
+    this.operationRetryTimeoutInMillis = operationRetryTimeout;
+    connect(connectionTimeout, this);
+  }
+
+  public void setZkSerializer(ZkSerializer zkSerializer) {
+    _zkSerializer = zkSerializer;
+  }
+
+  public List<String> subscribeChildChanges(String path, IZkChildListener listener) {
+    synchronized (_childListener) {
+      Set<IZkChildListener> listeners = _childListener.get(path);
+      if (listeners == null) {
+        listeners = new CopyOnWriteArraySet<IZkChildListener>();
+        _childListener.put(path, listeners);
+      }
+      listeners.add(listener);
+    }
+    return watchForChilds(path);
+  }
+
+  public void unsubscribeChildChanges(String path, IZkChildListener childListener) {
+    synchronized (_childListener) {
+      final Set<IZkChildListener> listeners = _childListener.get(path);
+      if (listeners != null) {
+        listeners.remove(childListener);
+      }
+    }
+  }
+
+  public void subscribeDataChanges(String path, IZkDataListener listener) {
+    Set<IZkDataListener> listeners;
+    synchronized (_dataListener) {
+      listeners = _dataListener.get(path);
+      if (listeners == null) {
+        listeners = new CopyOnWriteArraySet<IZkDataListener>();
+        _dataListener.put(path, listeners);
+      }
+      listeners.add(listener);
+    }
+    watchForData(path);
+    LOG.debug("Subscribed data changes for " + path);
+  }
+
+  public void unsubscribeDataChanges(String path, IZkDataListener dataListener) {
+    synchronized (_dataListener) {
+      final Set<IZkDataListener> listeners = _dataListener.get(path);
+      if (listeners != null) {
+        listeners.remove(dataListener);
+      }
+      if (listeners == null || listeners.isEmpty()) {
+        _dataListener.remove(path);
+      }
+    }
+  }
+
+  public void subscribeStateChanges(final IZkStateListener listener) {
+    synchronized (_stateListener) {
+      _stateListener.add(listener);
+    }
+  }
+
+  public void unsubscribeStateChanges(IZkStateListener stateListener) {
+    synchronized (_stateListener) {
+      _stateListener.remove(stateListener);
+    }
+  }
+
+  public void unsubscribeAll() {
+    synchronized (_childListener) {
+      _childListener.clear();
+    }
+    synchronized (_dataListener) {
+      _dataListener.clear();
+    }
+    synchronized (_stateListener) {
+      _stateListener.clear();
+    }
+  }
+
+  // </listeners>
+
+  /**
+   * Create a persistent node.
+   *
+   * @param path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createPersistent(String path)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    createPersistent(path, false);
+  }
+
+  /**
+   * Create a persistent node and set its ACLs.
+   *
+   * @param path
+   * @param createParents
+   *            if true all parent dirs are created as well and no {@link ZkNodeExistsException} is thrown in case the
+   *            path already exists
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createPersistent(String path, boolean createParents)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    createPersistent(path, createParents, ZooDefs.Ids.OPEN_ACL_UNSAFE);
+  }
+
+  /**
+   * Create a persistent node and set its ACLs.
+   *
+   * @param path
+   * @param acl
+   *            List of ACL permissions to assign to the node
+   * @param createParents
+   *            if true all parent dirs are created as well and no {@link ZkNodeExistsException} is thrown in case the
+   *            path already exists
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createPersistent(String path, boolean createParents, List<ACL> acl)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    try {
+      create(path, null, acl, CreateMode.PERSISTENT);
+    } catch (ZkNodeExistsException e) {
+      if (!createParents) {
+        throw e;
+      }
+    } catch (ZkNoNodeException e) {
+      if (!createParents) {
+        throw e;
+      }
+      String parentDir = path.substring(0, path.lastIndexOf('/'));
+      createPersistent(parentDir, createParents, acl);
+      createPersistent(path, createParents, acl);
+    }
+  }
+
+  /**
+   * Create a persistent node.
+   *
+   * @param path
+   * @param data
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createPersistent(String path, Object data)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    create(path, data, CreateMode.PERSISTENT);
+  }
+
+  /**
+   * Create a persistent node.
+   *
+   * @param path
+   * @param data
+   * @param acl
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createPersistent(String path, Object data, List<ACL> acl) {
+    create(path, data, acl, CreateMode.PERSISTENT);
+  }
+
+  /**
+   * Create a persistent, sequental node.
+   *
+   * @param path
+   * @param data
+   * @return create node's path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String createPersistentSequential(String path, Object data)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    return create(path, data, CreateMode.PERSISTENT_SEQUENTIAL);
+  }
+
+  /**
+   * Create a persistent, sequential node and set its ACL.
+   *
+   * @param path
+   * @param acl
+   * @param data
+   * @return create node's path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String createPersistentSequential(String path, Object data, List<ACL> acl)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    return create(path, data, acl, CreateMode.PERSISTENT_SEQUENTIAL);
+  }
+
+  /**
+   * Create an ephemeral node.
+   *
+   * @param path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createEphemeral(final String path)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    create(path, null, CreateMode.EPHEMERAL);
+  }
+
+  /**
+   * Create an ephemeral node and set its ACL.
+   *
+   * @param path
+   * @param acl
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createEphemeral(final String path, final List<ACL> acl)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    create(path, null, acl, CreateMode.EPHEMERAL);
+  }
+
+  /**
+   * Create a node.
+   *
+   * @param path
+   * @param data
+   * @param mode
+   * @return create node's path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String create(final String path, Object data, final CreateMode mode)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    return create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, mode);
+  }
+
+  /**
+   * Create a node with ACL.
+   *
+   * @param path
+   * @param data
+   * @param acl
+   * @param mode
+   * @return create node's path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String create(final String path, Object data, final List<ACL> acl, final CreateMode mode) {
+    if (path == null) {
+      throw new NullPointerException("Missing value for path");
+    }
+    if (acl == null || acl.size() == 0) {
+      throw new NullPointerException("Missing value for ACL");
+    }
+    final byte[] bytes = data == null ? null : serialize(data);
+
+    return retryUntilConnected(new Callable<String>() {
+      @Override public String call() throws Exception {
+        return _connection.create(path, bytes, acl, mode);
+      }
+    });
+
+  }
+
+  /**
+   * Create an ephemeral node.
+   *
+   * @param path
+   * @param data
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createEphemeral(final String path, final Object data)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    create(path, data, CreateMode.EPHEMERAL);
+  }
+
+  /**
+   * Create an ephemeral node.
+   *
+   * @param path
+   * @param data
+   * @param acl
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public void createEphemeral(final String path, final Object data, final List<ACL> acl)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    create(path, data, acl, CreateMode.EPHEMERAL);
+  }
+
+  /**
+   * Create an ephemeral, sequential node.
+   *
+   * @param path
+   * @param data
+   * @return created path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String createEphemeralSequential(final String path, final Object data)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    return create(path, data, CreateMode.EPHEMERAL_SEQUENTIAL);
+  }
+
+  /**
+   * Create an ephemeral, sequential node with ACL.
+   *
+   * @param path
+   * @param data
+   * @param acl
+   * @return created path
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs
+   */
+  public String createEphemeralSequential(final String path, final Object data, final List<ACL> acl)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    return create(path, data, acl, CreateMode.EPHEMERAL_SEQUENTIAL);
+  }
+
+  @Override public void process(WatchedEvent event) {
+    LOG.debug("Received event: " + event);
+    _zookeeperEventThread = Thread.currentThread();
+
+    boolean stateChanged = event.getPath() == null;
+    boolean znodeChanged = event.getPath() != null;
+    boolean dataChanged =
+        event.getType() == EventType.NodeDataChanged || event.getType() == EventType.NodeDeleted
+            || event.getType() == EventType.NodeCreated
+            || event.getType() == EventType.NodeChildrenChanged;
+
+    getEventLock().lock();
+    try {
+
+      // We might have to install child change event listener if a new node was created
+      if (getShutdownTrigger()) {
+        LOG.debug("ignoring event '{" + event.getType() + " | " + event.getPath()
+            + "}' since shutdown triggered");
+        return;
+      }
+      if (stateChanged) {
+        processStateChanged(event);
+      }
+      if (dataChanged) {
+        processDataOrChildChange(event);
+      }
+    } finally {
+      if (stateChanged) {
+        getEventLock().getStateChangedCondition().signalAll();
+
+        // If the session expired we have to signal all conditions, because watches might have been removed and
+        // there is no guarantee that those
+        // conditions will be signaled at all after an Expired event
+        // TODO PVo write a test for this
+        if (event.getState() == KeeperState.Expired) {
+          getEventLock().getZNodeEventCondition().signalAll();
+          getEventLock().getDataChangedCondition().signalAll();
+          // We also have to notify all listeners that something might have changed
+          fireAllEvents();
+        }
+      }
+      if (znodeChanged) {
+        getEventLock().getZNodeEventCondition().signalAll();
+      }
+      if (dataChanged) {
+        getEventLock().getDataChangedCondition().signalAll();
+      }
+      getEventLock().unlock();
+      LOG.debug("Leaving process event");
+    }
+  }
+
+  private void fireAllEvents() {
+    for (Entry<String, Set<IZkChildListener>> entry : _childListener.entrySet()) {
+      fireChildChangedEvents(entry.getKey(), entry.getValue());
+    }
+    for (Entry<String, Set<IZkDataListener>> entry : _dataListener.entrySet()) {
+      fireDataChangedEvents(entry.getKey(), entry.getValue());
+    }
+  }
+
+  public List<String> getChildren(String path) {
+    return getChildren(path, hasListeners(path));
+  }
+
+  protected List<String> getChildren(final String path, final boolean watch) {
+    return retryUntilConnected(new Callable<List<String>>() {
+      @Override public List<String> call() throws Exception {
+        return _connection.getChildren(path, watch);
+      }
+    });
+  }
+
+  /**
+   * Counts number of children for the given path.
+   *
+   * @param path
+   * @return number of children or 0 if path does not exist.
+   */
+  public int countChildren(String path) {
+    try {
+      return getChildren(path).size();
+    } catch (ZkNoNodeException e) {
+      return 0;
+    }
+  }
+
+  protected boolean exists(final String path, final boolean watch) {
+    return retryUntilConnected(new Callable<Boolean>() {
+      @Override public Boolean call() throws Exception {
+        return _connection.exists(path, watch);
+      }
+    });
+  }
+
+  public boolean exists(final String path) {
+    return exists(path, hasListeners(path));
+  }
+
+  private void processStateChanged(WatchedEvent event) {
+    LOG.info("zookeeper state changed (" + event.getState() + ")");
+    setCurrentState(event.getState());
+    if (getShutdownTrigger()) {
+      return;
+    }
+    fireStateChangedEvent(event.getState());
+    if (event.getState() == KeeperState.Expired) {
+      try {
+        reconnect();
+        fireNewSessionEvents();
+      } catch (final Exception e) {
+        LOG.info(
+            "Unable to re-establish connection. Notifying consumer of the following exception: ",
+            e);
+        fireSessionEstablishmentError(e);
+      }
+    }
+  }
+
+  private void fireNewSessionEvents() {
+    for (final IZkStateListener stateListener : _stateListener) {
+      _eventThread.send(new ZkEvent("New session event sent to " + stateListener) {
+
+        @Override public void run() throws Exception {
+          stateListener.handleNewSession();
+        }
+      });
+    }
+  }
+
+  private void fireStateChangedEvent(final KeeperState state) {
+    for (final IZkStateListener stateListener : _stateListener) {
+      _eventThread.send(new ZkEvent("State changed to " + state + " sent to " + stateListener) {
+
+        @Override public void run() throws Exception {
+          stateListener.handleStateChanged(state);
+        }
+      });
+    }
+  }
+
+  private void fireSessionEstablishmentError(final Throwable error) {
+    for (final IZkStateListener stateListener : _stateListener) {
+      _eventThread
+          .send(new ZkEvent("Session establishment error(" + error + ") sent to " + stateListener) {
+
+            @Override public void run() throws Exception {
+              stateListener.handleSessionEstablishmentError(error);
+            }
+          });
+    }
+  }
+
+  private boolean hasListeners(String path) {
+    Set<IZkDataListener> dataListeners = _dataListener.get(path);
+    if (dataListeners != null && dataListeners.size() > 0) {
+      return true;
+    }
+    Set<IZkChildListener> childListeners = _childListener.get(path);
+    if (childListeners != null && childListeners.size() > 0) {
+      return true;
+    }
+    return false;
+  }
+
+  public boolean deleteRecursive(String path) {
+    List<String> children;
+    try {
+      children = getChildren(path, false);
+    } catch (ZkNoNodeException e) {
+      return true;
+    }
+
+    for (String subPath : children) {
+      if (!deleteRecursive(path + "/" + subPath)) {
+        return false;
+      }
+    }
+
+    return delete(path);
+  }
+
+  private void processDataOrChildChange(WatchedEvent event) {
+    final String path = event.getPath();
+
+    if (event.getType() == EventType.NodeChildrenChanged || event.getType() == EventType.NodeCreated
+        || event.getType() == EventType.NodeDeleted) {
+      Set<IZkChildListener> childListeners = _childListener.get(path);
+      if (childListeners != null && !childListeners.isEmpty()) {
+        fireChildChangedEvents(path, childListeners);
+      }
+    }
+
+    if (event.getType() == EventType.NodeDataChanged || event.getType() == EventType.NodeDeleted
+        || event.getType() == EventType.NodeCreated) {
+      Set<IZkDataListener> listeners = _dataListener.get(path);
+      if (listeners != null && !listeners.isEmpty()) {
+        fireDataChangedEvents(event.getPath(), listeners);
+      }
+    }
+  }
+
+  private void fireDataChangedEvents(final String path, Set<IZkDataListener> listeners) {
+    for (final IZkDataListener listener : listeners) {
+      _eventThread.send(new ZkEvent("Data of " + path + " changed sent to " + listener) {
+
+        @Override public void run() throws Exception {
+          // reinstall watch
+          exists(path, true);
+          try {
+            Object data = readData(path, null, true);
+            listener.handleDataChange(path, data);
+          } catch (ZkNoNodeException e) {
+            listener.handleDataDeleted(path);
+          }
+        }
+      });
+    }
+  }
+
+  private void fireChildChangedEvents(final String path, Set<IZkChildListener> childListeners) {
+    try {
+      // reinstall the watch
+      for (final IZkChildListener listener : childListeners) {
+        _eventThread.send(new ZkEvent("Children of " + path + " changed sent to " + listener) {
+
+          @Override public void run() throws Exception {
+            try {
+              // if the node doesn't exist we should listen for the root node to reappear
+              exists(path);
+              List<String> children = getChildren(path);
+              listener.handleChildChange(path, children);
+            } catch (ZkNoNodeException e) {
+              listener.handleChildChange(path, null);
+            }
+          }
+        });
+      }
+    } catch (Exception e) {
+      LOG.error("Failed to fire child changed event. Unable to getChildren.  ", e);
+    }
+  }
+
+  public boolean waitUntilExists(String path, TimeUnit timeUnit, long time)
+      throws ZkInterruptedException {
+    Date timeout = new Date(System.currentTimeMillis() + timeUnit.toMillis(time));
+    LOG.debug("Waiting until znode '" + path + "' becomes available.");
+    if (exists(path)) {
+      return true;
+    }
+    acquireEventLock();
+    try {
+      while (!exists(path, true)) {
+        boolean gotSignal = getEventLock().getZNodeEventCondition().awaitUntil(timeout);
+        if (!gotSignal) {
+          return false;
+        }
+      }
+      return true;
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    } finally {
+      getEventLock().unlock();
+    }
+  }
+
+  protected Set<IZkDataListener> getDataListener(String path) {
+    return _dataListener.get(path);
+  }
+
+  public void waitUntilConnected() throws ZkInterruptedException {
+    waitUntilConnected(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
+  }
+
+  public boolean waitUntilConnected(long time, TimeUnit timeUnit) throws ZkInterruptedException {
+    return waitForKeeperState(KeeperState.SyncConnected, time, timeUnit);
+  }
+
+  public boolean waitForKeeperState(KeeperState keeperState, long time, TimeUnit timeUnit)
+      throws ZkInterruptedException {
+    if (_zookeeperEventThread != null && Thread.currentThread() == _zookeeperEventThread) {
+      throw new IllegalArgumentException("Must not be done in the zookeeper event thread.");
+    }
+    Date timeout = new Date(System.currentTimeMillis() + timeUnit.toMillis(time));
+
+    LOG.debug("Waiting for keeper state " + keeperState);
+    acquireEventLock();
+    try {
+      boolean stillWaiting = true;
+      while (_currentState != keeperState) {
+        if (!stillWaiting) {
+          return false;
+        }
+        stillWaiting = getEventLock().getStateChangedCondition().awaitUntil(timeout);
+      }
+      LOG.debug("State is " + _currentState);
+      return true;
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    } finally {
+      getEventLock().unlock();
+    }
+  }
+
+  private void acquireEventLock() {
+    try {
+      getEventLock().lockInterruptibly();
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    }
+  }
+
+  /**
+   *
+   * @param <T>
+   * @param callable
+   * @return result of Callable
+   * @throws ZkInterruptedException
+   *             if operation was interrupted, or a required reconnection got interrupted
+   * @throws IllegalArgumentException
+   *             if called from anything except the ZooKeeper event thread
+   * @throws ZkException
+   *             if any ZooKeeper exception occurred
+   * @throws RuntimeException
+   *             if any other exception occurs from invoking the Callable
+   */
+  public <T> T retryUntilConnected(Callable<T> callable)
+      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+    if (_zookeeperEventThread != null && Thread.currentThread() == _zookeeperEventThread) {
+      throw new IllegalArgumentException("Must not be done in the zookeeper event thread.");
+    }
+    final long operationStartTime = System.currentTimeMillis();
+    while (true) {
+      if (_closed) {
+        throw new IllegalStateException("ZkClient already closed!");
+      }
+      try {
+        return callable.call();
+      } catch (ConnectionLossException e) {
+        // we give the event thread some time to update the status to 'Disconnected'
+        Thread.yield();
+        waitForRetry();
+      } catch (SessionExpiredException e) {
+        // we give the event thread some time to update the status to 'Expired'
+        Thread.yield();
+        waitForRetry();
+      } catch (KeeperException e) {
+        throw ZkException.create(e);
+      } catch (InterruptedException e) {
+        throw new ZkInterruptedException(e);
+      } catch (Exception e) {
+        throw ExceptionUtil.convertToRuntimeException(e);
+      }
+      // before attempting a retry, check whether retry timeout has elapsed
+      if (this.operationRetryTimeoutInMillis > -1
+          && (System.currentTimeMillis() - operationStartTime)
+          >= this.operationRetryTimeoutInMillis) {
+        throw new ZkTimeoutException("Operation cannot be retried because of retry timeout ("
+            + this.operationRetryTimeoutInMillis + " milli seconds)");
+      }
+    }
+  }
+
+  private void waitForRetry() {
+    if (this.operationRetryTimeoutInMillis < 0) {
+      this.waitUntilConnected();
+      return;
+    }
+    this.waitUntilConnected(this.operationRetryTimeoutInMillis, TimeUnit.MILLISECONDS);
+  }
+
+  public void setCurrentState(KeeperState currentState) {
+    getEventLock().lock();
+    try {
+      _currentState = currentState;
+    } finally {
+      getEventLock().unlock();
+    }
+  }
+
+  /**
+   * Returns a mutex all zookeeper events are synchronized aginst. So in case you need to do something without getting
+   * any zookeeper event interruption synchronize against this mutex. Also all threads waiting on this mutex object
+   * will be notified on an event.
+   *
+   * @return the mutex.
+   */
+  public ZkLock getEventLock() {
+    return _zkEventLock;
+  }
+
+  public boolean delete(final String path) {
+    try {
+      retryUntilConnected(new Callable<Object>() {
+
+        @Override public Object call() throws Exception {
+          _connection.delete(path);
+          return null;
+        }
+      });
+
+      return true;
+    } catch (ZkNoNodeException e) {
+      return false;
+    }
+  }
+
+  private byte[] serialize(Object data) {
+    return _zkSerializer.serialize(data);
+  }
+
+  @SuppressWarnings("unchecked") private <T extends Object> T derializable(byte[] data) {
+    if (data == null) {
+      return null;
+    }
+    return (T) _zkSerializer.deserialize(data);
+  }
+
+  @SuppressWarnings("unchecked") public <T extends Object> T readData(String path) {
+    return (T) readData(path, false);
+  }
+
+  @SuppressWarnings("unchecked") public <T extends Object> T readData(String path,
+      boolean returnNullIfPathNotExists) {
+    T data = null;
+    try {
+      data = (T) readData(path, null);
+    } catch (ZkNoNodeException e) {
+      if (!returnNullIfPathNotExists) {
+        throw e;
+      }
+    }
+    return data;
+  }
+
+  @SuppressWarnings("unchecked") public <T extends Object> T readData(String path, Stat stat) {
+    return (T) readData(path, stat, hasListeners(path));
+  }
+
+  @SuppressWarnings("unchecked") protected <T extends Object> T readData(final String path,
+      final Stat stat, final boolean watch) {
+    byte[] data = retryUntilConnected(new Callable<byte[]>() {
+
+      @Override public byte[] call() throws Exception {
+        return _connection.readData(path, stat, watch);
+      }
+    });
+    return (T) derializable(data);
+  }
+
+  public void writeData(String path, Object object) {
+    writeData(path, object, -1);
+  }
+
+  /**
+   * Updates data of an existing znode. The current content of the znode is passed to the {@link DataUpdater} that is
+   * passed into this method, which returns the new content. The new content is only written back to ZooKeeper if
+   * nobody has modified the given znode in between. If a concurrent change has been detected the new data of the
+   * znode is passed to the updater once again until the new contents can be successfully written back to ZooKeeper.
+   *
+   * @param <T>
+   * @param path
+   *            The path of the znode.
+   * @param updater
+   *            Updater that creates the new contents.
+   */
+  @SuppressWarnings("unchecked") public <T extends Object> void updateDataSerialized(String path,
+      DataUpdater<T> updater) {
+    Stat stat = new Stat();
+    boolean retry;
+    do {
+      retry = false;
+      try {
+        T oldData = (T) readData(path, stat);
+        T newData = updater.update(oldData);
+        writeData(path, newData, stat.getVersion());
+      } catch (ZkBadVersionException e) {
+        retry = true;
+      }
+    } while (retry);
+  }
+
+  public void writeData(final String path, Object datat, final int expectedVersion) {
+    writeDataReturnStat(path, datat, expectedVersion);
+  }
+
+  public Stat writeDataReturnStat(final String path, Object datat, final int expectedVersion) {
+    final byte[] data = serialize(datat);
+    return (Stat) retryUntilConnected(new Callable<Object>() {
+
+      @Override public Object call() throws Exception {
+        Stat stat = _connection.writeDataReturnStat(path, data, expectedVersion);
+        return stat;
+      }
+    });
+  }
+
+  public void watchForData(final String path) {
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        _connection.exists(path, true);
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Installs a child watch for the given path.
+   *
+   * @param path
+   * @return the current children of the path or null if the zk node with the given path doesn't exist.
+   */
+  public List<String> watchForChilds(final String path) {
+    if (_zookeeperEventThread != null && Thread.currentThread() == _zookeeperEventThread) {
+      throw new IllegalArgumentException("Must not be done in the zookeeper event thread.");
+    }
+    return retryUntilConnected(new Callable<List<String>>() {
+      @Override public List<String> call() throws Exception {
+        exists(path, true);
+        try {
+          return getChildren(path, true);
+        } catch (ZkNoNodeException e) {
+          // ignore, the "exists" watch will listen for the parent node to appear
+        }
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Add authentication information to the connection. This will be used to identify the user and check access to
+   * nodes protected by ACLs
+   *
+   * @param scheme
+   * @param auth
+   */
+  public void addAuthInfo(final String scheme, final byte[] auth) {
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        _connection.addAuthInfo(scheme, auth);
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Connect to ZooKeeper.
+   *
+   * @param maxMsToWaitUntilConnected
+   * @param watcher
+   * @throws ZkInterruptedException
+   *             if the connection timed out due to thread interruption
+   * @throws ZkTimeoutException
+   *             if the connection timed out
+   * @throws IllegalStateException
+   *             if the connection timed out due to thread interruption
+   */
+  public void connect(final long maxMsToWaitUntilConnected, Watcher watcher)
+      throws ZkInterruptedException, ZkTimeoutException, IllegalStateException {
+    boolean started = false;
+    acquireEventLock();
+    try {
+      setShutdownTrigger(false);
+      _eventThread = new ZkEventThread(_connection.getServers());
+      _eventThread.start();
+      _connection.connect(watcher);
+
+      LOG.debug("Awaiting connection to Zookeeper server");
+      if (!waitUntilConnected(maxMsToWaitUntilConnected, TimeUnit.MILLISECONDS)) {
+        throw new ZkTimeoutException(
+            "Unable to connect to zookeeper server within timeout: " + maxMsToWaitUntilConnected);
+      }
+      started = true;
+    } finally {
+      getEventLock().unlock();
+
+      // we should close the zookeeper instance, otherwise it would keep
+      // on trying to connect
+      if (!started) {
+        close();
+      }
+    }
+  }
+
+  public long getCreationTime(String path) {
+    acquireEventLock();
+    try {
+      return _connection.getCreateTime(path);
+    } catch (KeeperException e) {
+      throw ZkException.create(e);
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    } finally {
+      getEventLock().unlock();
+    }
+  }
+
+  /**
+   * Close the client.
+   *
+   * @throws ZkInterruptedException
+   */
+  public void close() throws ZkInterruptedException {
+    if (_closed) {
+      return;
+    }
+    LOG.debug("Closing ZkClient...");
+    getEventLock().lock();
+    try {
+      setShutdownTrigger(true);
+      _eventThread.interrupt();
+      _eventThread.join(2000);
+      _connection.close();
+      _closed = true;
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    } finally {
+      getEventLock().unlock();
+    }
+    LOG.debug("Closing ZkClient...done");
+  }
+
+  private void reconnect() {
+    getEventLock().lock();
+    try {
+      _connection.close();
+      _connection.connect(this);
+    } catch (InterruptedException e) {
+      throw new ZkInterruptedException(e);
+    } finally {
+      getEventLock().unlock();
+    }
+  }
+
+  public void setShutdownTrigger(boolean triggerState) {
+    _shutdownTriggered = triggerState;
+  }
+
+  public boolean getShutdownTrigger() {
+    return _shutdownTriggered;
+  }
+
+  public int numberOfListeners() {
+    int listeners = 0;
+    for (Set<IZkChildListener> childListeners : _childListener.values()) {
+      listeners += childListeners.size();
+    }
+    for (Set<IZkDataListener> dataListeners : _dataListener.values()) {
+      listeners += dataListeners.size();
+    }
+    listeners += _stateListener.size();
+
+    return listeners;
+  }
+
+  public List<OpResult> multi(final Iterable<Op> ops) throws ZkException {
+    if (ops == null) {
+      throw new NullPointerException("ops must not be null.");
+    }
+
+    return retryUntilConnected(new Callable<List<OpResult>>() {
+
+      @Override public List<OpResult> call() throws Exception {
+        return _connection.multi(ops);
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/7fc03f4c/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
new file mode 100644
index 0000000..dcf7019
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
@@ -0,0 +1,85 @@
+/**
+ * Copyright 2010 the original author or authors.
+ * Licensed 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.
+ */
+package org.apache.helix.manager.zk.zookeeper;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * All listeners registered at the {@link ZkClient} will be notified from this event thread. This is to prevent
+ * dead-lock situations. The {@link ZkClient} pulls some information out of the {@link ZooKeeper} events to signal
+ * {@link ZkLock} conditions. Re-using the {@link ZooKeeper} event thread to also notify {@link ZkClient} listeners,
+ * would stop the ZkClient from receiving events from {@link ZooKeeper} as soon as one of the listeners blocks (because
+ * it is waiting for something). {@link ZkClient} would then for instance not be able to maintain it's connection state
+ * anymore.
+ */
+class ZkEventThread extends Thread {
+  private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
+
+  private BlockingQueue<ZkEvent> _events = new LinkedBlockingQueue<ZkEvent>();
+
+  private static AtomicInteger _eventId = new AtomicInteger(0);
+
+  public static abstract class ZkEvent {
+
+    private String _description;
+
+    public ZkEvent(String description) {
+      _description = description;
+    }
+
+    public abstract void run() throws Exception;
+
+    @Override public String toString() {
+      return "ZkEvent[" + _description + "]";
+    }
+  }
+
+  ZkEventThread(String name) {
+    setDaemon(true);
+    setName("ZkClient-EventThread-" + getId() + "-" + name);
+  }
+
+  @Override public void run() {
+    LOG.info("Starting ZkClient event thread.");
+    try {
+      while (!isInterrupted()) {
+        ZkEvent zkEvent = _events.take();
+        int eventId = _eventId.incrementAndGet();
+        LOG.debug("Delivering event #" + eventId + " " + zkEvent);
+        try {
+          zkEvent.run();
+        } catch (InterruptedException e) {
+          interrupt();
+        } catch (ZkInterruptedException e) {
+          interrupt();
+        } catch (Throwable e) {
+          LOG.error("Error handling event " + zkEvent, e);
+        }
+        LOG.debug("Delivering event #" + eventId + " done");
+      }
+    } catch (InterruptedException e) {
+      LOG.info("Terminate ZkClient event thread.");
+    }
+  }
+
+  public void send(ZkEvent event) {
+    if (!isInterrupted()) {
+      LOG.debug("New event: " + event);
+      _events.add(event);
+    }
+  }
+}


[45/50] [abbrv] helix git commit: [helix-front] Upgrade Angular to 5; Material to 5; ngx-datatable to 11

Posted by jx...@apache.org.
[helix-front] Upgrade Angular to 5; Material to 5; ngx-datatable to 11


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

Branch: refs/heads/master
Commit: b3ecd2a7f0d054c6a5b8586d65df75de1ea3049c
Parents: 52d3bb8
Author: Vivo Xu <vx...@linkedin.com>
Authored: Thu Dec 21 15:38:07 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:11 2018 -0800

----------------------------------------------------------------------
 helix-front/client/app/app.component.html       |    16 +-
 helix-front/client/app/app.component.spec.ts    |     4 +-
 helix-front/client/app/app.module.ts            |     4 -
 .../helix-list/helix-list.component.html        |    18 +-
 .../helix-list/helix-list.component.spec.ts     |     8 +-
 .../cluster-detail.component.html               |    98 +-
 .../cluster-detail.component.scss               |     6 +-
 .../cluster-detail/cluster-detail.component.ts  |     4 +-
 .../cluster-list/cluster-list.component.html    |    26 +-
 .../cluster-list/cluster-list.component.ts      |     6 +-
 .../client/app/cluster/cluster.component.html   |    12 +-
 .../client/app/cluster/cluster.component.scss   |     2 +-
 .../client/app/cluster/cluster.module.ts        |     3 -
 .../config-detail/config-detail.component.scss  |    19 -
 .../controller-detail.component.html            |    20 +-
 .../controller-detail.component.scss            |     4 +-
 .../client/app/controller/controller.module.ts  |     4 -
 helix-front/client/app/core/settings.ts         |     3 +-
 .../history-list/history-list.component.html    |    12 +-
 .../history-list/history-list.component.scss    |     4 +-
 .../history-list/history-list.component.ts      |     4 +-
 .../client/app/history/history.module.ts        |     8 +-
 .../instance-detail.component.html              |    44 +-
 .../instance-detail.component.scss              |     2 -
 .../instance-list/instance-list.component.html  |    10 +-
 .../instance-list/instance-list.component.ts    |     4 +-
 .../client/app/instance/instance.module.ts      |     4 -
 .../partition-detail.component.html             |    12 +-
 .../partition-detail.component.ts               |     4 +-
 .../partition-list.component.html               |    14 +-
 .../partition-list/partition-list.component.ts  |     4 +-
 .../resource-detail-for-instance.component.html |     7 +-
 .../resource-detail-for-instance.component.scss |     4 +-
 .../resource-detail.component.html              |    44 +-
 .../resource-detail.component.scss              |     2 -
 .../resource-list/resource-list.component.html  |    14 +-
 .../resource-list/resource-list.component.ts    |     9 +-
 .../client/app/resource/resource.module.ts      |     4 -
 .../shared/data-table/data-table.component.html |     8 +-
 .../data-table/data-table.component.spec.ts     |     6 +-
 .../shared/data-table/data-table.component.ts   |     4 +-
 .../detail-header/detail-header.component.html  |     6 +-
 .../detail-header/detail-header.component.scss  |     5 +-
 .../alert-dialog/alert-dialog.component.html    |     4 +-
 .../alert-dialog/alert-dialog.component.ts      |     4 +-
 .../confirm-dialog.component.html               |    10 +-
 .../confirm-dialog/confirm-dialog.component.ts  |     8 +-
 .../input-dialog/input-dialog.component.html    |    20 +-
 .../input-dialog/input-dialog.component.ts      |     8 +-
 .../disabled-label.component.scss               |     2 +-
 helix-front/client/app/shared/helper.service.ts |     6 +-
 .../input-inline/input-inline.component.html    |    12 +-
 .../key-value-pairs.component.scss              |     2 +
 .../client/app/shared/material.module.ts        |    66 +
 .../node-viewer/node-viewer.component.html      |   102 +-
 .../node-viewer/node-viewer.component.scss      |     4 +-
 .../node-viewer/node-viewer.component.spec.ts   |     8 +-
 .../shared/node-viewer/node-viewer.component.ts |     5 +-
 helix-front/client/app/shared/shared.module.ts  |     2 +-
 .../state-label/state-label.component.html      |     2 +-
 .../workflow-detail.component.html              |    16 +-
 .../workflow-list/workflow-list.component.html  |    10 +-
 helix-front/client/styles.scss                  |     4 +
 helix-front/client/testing/testing.module.ts    |     9 +-
 helix-front/client/tsconfig.app.json            |     4 +
 helix-front/package-lock.json                   | 11094 +++++++++++++++++
 helix-front/package.json                        |    43 +-
 67 files changed, 11542 insertions(+), 399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/app.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/app.component.html b/helix-front/client/app/app.component.html
index f10f7dd..87be9e4 100644
--- a/helix-front/client/app/app.component.html
+++ b/helix-front/client/app/app.component.html
@@ -1,22 +1,22 @@
 <section class="app" fxLayout="column" fxFill>
-  <md-toolbar *ngIf="headerEnabled" class="header mat-elevation-z7" color="primary">
-    <button md-icon-button
+  <mat-toolbar *ngIf="headerEnabled" class="header mat-elevation-z7" color="primary">
+    <button mat-icon-button
       *ngIf="isNarrowView"
       (click)="sidenav.open()">
-      <md-icon>menu</md-icon>
+      <mat-icon>menu</mat-icon>
     </button>
     <h2 routerLink="/">Helix</h2>
     <span fxFlex="1 1 auto"></span>
-    <a md-button>
-      <md-icon>person</md-icon>
+    <a mat-button>
+      <mat-icon>person</mat-icon>
       {{ currentUser | async }}
     </a>
-  </md-toolbar>
-  <md-progress-bar *ngIf="isLoading" mode="indeterminate" [ngClass]="{'no-header': !headerEnabled}"></md-progress-bar>
+  </mat-toolbar>
+  <mat-progress-bar *ngIf="isLoading" mode="indeterminate" [ngClass]="{'no-header': !headerEnabled}"></mat-progress-bar>
   <section class="main-container" [ngClass]="{'no-header': !headerEnabled}">
     <router-outlet></router-outlet>
   </section>
   <section *ngIf="footerEnabled" class="footer mat-elevation-z7" fxLayout="row" fxLayoutAlign="center center">
-    <span>&copy; 2017 Helix. All rights reserved.</span>
+    <span>&copy; 2018 Helix. All rights reserved.</span>
   </section>
 </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/app.component.spec.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/app.component.spec.ts b/helix-front/client/app/app.component.spec.ts
index 030b256..2ac2d90 100644
--- a/helix-front/client/app/app.component.spec.ts
+++ b/helix-front/client/app/app.component.spec.ts
@@ -39,10 +39,10 @@ describe('AppComponent', () => {
     expect(app.footerEnabled).toBeDefined();
   }));
 
-  xit('should render title in a md-toolbar', async(() => {
+  xit('should render title in a mat-toolbar', async(() => {
     const fixture = TestBed.createComponent(AppComponent);
     fixture.detectChanges();
     const compiled = fixture.debugElement.nativeElement;
-    expect(compiled.querySelector('md-toolbar').textContent).toContain('Helix');
+    expect(compiled.querySelector('mat-toolbar').textContent).toContain('Helix');
   }));
 });

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/app.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/app.module.ts b/helix-front/client/app/app.module.ts
index 2c66e5c..0b5fd25 100644
--- a/helix-front/client/app/app.module.ts
+++ b/helix-front/client/app/app.module.ts
@@ -2,8 +2,6 @@ import { BrowserModule } from '@angular/platform-browser';
 import { NgModule } from '@angular/core';
 import { FormsModule } from '@angular/forms';
 import { HttpModule } from '@angular/http';
-import { FlexLayoutModule } from '@angular/flex-layout';
-import { MaterialModule } from '@angular/material';
 
 import { Angulartics2Module, Angulartics2Piwik } from 'angulartics2';
 
@@ -32,8 +30,6 @@ import { ChooserModule } from './chooser/chooser.module';
     Angulartics2Module.forRoot([ Angulartics2Piwik ]),
     CoreModule,
     SharedModule,
-    FlexLayoutModule,
-    MaterialModule,
     ClusterModule,
     ConfigurationModule,
     InstanceModule,

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/chooser/helix-list/helix-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/chooser/helix-list/helix-list.component.html b/helix-front/client/app/chooser/helix-list/helix-list.component.html
index d042c51..e8df780 100644
--- a/helix-front/client/app/chooser/helix-list/helix-list.component.html
+++ b/helix-front/client/app/chooser/helix-list/helix-list.component.html
@@ -1,15 +1,15 @@
-<md-card *ngFor="let group of keys(groups)">
-  <md-card-header>
-    <md-card-title>{{ group }}</md-card-title>
-  </md-card-header>
-  <md-card-content>
+<mat-card *ngFor="let group of keys(groups)">
+  <mat-card-header>
+    <mat-card-title>{{ group }}</mat-card-title>
+  </mat-card-header>
+  <mat-card-content>
     <section *ngFor="let section of groups[group]" class="section">
       <a *ngFor="let helix of keys(section)"
-        md-button
+        mat-button
         [routerLink]="['/', group + '.' + helix]">
-        <md-icon>group_work</md-icon>
+        <mat-icon>group_work</mat-icon>
         {{ helix }}
       </a>
     </section>
-  </md-card-content>
-</md-card>
+  </mat-card-content>
+</mat-card>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/chooser/helix-list/helix-list.component.spec.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/chooser/helix-list/helix-list.component.spec.ts b/helix-front/client/app/chooser/helix-list/helix-list.component.spec.ts
index 08d5161..6d5c7a7 100644
--- a/helix-front/client/app/chooser/helix-list/helix-list.component.spec.ts
+++ b/helix-front/client/app/chooser/helix-list/helix-list.component.spec.ts
@@ -1,9 +1,7 @@
 import { async, ComponentFixture, TestBed } from '@angular/core/testing';
 import { NO_ERRORS_SCHEMA } from '@angular/core';
-import { MaterialModule } from '@angular/material';
-import { RouterTestingModule } from '@angular/router/testing';
-import { HttpModule } from '@angular/http';
 
+import { TestingModule } from '../../../testing/testing.module';
 import { HelixListComponent } from './helix-list.component';
 import { ChooserService } from '../shared/chooser.service';
 
@@ -14,9 +12,7 @@ describe('HelixListComponent', () => {
   beforeEach(async(() => {
     TestBed.configureTestingModule({
       imports: [
-        MaterialModule,
-        RouterTestingModule,
-        HttpModule
+        TestingModule
       ],
       declarations: [ HelixListComponent ],
       providers: [ ChooserService ],

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.html b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.html
index fe9091a..5b8a78c 100644
--- a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.html
+++ b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.html
@@ -1,54 +1,56 @@
-<section class="cluster-detail">
-  <section *ngIf="clusterName">
-    <md-toolbar class="mat-elevation-z1">
+<section *ngIf="clusterName" class="cluster-detail" fxLayout="column" fxFill>
+  <mat-toolbar class="mat-elevation-z1" fxFlex="none">
+    <mat-toolbar-row>
       <hi-detail-header [cluster]="clusterName"></hi-detail-header>
       <hi-disabled-label *ngIf="!cluster?.enabled" text="DISABLED"></hi-disabled-label>
-      <md-toolbar-row class="information">
-        <md-spinner *ngIf="isLoading"></md-spinner>
-        <h6 *ngIf="!isLoading">Controller:
-          <a md-button color="accent" routerLink="controller">{{ cluster.controller }}</a>
-        </h6>
-        <span fxFlex="1 1 auto"></span>
-        <button md-mini-fab *ngIf="can" [mdMenuTriggerFor]="menu">
-          <md-icon>menu</md-icon>
+    </mat-toolbar-row>
+    <mat-toolbar-row class="information">
+      <mat-spinner *ngIf="isLoading" diameter="30"></mat-spinner>
+      <h6 *ngIf="!isLoading">Controller:
+        <a mat-button color="accent" routerLink="controller">{{ cluster.controller }}</a>
+      </h6>
+      <span fxFlex="1 1 auto"></span>
+      <button mat-mini-fab *ngIf="can" [matMenuTriggerFor]="menu">
+        <mat-icon>menu</mat-icon>
+      </button>
+      <mat-menu #menu="matMenu">
+        <button mat-menu-item *ngIf="cluster?.controller == 'No Lead Controller!'" (click)="activateCluster()">
+          <mat-icon>settings_input_antenna</mat-icon>
+          <span>Activate this Cluster</span>
         </button>
-        <md-menu #menu="mdMenu">
-          <button md-menu-item *ngIf="cluster?.controller == 'No Lead Controller!'" (click)="activateCluster()">
-            <md-icon>settings_input_antenna</md-icon>
-            <span>Activate this Cluster</span>
-          </button>
-          <button md-menu-item *ngIf="cluster?.enabled" (click)="disableCluster()">
-            <md-icon>not_interested</md-icon>
-            <span>Disable this Cluster</span>
-          </button>
-          <button md-menu-item *ngIf="!cluster?.enabled" (click)="enableCluster()">
-            <md-icon>play_circle_outline</md-icon>
-            <span>Enable this Cluster</span>
-          </button>
-          <button md-menu-item *ngIf="false" (click)="addResource()">
-            <md-icon>note_add</md-icon>
-            <span>Add a Resource</span>
-          </button>
-          <button md-menu-item (click)="addInstance()">
-            <md-icon>add_circle</md-icon>
-            <span>Add an Instance</span>
-          </button>
-          <button md-menu-item *ngIf="false" (click)="deleteCluster()" disabled>
-            <md-icon>delete</md-icon>
-            <span>DELETE this Cluster</span>
-          </button>
-        </md-menu>
-      </md-toolbar-row>
-    </md-toolbar>
-    <nav md-tab-nav-bar>
-      <a md-tab-link
-        *ngFor="let tabLink of tabLinks"
-        [routerLink]="tabLink.link"
-        routerLinkActive #rla="routerLinkActive"
-        [active]="rla.isActive">
-        {{ tabLink.label }}
-      </a>
-    </nav>
+        <button mat-menu-item *ngIf="cluster?.enabled" (click)="disableCluster()">
+          <mat-icon>not_interested</mat-icon>
+          <span>Disable this Cluster</span>
+        </button>
+        <button mat-menu-item *ngIf="!cluster?.enabled" (click)="enableCluster()">
+          <mat-icon>play_circle_outline</mat-icon>
+          <span>Enable this Cluster</span>
+        </button>
+        <button mat-menu-item *ngIf="false" (click)="addResource()">
+          <mat-icon>note_add</mat-icon>
+          <span>Add a Resource</span>
+        </button>
+        <button mat-menu-item (click)="addInstance()">
+          <mat-icon>add_circle</mat-icon>
+          <span>Add an Instance</span>
+        </button>
+        <button mat-menu-item *ngIf="false" (click)="deleteCluster()" disabled>
+          <mat-icon>delete</mat-icon>
+          <span>DELETE this Cluster</span>
+        </button>
+      </mat-menu>
+    </mat-toolbar-row>
+  </mat-toolbar>
+  <nav mat-tab-nav-bar>
+    <a mat-tab-link
+      *ngFor="let tabLink of tabLinks"
+      [routerLink]="tabLink.link"
+      routerLinkActive #rla="routerLinkActive"
+      [active]="rla.isActive">
+      {{ tabLink.label }}
+    </a>
+  </nav>
+  <section fxFlex>
     <router-outlet></router-outlet>
   </section>
 </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.scss b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.scss
index 41884b5..d16a356 100644
--- a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.scss
+++ b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.scss
@@ -1,5 +1,7 @@
 .mat-spinner {
-  width: 30px;
-  height: 30px;
   margin: 0 20px;
 }
+
+.mat-toolbar h6 {
+  font-size: 14px;
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.ts b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.ts
index e0db1b5..1308b4d 100644
--- a/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.ts
+++ b/helix-front/client/app/cluster/cluster-detail/cluster-detail.component.ts
@@ -1,6 +1,6 @@
 import { Component, OnInit } from '@angular/core';
 import { ActivatedRoute, Router } from '@angular/router';
-import { MdDialog } from '@angular/material';
+import { MatDialog } from '@angular/material';
 
 import { Cluster } from '../shared/cluster.model';
 import { HelperService } from '../../shared/helper.service';
@@ -32,7 +32,7 @@ export class ClusterDetailComponent implements OnInit {
   constructor(
     protected route: ActivatedRoute,
     protected router: Router,
-    protected dialog: MdDialog,
+    protected dialog: MatDialog,
     protected helperService: HelperService,
     protected clusterService: ClusterService,
     protected instanceService: InstanceService

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-list/cluster-list.component.html b/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
index 7df5531..c32b1b8 100644
--- a/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
+++ b/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
@@ -1,29 +1,29 @@
 <section class="cluster-list">
   <section *ngIf="isLoading" fxLayout="row" fxLayoutAlign="center center">
-    <md-spinner>
+    <mat-spinner>
       Loading all clusters ...
-    </md-spinner>
+    </mat-spinner>
   </section>
-  <md-nav-list *ngIf="!isLoading && !errorMessage">
-    <button md-button routerLink="/">
-      <md-icon>arrow_back</md-icon> Back to Index
+  <mat-nav-list *ngIf="!isLoading && !errorMessage">
+    <button mat-button routerLink="/">
+      <mat-icon>arrow_back</mat-icon> Back to Index
     </button>
-    <button md-mini-fab *ngIf="can" (click)="createCluster()">
-      <md-icon>add</md-icon>
+    <button mat-mini-fab *ngIf="can" (click)="createCluster()">
+      <mat-icon>add</mat-icon>
     </button>
-    <h3 md-subheader>Clusters ({{ clusters.length }})</h3>
+    <h3 mat-subheader>Clusters ({{ clusters.length }})</h3>
     <a *ngFor="let cluster of clusters"
-      md-list-item
+      mat-list-item
       [routerLink]="[cluster.name]"
       routerLinkActive="cluster-list-item-selected">
-      <md-icon md-list-icon>blur_circular</md-icon>
-      <h4 md-line>{{ cluster.name }}</h4>
+      <mat-icon mat-list-icon>blur_circular</mat-icon>
+      <h4 mat-line>{{ cluster.name }}</h4>
     </a>
     <div *ngIf="clusters.length == 0" class="empty">
       There's no cluster here.
-      <a md-button *ngIf="can" (click)="createCluster()">Create one?</a>
+      <a mat-button *ngIf="can" (click)="createCluster()">Create one?</a>
     </div>
-  </md-nav-list>
+  </mat-nav-list>
   <section class="error-message" *ngIf="errorMessage">
     {{ errorMessage }}
   </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster-list/cluster-list.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-list/cluster-list.component.ts b/helix-front/client/app/cluster/cluster-list/cluster-list.component.ts
index f9a2321..35d0b92 100644
--- a/helix-front/client/app/cluster/cluster-list/cluster-list.component.ts
+++ b/helix-front/client/app/cluster/cluster-list/cluster-list.component.ts
@@ -1,5 +1,5 @@
 import { Component, OnInit } from '@angular/core';
-import { MdDialog, MdSnackBar } from '@angular/material';
+import { MatDialog, MatSnackBar } from '@angular/material';
 
 import { ClusterService } from '../shared/cluster.service';
 import { Cluster } from '../shared/cluster.model';
@@ -20,8 +20,8 @@ export class ClusterListComponent implements OnInit {
 
   constructor(
     protected clusterService: ClusterService,
-    protected dialog: MdDialog,
-    protected snackBar: MdSnackBar
+    protected dialog: MatDialog,
+    protected snackBar: MatSnackBar
   ) { }
 
   ngOnInit() {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster.component.html b/helix-front/client/app/cluster/cluster.component.html
index 6f98d10..d1a31e8 100644
--- a/helix-front/client/app/cluster/cluster.component.html
+++ b/helix-front/client/app/cluster/cluster.component.html
@@ -1,11 +1,13 @@
-<md-sidenav-container fxFill>
-  <md-sidenav
+<mat-sidenav-container fxFill>
+  <mat-sidenav
     #sidenav
     [mode]="isNarrowView ? 'over' : 'side'"
     [opened]="!isNarrowView"
     [disableClose]="!isNarrowView"
     class="mat-elevation-z4">
     <hi-cluster-list></hi-cluster-list>
-  </md-sidenav>
-  <router-outlet></router-outlet>
-</md-sidenav-container>
+  </mat-sidenav>
+  <section fxFill>
+    <router-outlet></router-outlet>
+  </section>
+</mat-sidenav-container>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster.component.scss b/helix-front/client/app/cluster/cluster.component.scss
index 5845753..3f4d819 100644
--- a/helix-front/client/app/cluster/cluster.component.scss
+++ b/helix-front/client/app/cluster/cluster.component.scss
@@ -1,4 +1,4 @@
-md-sidenav {
+mat-sidenav {
   width: 25vw;
   min-width: 200px;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/cluster/cluster.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster.module.ts b/helix-front/client/app/cluster/cluster.module.ts
index b58b4ae..02efb7f 100644
--- a/helix-front/client/app/cluster/cluster.module.ts
+++ b/helix-front/client/app/cluster/cluster.module.ts
@@ -1,8 +1,6 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
 import { HttpModule } from '@angular/http';
-import { BrowserAnimationsModule } from '@angular/platform-browser/animations';
-import 'hammerjs';
 
 import { SharedModule } from '../shared/shared.module';
 import { ClusterService } from './shared/cluster.service';
@@ -15,7 +13,6 @@ import { ClusterComponent } from './cluster.component';
   imports: [
     CommonModule,
     HttpModule,
-    BrowserAnimationsModule,
     SharedModule
   ],
   declarations: [

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/configuration/config-detail/config-detail.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/configuration/config-detail/config-detail.component.scss b/helix-front/client/app/configuration/config-detail/config-detail.component.scss
index 5863640..e69de29 100644
--- a/helix-front/client/app/configuration/config-detail/config-detail.component.scss
+++ b/helix-front/client/app/configuration/config-detail/config-detail.component.scss
@@ -1,19 +0,0 @@
-@import '~@angular/material/theming';
-@import 'client/theme.scss';
-
-.primary {
-  color: mat-color($hi-primary);
-}
-
-md-input-container {
-  width: 300px;
-  padding: 10px 0 0 5px;
-}
-
-md-card {
-  margin-bottom: 10px;
-}
-
-ngx-datatable {
-  word-break: break-all;
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/controller/controller-detail/controller-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/controller/controller-detail/controller-detail.component.html b/helix-front/client/app/controller/controller-detail/controller-detail.component.html
index 62a3674..7f5f2bf 100644
--- a/helix-front/client/app/controller/controller-detail/controller-detail.component.html
+++ b/helix-front/client/app/controller/controller-detail/controller-detail.component.html
@@ -1,18 +1,20 @@
 <section>
-  <md-toolbar class="mat-elevation-z1">
-    <hi-detail-header [cluster]="clusterName" [controller]="controller?.name"></hi-detail-header>
-    <md-toolbar-row class="information" fxLayout="row">
-      <a md-mini-fab routerLink="../"><md-icon>arrow_back</md-icon></a>
-      <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-toolbar class="mat-elevation-z1">
+    <mat-toolbar-row>
+      <hi-detail-header [cluster]="clusterName" [controller]="controller?.name"></hi-detail-header>
+    </mat-toolbar-row>
+    <mat-toolbar-row class="information" fxLayout="row">
+      <a mat-mini-fab routerLink="../"><mat-icon>arrow_back</mat-icon></a>
+      <mat-spinner *ngIf="isLoading" diameter="30"></mat-spinner>
       <hi-key-value-pairs *ngIf="!isLoading" [obj]="controller">
         <hi-key-value-pair name="Instance" prop="liveInstance"></hi-key-value-pair>
         <hi-key-value-pair name="Session ID" prop="sessionId"></hi-key-value-pair>
         <hi-key-value-pair name="Helix Version" prop="helixVersion"></hi-key-value-pair>
       </hi-key-value-pairs>
-    </md-toolbar-row>
-  </md-toolbar>
-  <nav md-tab-nav-bar>
-    <a md-tab-link
+    </mat-toolbar-row>
+  </mat-toolbar>
+  <nav mat-tab-nav-bar>
+    <a mat-tab-link
       routerLink="history"
       routerLinkActive #rla="routerLinkActive"
       [active]="rla.isActive">

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/controller/controller-detail/controller-detail.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/controller/controller-detail/controller-detail.component.scss b/helix-front/client/app/controller/controller-detail/controller-detail.component.scss
index bef3b1f..cd18c9e 100644
--- a/helix-front/client/app/controller/controller-detail/controller-detail.component.scss
+++ b/helix-front/client/app/controller/controller-detail/controller-detail.component.scss
@@ -1,7 +1,5 @@
 .information {
-  md-spinner {
-    width: 30px;
-    height: 30px;
+  .mat-spinner {
     margin: 0 20px;
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/controller/controller.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/controller/controller.module.ts b/helix-front/client/app/controller/controller.module.ts
index 1e1b92f..40b454a 100644
--- a/helix-front/client/app/controller/controller.module.ts
+++ b/helix-front/client/app/controller/controller.module.ts
@@ -1,7 +1,5 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
-import { MaterialModule } from '@angular/material';
-import { RouterModule } from '@angular/router';
 
 import { SharedModule } from '../shared/shared.module';
 import { ControllerDetailComponent } from './controller-detail/controller-detail.component';
@@ -9,8 +7,6 @@ import { ControllerDetailComponent } from './controller-detail/controller-detail
 @NgModule({
   imports: [
     CommonModule,
-    RouterModule,
-    MaterialModule,
     SharedModule
   ],
   declarations: [ControllerDetailComponent]

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/core/settings.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/core/settings.ts b/helix-front/client/app/core/settings.ts
index a68263c..cdb496f 100644
--- a/helix-front/client/app/core/settings.ts
+++ b/helix-front/client/app/core/settings.ts
@@ -1,5 +1,6 @@
 export class Settings {
-  static readonly tableRowHeight = 40;
+  static readonly tableHeaderHeight = 40;
+  static readonly tableRowHeight = 50;
   static readonly helixAPI = '/api/helix';
   static readonly userAPI = '/api/user';
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/history/history-list/history-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/history/history-list/history-list.component.html b/helix-front/client/app/history/history-list/history-list.component.html
index 6249ac5..5a333b9 100644
--- a/helix-front/client/app/history/history-list/history-list.component.html
+++ b/helix-front/client/app/history/history-list/history-list.component.html
@@ -3,21 +3,21 @@
     #historyTable
     class="material"
     [loadingIndicator]="isLoading"
-    [headerHeight]="rowHeight"
-    rowHeight="auto"
+    [headerHeight]="headerHeight"
+    [rowHeight]="rowHeight"
     columnMode="force"
     [rows]="rows"
     [sorts]="sorts">
     <ngx-datatable-column name="Date (GMT)" prop="date" [width]="240" [canAutoResize]="false"></ngx-datatable-column>
     <ngx-datatable-column name="Epoch Time" prop="time" [width]="200" [canAutoResize]="false">
       <ng-template let-value="value" ngx-datatable-cell-template>
-        <span mdTooltip="Local time: {{ value | date: 'yMMMdjms' }}">
+        <span matTooltip="Local time: {{ value | date: 'medium' }}">
           {{ value }}
         </span>
-        <button md-icon-button
-          mdTooltip="Copy to clipboard"
+        <button mat-icon-button
+          matTooltip="Copy to clipboard"
           ngxClipboard [cbContent]="value">
-          <md-icon>content_copy</md-icon>
+          <mat-icon>content_copy</mat-icon>
         </button>
       </ng-template>
     </ngx-datatable-column>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/history/history-list/history-list.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/history/history-list/history-list.component.scss b/helix-front/client/app/history/history-list/history-list.component.scss
index 8171744..459c950 100644
--- a/helix-front/client/app/history/history-list/history-list.component.scss
+++ b/helix-front/client/app/history/history-list/history-list.component.scss
@@ -8,7 +8,7 @@ hi-history-list {
     color: mat-color($hi-primary);
   }
 
-  .datatable-body-cell {
-    line-height: 40px !important;
+  .mat-icon {
+    @include md-icon-size(20px);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/history/history-list/history-list.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/history/history-list/history-list.component.ts b/helix-front/client/app/history/history-list/history-list.component.ts
index 008d72f..eee1ff7 100644
--- a/helix-front/client/app/history/history-list/history-list.component.ts
+++ b/helix-front/client/app/history/history-list/history-list.component.ts
@@ -3,6 +3,7 @@ import { ActivatedRoute } from '@angular/router';
 
 import * as _ from 'lodash';
 
+import { Settings } from '../../core/settings';
 import { HistoryService } from '../shared/history.service';
 import { History } from '../shared/history.model';
 
@@ -17,7 +18,8 @@ import { History } from '../shared/history.model';
 export class HistoryListComponent implements OnInit {
 
   rows: History[];
-  rowHeight = 40;
+  rowHeight = Settings.tableRowHeight;
+  headerHeight = Settings.tableHeaderHeight;
   isController: boolean;
   isLoading = true;
   sorts = [

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/history/history.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/history/history.module.ts b/helix-front/client/app/history/history.module.ts
index e0807b4..34fbf25 100644
--- a/helix-front/client/app/history/history.module.ts
+++ b/helix-front/client/app/history/history.module.ts
@@ -1,20 +1,18 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
-import { RouterModule } from '@angular/router';
-import { MaterialModule } from '@angular/material';
 
 import { NgxDatatableModule } from '@swimlane/ngx-datatable';
 import { ClipboardModule } from 'ngx-clipboard';
 
+import { SharedModule } from '../shared/shared.module';
 import { HistoryListComponent } from './history-list/history-list.component';
 
 @NgModule({
   imports: [
     CommonModule,
-    RouterModule,
-    MaterialModule,
     NgxDatatableModule,
-    ClipboardModule
+    ClipboardModule,
+    SharedModule
   ],
   declarations: [HistoryListComponent]
 })

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/instance/instance-detail/instance-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/instance/instance-detail/instance-detail.component.html b/helix-front/client/app/instance/instance-detail/instance-detail.component.html
index 4e96fa0..bcdc12d 100644
--- a/helix-front/client/app/instance/instance-detail/instance-detail.component.html
+++ b/helix-front/client/app/instance/instance-detail/instance-detail.component.html
@@ -1,38 +1,40 @@
 <section>
-  <md-toolbar class="mat-elevation-z1">
-    <hi-detail-header [cluster]="clusterName" [instance]="instanceName"></hi-detail-header>
-    <hi-disabled-label *ngIf="!isLoading && !instance.liveInstance" text="OFFLINE"></hi-disabled-label>
-    <hi-disabled-label *ngIf="!isLoading && !instance.enabled" text="DISABLED"></hi-disabled-label>
-    <md-toolbar-row class="information">
-      <a md-mini-fab routerLink="../"><md-icon>arrow_back</md-icon></a>
-      <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-toolbar class="mat-elevation-z1">
+    <mat-toolbar-row>
+      <hi-detail-header [cluster]="clusterName" [instance]="instanceName"></hi-detail-header>
+      <hi-disabled-label *ngIf="!isLoading && !instance.liveInstance" text="OFFLINE"></hi-disabled-label>
+      <hi-disabled-label *ngIf="!isLoading && !instance.enabled" text="DISABLED"></hi-disabled-label>
+    </mat-toolbar-row>
+    <mat-toolbar-row class="information">
+      <a mat-mini-fab routerLink="../"><mat-icon>arrow_back</mat-icon></a>
+      <mat-spinner *ngIf="isLoading" diameter="30"></mat-spinner>
       <hi-key-value-pairs *ngIf="!isLoading" [obj]="instance">
         <hi-key-value-pair name="Instance" prop="liveInstance"></hi-key-value-pair>
         <hi-key-value-pair name="Session ID" prop="sessionId"></hi-key-value-pair>
         <hi-key-value-pair name="Helix Version" prop="helixVersion"></hi-key-value-pair>
       </hi-key-value-pairs>
       <span fxFlex="1 1 auto"></span>
-      <button md-mini-fab *ngIf="can" [mdMenuTriggerFor]="menu">
-        <md-icon>menu</md-icon>
+      <button mat-mini-fab *ngIf="can" [matMenuTriggerFor]="menu">
+        <mat-icon>menu</mat-icon>
       </button>
-      <md-menu #menu="mdMenu">
-        <button md-menu-item *ngIf="instance && instance.enabled" (click)="disableInstance()">
-          <md-icon>not_interested</md-icon>
+      <mat-menu #menu="matMenu">
+        <button mat-menu-item *ngIf="instance && instance.enabled" (click)="disableInstance()">
+          <mat-icon>not_interested</mat-icon>
           <span>Disable this Instance</span>
         </button>
-        <button md-menu-item *ngIf="instance && !instance.enabled" (click)="enableInstance()">
-          <md-icon>play_circle_outline</md-icon>
+        <button mat-menu-item *ngIf="instance && !instance.enabled" (click)="enableInstance()">
+          <mat-icon>play_circle_outline</mat-icon>
           <span>Enable this Instance</span>
         </button>
-        <button md-menu-item (click)="removeInstance()">
-          <md-icon>delete</md-icon>
+        <button mat-menu-item (click)="removeInstance()">
+          <mat-icon>delete</mat-icon>
           <span>REMOVE this Instance</span>
         </button>
-      </md-menu>
-    </md-toolbar-row>
-  </md-toolbar>
-  <nav md-tab-nav-bar>
-    <a md-tab-link
+      </mat-menu>
+    </mat-toolbar-row>
+  </mat-toolbar>
+  <nav mat-tab-nav-bar>
+    <a mat-tab-link
       *ngFor="let tabLink of tabLinks"
       [routerLink]="tabLink.link"
       routerLinkActive #rla="routerLinkActive"

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/instance/instance-detail/instance-detail.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/instance/instance-detail/instance-detail.component.scss b/helix-front/client/app/instance/instance-detail/instance-detail.component.scss
index 999ab59..1cbb18e 100644
--- a/helix-front/client/app/instance/instance-detail/instance-detail.component.scss
+++ b/helix-front/client/app/instance/instance-detail/instance-detail.component.scss
@@ -3,8 +3,6 @@
   font-size: 14px;
 
   .mat-spinner {
-    width: 30px;
-    height: 30px;
     margin: 0 20px;
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/instance/instance-list/instance-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/instance/instance-list/instance-list.component.html b/helix-front/client/app/instance/instance-list/instance-list.component.html
index b9e5cb7..17a0301 100644
--- a/helix-front/client/app/instance/instance-list/instance-list.component.html
+++ b/helix-front/client/app/instance/instance-list/instance-list.component.html
@@ -2,7 +2,7 @@
   <ngx-datatable
     #instancesTable
     class="material"
-    [headerHeight]="rowHeight"
+    [headerHeight]="headerHeight"
     [rowHeight]="rowHeight"
     columnMode="force"
     [footerHeight]="rowHeight"
@@ -19,10 +19,10 @@
       [draggable]="false"
       [canAutoResize]="false">
       <ng-template let-row="row" ngx-datatable-cell-template>
-        <md-icon *ngIf="row.healthy" class="status-healthy">lens</md-icon>
-        <md-icon *ngIf="!row.healthy && row.enabled" class="status-not-healthy" mdTooltip="The instance is offline.">lens</md-icon>
-        <md-icon *ngIf="!row.healthy && row.liveInstance" class="status-not-healthy" mdTooltip="The instance is disabled.">lens</md-icon>
-        <md-icon *ngIf="!row.healthy && !row.enabled && !row.liveInstance" class="status-not-healthy" mdTooltip="The instance is offline and disabled.">lens</md-icon>
+        <mat-icon *ngIf="row.healthy" class="status-healthy">lens</mat-icon>
+        <mat-icon *ngIf="!row.healthy && row.enabled" class="status-not-healthy" matTooltip="The instance is offline.">lens</mat-icon>
+        <mat-icon *ngIf="!row.healthy && row.liveInstance" class="status-not-healthy" matTooltip="The instance is disabled.">lens</mat-icon>
+        <mat-icon *ngIf="!row.healthy && !row.enabled && !row.liveInstance" class="status-not-healthy" matTooltip="The instance is offline and disabled.">lens</mat-icon>
       </ng-template>
     </ngx-datatable-column>
     <ngx-datatable-column name="Name">

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/instance/instance-list/instance-list.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/instance/instance-list/instance-list.component.ts b/helix-front/client/app/instance/instance-list/instance-list.component.ts
index 748dbc6..985e193 100644
--- a/helix-front/client/app/instance/instance-list/instance-list.component.ts
+++ b/helix-front/client/app/instance/instance-list/instance-list.component.ts
@@ -1,6 +1,7 @@
 import { Component, OnInit } from '@angular/core';
 import { Router, ActivatedRoute } from '@angular/router';
 
+import { Settings } from '../../core/settings';
 import { InstanceService } from '../shared/instance.service';
 import { HelperService } from '../../shared/helper.service';
 
@@ -14,7 +15,8 @@ export class InstanceListComponent implements OnInit {
   isLoading = true;
   clusterName: string;
   instances: any[];
-  rowHeight = 40;
+  rowHeight = Settings.tableRowHeight;
+  headerHeight = Settings.tableHeaderHeight;
   sorts = [
     { prop: 'liveInstance', dir: 'asc'},
     { prop: 'name', dir: 'asc'}

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/instance/instance.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/instance/instance.module.ts b/helix-front/client/app/instance/instance.module.ts
index 7c85835..2332bb5 100644
--- a/helix-front/client/app/instance/instance.module.ts
+++ b/helix-front/client/app/instance/instance.module.ts
@@ -1,7 +1,5 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
-import { RouterModule } from '@angular/router';
-import { MaterialModule } from '@angular/material';
 
 import { NgxDatatableModule } from '@swimlane/ngx-datatable';
 
@@ -13,8 +11,6 @@ import { InstanceDetailComponent } from './instance-detail/instance-detail.compo
 @NgModule({
   imports: [
     CommonModule,
-    RouterModule,
-    MaterialModule,
     NgxDatatableModule,
     SharedModule
   ],

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/partition-detail/partition-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-detail/partition-detail.component.html b/helix-front/client/app/resource/partition-detail/partition-detail.component.html
index 372368f..de0e08c 100644
--- a/helix-front/client/app/resource/partition-detail/partition-detail.component.html
+++ b/helix-front/client/app/resource/partition-detail/partition-detail.component.html
@@ -1,7 +1,7 @@
 <ngx-datatable
   class="material"
-  [headerHeight]="rowHeight"
-  rowHeight="auto"
+  [headerHeight]="headerHeight"
+  [rowHeight]="rowHeight"
   columnMode="force"
   [rows]="partition?.replicas">
   <ngx-datatable-column
@@ -9,8 +9,8 @@
     [width]="80"
     [resizeable]="false"
     [canAutoResize]="false">
-    <ng-template let-row="row" ngx-datatable-cell-template>
-      <strong>#{{ row.$$index + 1 }}</strong>
+    <ng-template let-row="row" let-rowIndex="rowIndex" ngx-datatable-cell-template>
+      <strong>#{{ rowIndex + 1 }}</strong>
     </ng-template>
   </ngx-datatable-column>
   <ngx-datatable-column
@@ -18,10 +18,10 @@
     prop="instanceName">
     <ng-template let-value="value" ngx-datatable-cell-template>
       {{ value }}
-      <a md-icon-button
+      <a mat-icon-button
         color="accent"
         [routerLink]="['../../..', 'instances', value, 'resources']">
-        <md-icon>arrow_forward</md-icon>
+        <mat-icon>arrow_forward</mat-icon>
       </a>
     </ng-template>
   </ngx-datatable-column>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/partition-detail/partition-detail.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-detail/partition-detail.component.ts b/helix-front/client/app/resource/partition-detail/partition-detail.component.ts
index b16ea9c..9c5623a 100644
--- a/helix-front/client/app/resource/partition-detail/partition-detail.component.ts
+++ b/helix-front/client/app/resource/partition-detail/partition-detail.component.ts
@@ -1,5 +1,6 @@
 import { Component, OnInit, Input, ViewEncapsulation } from '@angular/core';
 
+import { Settings } from '../../core/settings';
 import { Partition, IReplica } from '../shared/resource.model';
 
 @Component({
@@ -13,7 +14,8 @@ export class PartitionDetailComponent implements OnInit {
   @Input() clusterName: string;
   @Input() partition: Partition;
 
-  rowHeight = 40;
+  headerHeight = Settings.tableHeaderHeight;
+  rowHeight = Settings.tableRowHeight;
 
   constructor() { }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/partition-list/partition-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-list/partition-list.component.html b/helix-front/client/app/resource/partition-list/partition-list.component.html
index 238d5e8..041b97e 100644
--- a/helix-front/client/app/resource/partition-list/partition-list.component.html
+++ b/helix-front/client/app/resource/partition-list/partition-list.component.html
@@ -3,8 +3,8 @@
     *ngIf="canAnalyse()"
     #partitionsTable
     class="material"
-    [headerHeight]="rowHeight"
-    rowHeight="auto"
+    [headerHeight]="headerHeight"
+    [rowHeight]="rowHeight"
     columnMode="force"
     [footerHeight]="rowHeight"
     [rows]="partitions"
@@ -18,8 +18,8 @@
      [sortable]="false"
      [draggable]="false"
      [canAutoResize]="false">
-     <ng-template let-row="row" ngx-datatable-cell-template>
-       <md-icon>{{ row.$$expanded ? 'expand_more' : 'chevron_right' }}</md-icon>
+     <ng-template let-expanded="expanded" ngx-datatable-cell-template>
+       <mat-icon>{{ expanded ? 'expand_more' : 'chevron_right' }}</mat-icon>
      </ng-template>
     </ngx-datatable-column>
     <ngx-datatable-column
@@ -30,7 +30,7 @@
       [draggable]="false"
       [canAutoResize]="false">
       <ng-template let-value="value" ngx-datatable-cell-template>
-        <md-icon [ngClass]="value ? 'status-ready' : 'status-not-ready'">lens</md-icon>
+        <mat-icon [ngClass]="value ? 'status-ready' : 'status-not-ready'">lens</mat-icon>
       </ng-template>
     </ngx-datatable-column>
     <ngx-datatable-column name="Name"></ngx-datatable-column>
@@ -40,7 +40,7 @@
       [resizeable]="false"
       [canAutoResize]="false">
       <ng-template let-value="value" ngx-datatable-cell-template>
-        <span *ngFor="let replica of value" [mdTooltip]="replica.instanceName">
+        <span *ngFor="let replica of value" [matTooltip]="replica.instanceName">
           <hi-state-label [state]="replica.externalView" [isReady]="replica.externalView && replica.externalView == replica.idealState"></hi-state-label>
         </span>
       </ng-template>
@@ -78,7 +78,7 @@
     </ngx-datatable-footer>
   </ngx-datatable>
   <div *ngIf="!canAnalyse()" class="message" fxLayout="column" fxLayoutAlign="center center">
-    <md-spinner *ngIf="isLoading"></md-spinner>
+    <mat-spinner *ngIf="isLoading"></mat-spinner>
     <section *ngIf="!isLoading && getReasonWhyCannotAnalyse()" fxFlexFill>
       {{ getReasonWhyCannotAnalyse() }}
     </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/partition-list/partition-list.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-list/partition-list.component.ts b/helix-front/client/app/resource/partition-list/partition-list.component.ts
index bee8e72..158da29 100644
--- a/helix-front/client/app/resource/partition-list/partition-list.component.ts
+++ b/helix-front/client/app/resource/partition-list/partition-list.component.ts
@@ -1,6 +1,7 @@
 import { Component, OnInit, ViewChild } from '@angular/core';
 import { ActivatedRoute } from '@angular/router';
 
+import { Settings } from '../../core/settings';
 import { Partition, IReplica, Resource } from '../shared/resource.model';
 import { HelperService } from '../../shared/helper.service';
 import { ResourceService } from '../shared/resource.service';
@@ -19,7 +20,8 @@ export class PartitionListComponent implements OnInit {
   clusterName: string;
   resource: Resource;
   partitions: Partition[];
-  rowHeight = 40;
+  headerHeight = Settings.tableHeaderHeight;
+  rowHeight = Settings.tableRowHeight;
   sorts = [
     { prop: 'isReady', dir: 'asc'},
     { prop: 'name', dir: 'asc'}

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.html b/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.html
index d2e75d2..c2b3739 100644
--- a/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.html
+++ b/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.html
@@ -1,7 +1,7 @@
 <section>
-  <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-spinner *ngIf="isLoading"></mat-spinner>
   <section *ngIf="!isLoading">
-    <a md-button
+    <a mat-button
       color="accent"
       [routerLink]="['../../..', 'resources', resourceName]">
       Other Partitions
@@ -11,7 +11,6 @@
       <hi-key-value-pair name="Session ID" prop="sessionId"></hi-key-value-pair>
       <hi-key-value-pair name="State Model" prop="stateModelDef"></hi-key-value-pair>
       <hi-key-value-pair name="State Model Factory Name" prop="stateModelFactoryName"></hi-key-value-pair>
-      <hi-key-value-pair name="Bucket Size" prop="bucketSize"></hi-key-value-pair>
     </hi-key-value-pairs>
 
     <ngx-datatable
@@ -25,7 +24,7 @@
       <ngx-datatable-column name="Partition" prop="name"></ngx-datatable-column>
       <ngx-datatable-column name="Current State" [width]="120" [canAutoResize]="false">
         <ng-template let-row="row" ngx-datatable-cell-template>
-          <span [mdTooltip]="row.info">
+          <span [matTooltip]="row.info">
             <hi-state-label [state]="row.currentState"></hi-state-label>
           </span>
         </ng-template>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.scss b/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.scss
index 417c834..66ebc48 100644
--- a/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.scss
+++ b/helix-front/client/app/resource/resource-detail-for-instance/resource-detail-for-instance.component.scss
@@ -1,8 +1,8 @@
-md-spinner {
+mat-spinner {
   margin: 0 auto;
 }
 
-[md-button] {
+[mat-button] {
   float: right;
 }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-detail/resource-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-detail/resource-detail.component.html b/helix-front/client/app/resource/resource-detail/resource-detail.component.html
index 613ee82..82eca4c 100644
--- a/helix-front/client/app/resource/resource-detail/resource-detail.component.html
+++ b/helix-front/client/app/resource/resource-detail/resource-detail.component.html
@@ -1,11 +1,13 @@
 <section>
-  <md-toolbar class="mat-elevation-z1">
-    <hi-detail-header [cluster]="clusterName" [resource]="resourceName"></hi-detail-header>
-    <hi-disabled-label *ngIf="!isLoading && !resource.online" text="OFFLINE"></hi-disabled-label>
-    <hi-disabled-label *ngIf="!isLoading && !resource.enabled" text="DISABLED"></hi-disabled-label>
-    <md-toolbar-row class="information">
-      <a md-mini-fab routerLink="../"><md-icon>arrow_back</md-icon></a>
-      <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-toolbar class="mat-elevation-z1">
+    <mat-toolbar-row>
+      <hi-detail-header [cluster]="clusterName" [resource]="resourceName"></hi-detail-header>
+      <hi-disabled-label *ngIf="!isLoading && !resource.online" text="OFFLINE"></hi-disabled-label>
+      <hi-disabled-label *ngIf="!isLoading && !resource.enabled" text="DISABLED"></hi-disabled-label>
+    </mat-toolbar-row>
+    <mat-toolbar-row class="information">
+      <a mat-mini-fab routerLink="../"><mat-icon>arrow_back</mat-icon></a>
+      <mat-spinner *ngIf="isLoading" diameter="30"></mat-spinner>
       <hi-key-value-pairs *ngIf="!isLoading" [obj]="resource">
         <hi-key-value-pair name="Ideal State Mode" prop="idealStateMode"></hi-key-value-pair>
         <hi-key-value-pair name="Rebalance Mode" prop="rebalanceMode"></hi-key-value-pair>
@@ -14,27 +16,27 @@
         <hi-key-value-pair name="Replication Factor" prop="replicaCount"></hi-key-value-pair>
       </hi-key-value-pairs>
       <span fxFlex="1 1 auto"></span>
-      <button md-mini-fab *ngIf="can" [mdMenuTriggerFor]="menu">
-        <md-icon>menu</md-icon>
+      <button mat-mini-fab *ngIf="can" [matMenuTriggerFor]="menu">
+        <mat-icon>menu</mat-icon>
       </button>
-      <md-menu #menu="mdMenu">
-        <button md-menu-item *ngIf="resource && resource.enabled" (click)="disableResource()">
-          <md-icon>not_interested</md-icon>
+      <mat-menu #menu="matMenu">
+        <button mat-menu-item *ngIf="resource && resource.enabled" (click)="disableResource()">
+          <mat-icon>not_interested</mat-icon>
           <span>Disable this Resource</span>
         </button>
-        <button md-menu-item *ngIf="resource && !resource.enabled" (click)="enableResource()">
-          <md-icon>play_circle_outline</md-icon>
+        <button mat-menu-item *ngIf="resource && !resource.enabled" (click)="enableResource()">
+          <mat-icon>play_circle_outline</mat-icon>
           <span>Enable this Resource</span>
         </button>
-        <button md-menu-item *ngIf="false" (click)="removeResource()">
-          <md-icon>delete</md-icon>
+        <button mat-menu-item *ngIf="false" (click)="removeResource()">
+          <mat-icon>delete</mat-icon>
           <span>REMOVE this Resource</span>
         </button>
-      </md-menu>
-    </md-toolbar-row>
-  </md-toolbar>
-  <nav md-tab-nav-bar>
-    <a md-tab-link
+      </mat-menu>
+    </mat-toolbar-row>
+  </mat-toolbar>
+  <nav mat-tab-nav-bar>
+    <a mat-tab-link
       *ngFor="let tabLink of tabLinks"
       [routerLink]="tabLink.link"
       routerLinkActive #rla="routerLinkActive"

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-detail/resource-detail.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-detail/resource-detail.component.scss b/helix-front/client/app/resource/resource-detail/resource-detail.component.scss
index 265278b..bb71765 100644
--- a/helix-front/client/app/resource/resource-detail/resource-detail.component.scss
+++ b/helix-front/client/app/resource/resource-detail/resource-detail.component.scss
@@ -8,8 +8,6 @@
 
 .information {
   .mat-spinner {
-    width: 30px;
-    height: 30px;
     margin: 0 20px;
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-list/resource-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-list/resource-list.component.html b/helix-front/client/app/resource/resource-list/resource-list.component.html
index 5348c5a..8b61d00 100644
--- a/helix-front/client/app/resource/resource-list/resource-list.component.html
+++ b/helix-front/client/app/resource/resource-list/resource-list.component.html
@@ -2,8 +2,8 @@
   <ngx-datatable
     #resourcesTable
     class="material"
-    [headerHeight]="rowHeight"
-    rowHeight="auto"
+    [headerHeight]="headerHeight"
+    [rowHeight]="rowHeight"
     columnMode="force"
     [footerHeight]="rowHeight"
     [rows]="resources"
@@ -15,13 +15,13 @@
       *ngIf="!isForInstance"
       name="Status"
       prop="alive"
-      [width]="85"
+      [width]="88"
       [resizeable]="false"
       [draggable]="false"
       [canAutoResize]="false">
       <ng-template let-value="value" ngx-datatable-cell-template>
-        <md-icon *ngIf="value" color="primary">lens</md-icon>
-        <md-icon *ngIf="!value" color="warn" mdTooltip="The resource is offline.">panorama_fish_eye</md-icon>
+        <mat-icon *ngIf="value" color="primary">lens</mat-icon>
+        <mat-icon *ngIf="!value" color="warn" matTooltip="The resource is offline.">panorama_fish_eye</mat-icon>
       </ng-template>
     </ngx-datatable-column>
     <ngx-datatable-column
@@ -31,8 +31,8 @@
      [sortable]="false"
      [draggable]="false"
      [canAutoResize]="false">
-     <ng-template let-row="row" ngx-datatable-cell-template>
-       <md-icon>{{ row.$$expanded ? 'expand_more' : 'chevron_right' }}</md-icon>
+     <ng-template let-expanded="expanded" ngx-datatable-cell-template>
+       <mat-icon>{{ expanded ? 'expand_more' : 'chevron_right' }}</mat-icon>
      </ng-template>
     </ngx-datatable-column>
     <ngx-datatable-column name="Name"></ngx-datatable-column>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource-list/resource-list.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource-list/resource-list.component.ts b/helix-front/client/app/resource/resource-list/resource-list.component.ts
index 4c72cf3..fb9db0a 100644
--- a/helix-front/client/app/resource/resource-list/resource-list.component.ts
+++ b/helix-front/client/app/resource/resource-list/resource-list.component.ts
@@ -1,12 +1,14 @@
 import { Component, OnInit, ViewChild } from '@angular/core';
 import { Router, ActivatedRoute } from '@angular/router';
+import { Observable } from 'rxjs/Rx';
+
+import * as _ from 'lodash';
 
+import { Settings } from '../../core/settings';
 import { Resource } from '../shared/resource.model';
 import { ResourceService } from '../shared/resource.service';
 import { WorkflowService } from '../../workflow/shared/workflow.service';
 import { HelperService } from '../../shared/helper.service';
-import { Observable } from 'rxjs/Rx';
-import * as _ from 'lodash';
 
 @Component({
   selector: 'hi-resource-list',
@@ -20,7 +22,8 @@ export class ResourceListComponent implements OnInit {
   table: any;
 
   isForInstance = false;
-  rowHeight = 40;
+  headerHeight = Settings.tableHeaderHeight;
+  rowHeight = Settings.tableRowHeight;
   resources: Resource[];
   isLoading = true;
   clusterName: string;

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/resource/resource.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/resource.module.ts b/helix-front/client/app/resource/resource.module.ts
index b8051cd..f818e86 100644
--- a/helix-front/client/app/resource/resource.module.ts
+++ b/helix-front/client/app/resource/resource.module.ts
@@ -1,7 +1,5 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
-import { RouterModule } from '@angular/router';
-import { MaterialModule } from '@angular/material';
 
 import { NgxDatatableModule } from '@swimlane/ngx-datatable';
 import { NgxJsonViewerModule } from 'ngx-json-viewer';
@@ -19,8 +17,6 @@ import { ResourceNodeViewerComponent } from './resource-node-viewer/resource-nod
 @NgModule({
   imports: [
     CommonModule,
-    RouterModule,
-    MaterialModule,
     NgxDatatableModule,
     NgxJsonViewerModule,
     SharedModule

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/data-table/data-table.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/data-table/data-table.component.html b/helix-front/client/app/shared/data-table/data-table.component.html
index e665361..f396c97 100644
--- a/helix-front/client/app/shared/data-table/data-table.component.html
+++ b/helix-front/client/app/shared/data-table/data-table.component.html
@@ -15,8 +15,8 @@
     [draggable]="false"
     [canAutoResize]="false">
     <ng-template let-row="row" ngx-datatable-cell-template>
-      <button md-icon-button mdTooltip="Click to delete" (click)="onDelete(row)">
-        <md-icon>delete_forever</md-icon>
+      <button mat-icon-button matTooltip="Click to delete" (click)="onDelete(row)">
+        <mat-icon>delete_forever</mat-icon>
       </button>
     </ng-template>
   </ngx-datatable-column>
@@ -44,8 +44,8 @@
       let-pageSize="pageSize"
       let-curPage="curPage">
       <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
-        <button md-button *ngIf="insertable" (click)="onCreate()">
-          <md-icon>add</md-icon>
+        <button mat-button *ngIf="insertable" (click)="onCreate()">
+          <mat-icon>add</mat-icon>
           Add new entry
         </button>
         <section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/data-table/data-table.component.spec.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/data-table/data-table.component.spec.ts b/helix-front/client/app/shared/data-table/data-table.component.spec.ts
index c99d516..b6bff79 100644
--- a/helix-front/client/app/shared/data-table/data-table.component.spec.ts
+++ b/helix-front/client/app/shared/data-table/data-table.component.spec.ts
@@ -1,8 +1,7 @@
 import { async, ComponentFixture, TestBed } from '@angular/core/testing';
 import { NO_ERRORS_SCHEMA } from '@angular/core';
-import { MaterialModule } from '@angular/material';
-import { NoopAnimationsModule } from '@angular/platform-browser/animations';
 
+import { TestingModule } from '../../../testing/testing.module';
 import { DataTableComponent } from './data-table.component';
 
 describe('DataTableComponent', () => {
@@ -12,8 +11,7 @@ describe('DataTableComponent', () => {
   beforeEach(async(() => {
     TestBed.configureTestingModule({
       imports: [
-        MaterialModule,
-        NoopAnimationsModule
+        TestingModule
       ],
       declarations: [ DataTableComponent ],
       schemas: [

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/data-table/data-table.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/data-table/data-table.component.ts b/helix-front/client/app/shared/data-table/data-table.component.ts
index e92ec5f..dffae92 100644
--- a/helix-front/client/app/shared/data-table/data-table.component.ts
+++ b/helix-front/client/app/shared/data-table/data-table.component.ts
@@ -1,5 +1,5 @@
 import { Component, OnInit, Input, Output, EventEmitter } from '@angular/core';
-import { MdDialog } from '@angular/material';
+import { MatDialog } from '@angular/material';
 
 import { Settings } from '../../core/settings';
 import { InputDialogComponent } from '../dialog/input-dialog/input-dialog.component';
@@ -25,7 +25,7 @@ export class DataTableComponent implements OnInit {
   rowHeight = Settings.tableRowHeight;
 
   constructor(
-    protected dialog: MdDialog
+    protected dialog: MatDialog
   ) { }
 
   ngOnInit() {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/detail-header/detail-header.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/detail-header/detail-header.component.html b/helix-front/client/app/shared/detail-header/detail-header.component.html
index da4ea4f..0a735dc 100644
--- a/helix-front/client/app/shared/detail-header/detail-header.component.html
+++ b/helix-front/client/app/shared/detail-header/detail-header.component.html
@@ -1,11 +1,11 @@
-<h4>
+<h4 fxLayout="row" fxLayoutAlign="center center">
   <span [ngClass]="{'secondary': isSecondary()}">{{ cluster }}</span>
-  <md-icon *ngIf="isSecondary()" class="secondary">navigate_next</md-icon>
+  <mat-icon *ngIf="isSecondary()" class="secondary">navigate_next</mat-icon>
   <span *ngIf="controller">{{ controller }}</span>
   <span *ngIf="instance">{{ instance }}</span>
   <span *ngIf="resource">{{ resource }}</span>
   <span *ngIf="workflow">{{ workflow }}</span>
-  <span class="tag">
+  <span class="tag" fxLayout="row" fxLayoutAlign="center center">
     <span [ngClass]="getTag()">{{ getTag() }}</span>
     <!-- for testing purpose only
     <span class="cluster">Cluster</span>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/detail-header/detail-header.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/detail-header/detail-header.component.scss b/helix-front/client/app/shared/detail-header/detail-header.component.scss
index d2f8d6b..e17143d 100644
--- a/helix-front/client/app/shared/detail-header/detail-header.component.scss
+++ b/helix-front/client/app/shared/detail-header/detail-header.component.scss
@@ -8,14 +8,13 @@
   padding-left: 10px;
 
   span {
-    display: inline-block;
     color: #fff;
     font-size: 12px;
+    line-height: 12px;
     text-transform: uppercase;
     font-weight: normal;
     border-radius: 24px;
-    padding: 4px 8px;
-    vertical-align: text-bottom;; // ?
+    padding: 6px 8px;
   }
 
   .cluster {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.html b/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.html
index 3195db5..d20aac2 100644
--- a/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.html
+++ b/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.html
@@ -1,2 +1,2 @@
-<h1 md-dialog-title>{{ title }}</h1>
-<div md-dialog-content>{{ message }}</div>
+<h1 mat-dialog-title>{{ title }}</h1>
+<div mat-dialog-content>{{ message }}</div>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.ts b/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.ts
index 54c5643..d276e88 100644
--- a/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.ts
+++ b/helix-front/client/app/shared/dialog/alert-dialog/alert-dialog.component.ts
@@ -1,5 +1,5 @@
 import { Component, OnInit, Inject } from '@angular/core';
-import { MD_DIALOG_DATA } from '@angular/material';
+import { MAT_DIALOG_DATA } from '@angular/material';
 
 @Component({
   selector: 'hi-alert-dialog',
@@ -12,7 +12,7 @@ export class AlertDialogComponent implements OnInit {
   message: string;
 
   constructor(
-    @Inject(MD_DIALOG_DATA) protected data: any
+    @Inject(MAT_DIALOG_DATA) protected data: any
   ) { }
 
   ngOnInit() {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.html b/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.html
index c783641..f4aab6c 100644
--- a/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.html
+++ b/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.html
@@ -1,12 +1,12 @@
 <form (ngSubmit)="onCancel()" #inputForm="ngForm">
-  <h1 md-dialog-title>{{ title }}</h1>
-  <div md-dialog-content>
+  <h1 mat-dialog-title>{{ title }}</h1>
+  <div mat-dialog-content>
     <section>
       {{ message }}
     </section>
   </div>
-  <div md-dialog-actions>
-    <button md-button type="submit" color="primary" [disabled]="!inputForm.form.valid">Cancel</button>
-    <button md-button type="button" (click)="onConfirm()">Continue</button>
+  <div mat-dialog-actions>
+    <button mat-button type="submit" color="primary" [disabled]="!inputForm.form.valid">Cancel</button>
+    <button mat-button type="button" (click)="onConfirm()">Continue</button>
   </div>
 </form>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.ts b/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.ts
index 4db1198..6f4ec8d 100644
--- a/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.ts
+++ b/helix-front/client/app/shared/dialog/confirm-dialog/confirm-dialog.component.ts
@@ -1,6 +1,6 @@
 import { Component, OnInit, Inject } from '@angular/core';
-import { MdDialogRef } from '@angular/material';
-import { MD_DIALOG_DATA } from '@angular/material';
+import { MatDialogRef } from '@angular/material';
+import { MAT_DIALOG_DATA } from '@angular/material';
 
 @Component({
   selector: 'hi-confirm-dialog',
@@ -13,8 +13,8 @@ export class ConfirmDialogComponent implements OnInit {
   message: string;
 
   constructor(
-    @Inject(MD_DIALOG_DATA) protected data: any,
-    protected dialogRef: MdDialogRef<ConfirmDialogComponent>
+    @Inject(MAT_DIALOG_DATA) protected data: any,
+    protected dialogRef: MatDialogRef<ConfirmDialogComponent>
   ) { }
 
   ngOnInit() {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.html b/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.html
index 1086458..bf4cd78 100644
--- a/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.html
+++ b/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.html
@@ -1,29 +1,29 @@
 <form (ngSubmit)="onSubmit()" #inputForm="ngForm">
-  <h1 md-dialog-title>{{ title }}</h1>
-  <div md-dialog-content>
+  <h1 mat-dialog-title>{{ title }}</h1>
+  <div mat-dialog-content>
     <section>
       {{ message }}
     </section>
     <section *ngFor="let name of getKeys(values)">
       <section *ngIf="values[name].type === 'boolean'">
         {{ values[name].label }}:
-        <md-slide-toggle
+        <mat-slide-toggle
           [name]="name"
           [(ngModel)]="values[name].value">
           {{ values[name].value ? 'True' : 'False' }}
-        </md-slide-toggle>
+        </mat-slide-toggle>
       </section>
-      <md-input-container *ngIf="values[name].type !== 'boolean'">
-        <input mdInput
+      <mat-form-field *ngIf="values[name].type !== 'boolean'">
+        <input matInput
           [name]="name"
           [(ngModel)]="values[name].value"
           [placeholder]="values[name].label"
           required>
-      </md-input-container>
+      </mat-form-field>
     </section>
   </div>
-  <div md-dialog-actions>
-    <button md-button type="submit" color="primary" [disabled]="!inputForm.form.valid">OK</button>
-    <button md-button type="button" (click)="onCancel()">Cancel</button>
+  <div mat-dialog-actions>
+    <button mat-button type="submit" color="primary" [disabled]="!inputForm.form.valid">OK</button>
+    <button mat-button type="button" (click)="onCancel()">Cancel</button>
   </div>
 </form>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.ts b/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.ts
index 5934997..d098008 100644
--- a/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.ts
+++ b/helix-front/client/app/shared/dialog/input-dialog/input-dialog.component.ts
@@ -1,6 +1,6 @@
 import { Component, OnInit, Inject } from '@angular/core';
-import { MdDialogRef } from '@angular/material';
-import { MD_DIALOG_DATA } from '@angular/material';
+import { MatDialogRef } from '@angular/material';
+import { MAT_DIALOG_DATA } from '@angular/material';
 
 @Component({
   selector: 'hi-input-dialog',
@@ -14,8 +14,8 @@ export class InputDialogComponent implements OnInit {
   values: any[];
 
   constructor(
-    @Inject(MD_DIALOG_DATA) protected data: any,
-    protected dialogRef: MdDialogRef<InputDialogComponent>
+    @Inject(MAT_DIALOG_DATA) protected data: any,
+    protected dialogRef: MatDialogRef<InputDialogComponent>
   ) { }
 
   ngOnInit() {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/disabled-label/disabled-label.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/disabled-label/disabled-label.component.scss b/helix-front/client/app/shared/disabled-label/disabled-label.component.scss
index 9b10103..0b492f1 100644
--- a/helix-front/client/app/shared/disabled-label/disabled-label.component.scss
+++ b/helix-front/client/app/shared/disabled-label/disabled-label.component.scss
@@ -4,9 +4,9 @@
   margin-left: 10px;
   padding: 4px 8px;
   font-size: 12px;
+  line-height: 12px;
   border-radius: 4px;
   border: 1px solid mat-color(mat-palette($mat-red), 900);
   background-color: mat-color(mat-palette($mat-red), darker);
   color: rgb(255, 255, 255);
-  line-height: 26px;
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/helper.service.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/helper.service.ts b/helix-front/client/app/shared/helper.service.ts
index 2bc7cb4..29c10ec 100644
--- a/helix-front/client/app/shared/helper.service.ts
+++ b/helix-front/client/app/shared/helper.service.ts
@@ -1,5 +1,5 @@
 import { Injectable } from '@angular/core';
-import { MdDialog, MdSnackBar } from '@angular/material';
+import { MatDialog, MatSnackBar } from '@angular/material';
 
 import { AlertDialogComponent } from './dialog/alert-dialog/alert-dialog.component';
 import { ConfirmDialogComponent } from './dialog/confirm-dialog/confirm-dialog.component';
@@ -8,8 +8,8 @@ import { ConfirmDialogComponent } from './dialog/confirm-dialog/confirm-dialog.c
 export class HelperService {
 
   constructor(
-    protected snackBar: MdSnackBar,
-    protected dialog: MdDialog
+    protected snackBar: MatSnackBar,
+    protected dialog: MatDialog
   ) { }
 
   showError(message: string) {

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/input-inline/input-inline.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/input-inline/input-inline.component.html b/helix-front/client/app/shared/input-inline/input-inline.component.html
index fdabc36..21ea2c9 100644
--- a/helix-front/client/app/shared/input-inline/input-inline.component.html
+++ b/helix-front/client/app/shared/input-inline/input-inline.component.html
@@ -1,6 +1,6 @@
 <section>
-  <md-input-container *ngIf="editing" class="full-width {{ hasError() ? 'error' : ''}}">
-    <input mdInput
+  <mat-form-field *ngIf="editing" class="full-width {{ hasError() ? 'error' : ''}}">
+    <input matInput
       #inputControl
       [min]="min"
       [max]="max"
@@ -15,13 +15,13 @@
       [(ngModel)]="value"
       [type]="type"
       [placeholder]="label" />
-    <md-hint *ngIf="hasError()" align="start">{{ errorLabel }}</md-hint>
-    <md-hint align="end">press ESC to cancel</md-hint>
-  </md-input-container>
+    <mat-hint *ngIf="hasError()" align="start">{{ errorLabel }}</mat-hint>
+    <mat-hint align="end">press ESC to cancel</mat-hint>
+  </mat-form-field>
   <section *ngIf="!editing">
     <div
       class="inline-edit {{hasError() ? 'error': ''}}"
-      [md-tooltip]="editLabel"
+      [matTooltip]="editLabel"
       (click)="edit(value)"
       (focus)="edit(value)"
       tabindex="0">

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/key-value-pairs/key-value-pairs.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/key-value-pairs/key-value-pairs.component.scss b/helix-front/client/app/shared/key-value-pairs/key-value-pairs.component.scss
index a70ad51..af8355f 100644
--- a/helix-front/client/app/shared/key-value-pairs/key-value-pairs.component.scss
+++ b/helix-front/client/app/shared/key-value-pairs/key-value-pairs.component.scss
@@ -1,5 +1,6 @@
 .groups {
   font-size: 14px;
+  line-height: 14px;
 
   .column {
     padding-left: 40px;
@@ -7,6 +8,7 @@
 
   .name {
     font-size: 12px;
+    line-height: 12px;
     color: rgba(0, 0, 0, .54);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/material.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/material.module.ts b/helix-front/client/app/shared/material.module.ts
new file mode 100644
index 0000000..c082d0b
--- /dev/null
+++ b/helix-front/client/app/shared/material.module.ts
@@ -0,0 +1,66 @@
+import { NgModule } from '@angular/core';
+import { BrowserAnimationsModule } from '@angular/platform-browser/animations';
+import {
+  MatButtonModule,
+  MatButtonToggleModule,
+  MatCardModule,
+  MatCheckboxModule,
+  MatToolbarModule,
+  MatTooltipModule,
+  MatDialogModule,
+  MatSnackBarModule,
+  MatSlideToggleModule,
+  MatInputModule,
+  MatIconModule,
+  MatProgressBarModule,
+  MatProgressSpinnerModule,
+  MatSidenavModule,
+  MatListModule,
+  MatMenuModule,
+  MatTabsModule
+} from '@angular/material';
+import 'hammerjs';
+
+@NgModule({
+  imports: [
+    BrowserAnimationsModule,
+    MatButtonModule,
+    MatButtonToggleModule,
+    MatCardModule,
+    MatCheckboxModule,
+    MatToolbarModule,
+    MatTooltipModule,
+    MatDialogModule,
+    MatSnackBarModule,
+    MatSlideToggleModule,
+    MatInputModule,
+    MatIconModule,
+    MatProgressBarModule,
+    MatProgressSpinnerModule,
+    MatSidenavModule,
+    MatListModule,
+    MatMenuModule,
+    MatTabsModule
+  ],
+  exports: [
+    BrowserAnimationsModule,
+    MatButtonModule,
+    MatButtonToggleModule,
+    MatCardModule,
+    MatCheckboxModule,
+    MatToolbarModule,
+    MatTooltipModule,
+    MatDialogModule,
+    MatSnackBarModule,
+    MatSlideToggleModule,
+    MatInputModule,
+    MatIconModule,
+    MatProgressBarModule,
+    MatProgressSpinnerModule,
+    MatSidenavModule,
+    MatListModule,
+    MatMenuModule,
+    MatTabsModule
+  ]
+})
+export class MaterialModule { }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/node-viewer/node-viewer.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/node-viewer/node-viewer.component.html b/helix-front/client/app/shared/node-viewer/node-viewer.component.html
index 006761b..25777e3 100644
--- a/helix-front/client/app/shared/node-viewer/node-viewer.component.html
+++ b/helix-front/client/app/shared/node-viewer/node-viewer.component.html
@@ -1,46 +1,46 @@
 <section class="node-viewer" fxLayout="column" fxLayoutAlign="center center" fxLayoutGap="10px">
-  <md-progress-bar *ngIf="loadingIndicator" mode="indeterminate"></md-progress-bar>
-  <md-button-toggle-group #group="mdButtonToggleGroup" value="table">
-    <md-button-toggle value="table">
+  <mat-progress-bar *ngIf="loadingIndicator" mode="indeterminate"></mat-progress-bar>
+  <mat-button-toggle-group #group="matButtonToggleGroup" value="table">
+    <mat-button-toggle value="table">
       Table View
-    </md-button-toggle>
-    <md-button-toggle value="json">
+    </mat-button-toggle>
+    <mat-button-toggle value="json">
       JSON View
-    </md-button-toggle>
-  </md-button-toggle-group>
+    </mat-button-toggle>
+  </mat-button-toggle-group>
   <section class="viewer" [ngSwitch]="group.value" fxFlexFill>
     <ngx-json-viewer *ngSwitchCase="'json'" [json]="obj"></ngx-json-viewer>
     <section *ngSwitchCase="'table'">
 
-      <!-- TODO vxu: use md-simple-table when it's available -->
+      <!-- TODO vxu: use mat-simple-table when it's available -->
 
       <section fxLayout="row" fxLayoutAlign="center center">
         <span fxFlex="1 1 auto"></span>
-        <md-icon>search</md-icon>
-        <md-input-container>
-          <input mdInput
+        <mat-icon>search</mat-icon>
+        <mat-form-field class="search-form-field">
+          <input matInput
             placeholder="Type to filter the fields..."
             (keyup)="updateFilter($event)"/>
-        </md-input-container>
+        </mat-form-field>
         <span fxFlex="1 1 auto"></span>
-        <button md-button
+        <button mat-button
           *ngIf="unlockable"
           (click)="editable = !editable"
-          [mdTooltip]="editable ? 'Click to prevent further changes' : 'Click to make changes'">
-          <md-icon>{{ editable ? 'lock_open' : 'lock' }}</md-icon>
+          [matTooltip]="editable ? 'Click to prevent further changes' : 'Click to make changes'">
+          <mat-icon>{{ editable ? 'lock_open' : 'lock' }}</mat-icon>
           {{ editable ? 'Unlocked' : 'Locked' }}
         </button>
       </section>
 
-      <md-card>
-        <md-card-header>
-          <md-card-title>
+      <mat-card>
+        <mat-card-header>
+          <mat-card-title>
             Simple Fields
             <span *ngIf="simpleConfigs.length == 0">is empty.</span>
             <span *ngIf="keyword" class="primary">(filtered)</span>
-          </md-card-title>
-        </md-card-header>
-        <md-card-content>
+          </mat-card-title>
+        </mat-card-header>
+        <mat-card-content>
           <hi-data-table
             *ngIf="simpleConfigs.length || editable"
             [rows]="simpleConfigs"
@@ -52,25 +52,25 @@
             (create)="created('simple', $event)"
             (delete)="onDelete('simple', $event.row)">
           </hi-data-table>
-        </md-card-content>
-      </md-card>
+        </mat-card-content>
+      </mat-card>
 
-      <md-card>
-        <md-card-header>
-          <md-card-title>
+      <mat-card>
+        <mat-card-header>
+          <mat-card-title>
             List Fields
             <span *ngIf="listConfigs.length == 0">is empty.</span>
             <span *ngIf="keyword" class="primary">(filtered)</span>
-          </md-card-title>
-        </md-card-header>
-        <md-card-content>
+          </mat-card-title>
+        </mat-card-header>
+        <mat-card-content>
           <ngx-datatable
             *ngIf="listConfigs.length || editable"
             #listTable
             class="material"
-            [headerHeight]="rowHeight"
+            [headerHeight]="headerHeight"
             rowHeight="auto"
-            [footerHeight]="rowHeight"
+            [footerHeight]="headerHeight"
             columnMode="force"
             [rows]="listConfigs"
             [sorts]="sorts"
@@ -82,8 +82,8 @@
               [draggable]="false"
               [canAutoResize]="false">
               <ng-template let-row="row" ngx-datatable-cell-template>
-                <button md-icon-button class="delete-button" mdTooltip="Click to delete" (click)="beforeDelete('list', row)">
-                  <md-icon>delete_forever</md-icon>
+                <button mat-icon-button class="delete-button" matTooltip="Click to delete" (click)="beforeDelete('list', row)">
+                  <mat-icon>delete_forever</mat-icon>
                 </button>
               </ng-template>
             </ngx-datatable-column>
@@ -109,8 +109,8 @@
                 let-pageSize="pageSize"
                 let-curPage="curPage">
                 <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
-                  <button md-button *ngIf="editable" (click)="onCreate('list')">
-                    <md-icon>add</md-icon>
+                  <button mat-button *ngIf="editable" (click)="onCreate('list')">
+                    <mat-icon>add</mat-icon>
                     Add new entry
                   </button>
                   <section>
@@ -133,25 +133,25 @@
               </ng-template>
             </ngx-datatable-footer>
           </ngx-datatable>
-        </md-card-content>
-      </md-card>
+        </mat-card-content>
+      </mat-card>
 
-      <md-card>
-        <md-card-header>
-          <md-card-title>
+      <mat-card>
+        <mat-card-header>
+          <mat-card-title>
             Map Fields
             <span *ngIf="mapConfigs.length == 0">is empty.</span>
             <span *ngIf="keyword" class="primary">(filtered)</span>
-          </md-card-title>
-        </md-card-header>
-        <md-card-content>
+          </mat-card-title>
+        </mat-card-header>
+        <mat-card-content>
           <ngx-datatable
             *ngIf="mapConfigs.length || editable"
             #mapTable
             class="material"
-            [headerHeight]="rowHeight"
+            [headerHeight]="headerHeight"
             rowHeight="auto"
-            [footerHeight]="rowHeight"
+            [footerHeight]="headerHeight"
             columnMode="force"
             [rows]="mapConfigs"
             [sorts]="sorts"
@@ -163,8 +163,8 @@
               [draggable]="false"
               [canAutoResize]="false">
               <ng-template let-row="row" ngx-datatable-cell-template>
-                <button md-icon-button class="delete-button" mdTooltip="Click to delete" (click)="beforeDelete('map', row)">
-                  <md-icon>delete_forever</md-icon>
+                <button mat-icon-button class="delete-button" matTooltip="Click to delete" (click)="beforeDelete('map', row)">
+                  <mat-icon>delete_forever</mat-icon>
                 </button>
               </ng-template>
             </ngx-datatable-column>
@@ -190,8 +190,8 @@
                 let-pageSize="pageSize"
                 let-curPage="curPage">
                 <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
-                  <button md-button *ngIf="editable" (click)="onCreate('map')">
-                    <md-icon>add</md-icon>
+                  <button mat-button *ngIf="editable" (click)="onCreate('map')">
+                    <mat-icon>add</mat-icon>
                     Add new entry
                   </button>
                   <section>
@@ -214,8 +214,8 @@
               </ng-template>
             </ngx-datatable-footer>
           </ngx-datatable>
-        </md-card-content>
-      </md-card>
+        </mat-card-content>
+      </mat-card>
 
     </section>
   </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/node-viewer/node-viewer.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/node-viewer/node-viewer.component.scss b/helix-front/client/app/shared/node-viewer/node-viewer.component.scss
index 9fd5d3c..a1e27f2 100644
--- a/helix-front/client/app/shared/node-viewer/node-viewer.component.scss
+++ b/helix-front/client/app/shared/node-viewer/node-viewer.component.scss
@@ -9,12 +9,12 @@
   color: mat-color($hi-primary);
 }
 
-md-input-container {
+.search-form-field {
   width: 300px;
   padding: 10px 0 0 5px;
 }
 
-md-card {
+mat-card {
   margin-bottom: 10px;
 }
 


[46/50] [abbrv] helix git commit: Add monitor to ZkClient to monitor the pending callbacks.

Posted by jx...@apache.org.
Add monitor to ZkClient to monitor the pending callbacks.


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

Branch: refs/heads/master
Commit: fde1a6a441a1fcfac6a8c6f1849d54bb34291643
Parents: b3ecd2a
Author: Lei Xia <lx...@linkedin.com>
Authored: Thu Dec 21 10:19:32 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:20 2018 -0800

----------------------------------------------------------------------
 .../apache/helix/manager/zk/zookeeper/ZkClient.java  |  2 +-
 .../helix/manager/zk/zookeeper/ZkEventThread.java    |  8 ++++++--
 .../helix/monitoring/mbeans/ZkClientMonitor.java     | 15 +++++++++++++++
 .../monitoring/mbeans/ZkClientMonitorMBean.java      |  1 +
 4 files changed, 23 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/fde1a6a4/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
index 4748d6e..8e0a379 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
@@ -35,7 +35,6 @@ import org.I0Itec.zkclient.exception.ZkInterruptedException;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.I0Itec.zkclient.exception.ZkTimeoutException;
-import org.I0Itec.zkclient.serialize.SerializableSerializer;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
@@ -102,6 +101,7 @@ public class ZkClient implements Watcher {
           .isEmpty()) {
         _monitor =
             new ZkClientMonitor(monitorType, monitorKey, monitorInstanceName, monitorRootPathOnly);
+        _monitor.setZkEventThread(_eventThread);
       } else {
         LOG.info("ZkClient monitor key or type is not provided. Skip monitoring.");
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/fde1a6a4/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
index dcf7019..968f7f3 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkEventThread.java
@@ -26,10 +26,10 @@ import org.slf4j.LoggerFactory;
  * it is waiting for something). {@link ZkClient} would then for instance not be able to maintain it's connection state
  * anymore.
  */
-class ZkEventThread extends Thread {
+public class ZkEventThread extends Thread {
   private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
 
-  private BlockingQueue<ZkEvent> _events = new LinkedBlockingQueue<ZkEvent>();
+  private BlockingQueue<ZkEvent> _events = new LinkedBlockingQueue<>();
 
   private static AtomicInteger _eventId = new AtomicInteger(0);
 
@@ -82,4 +82,8 @@ class ZkEventThread extends Thread {
       _events.add(event);
     }
   }
+
+  public int getPendingEventsCount() {
+    return _events.size();
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/fde1a6a4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
index 6cdf6e7..3e4022f 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
@@ -26,6 +26,7 @@ import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.helix.manager.zk.zookeeper.ZkEventThread;
 
 public class ZkClientMonitor implements ZkClientMonitorMBean {
   public static final String MONITOR_TYPE = "Type";
@@ -41,6 +42,7 @@ public class ZkClientMonitor implements ZkClientMonitorMBean {
 
   private long _stateChangeEventCounter;
   private long _dataChangeEventCounter;
+  private ZkEventThread _zkEventThread;
 
   private Map<ZkClientPathMonitor.PredefinedPath, ZkClientPathMonitor> _zkClientPathMonitorMap =
       new ConcurrentHashMap<>();
@@ -68,6 +70,10 @@ public class ZkClientMonitor implements ZkClientMonitorMBean {
     }
   }
 
+  public void setZkEventThread(ZkEventThread zkEventThread) {
+    _zkEventThread = zkEventThread;
+  }
+
   protected static ObjectName getObjectName(String monitorType, String monitorKey,
       String monitorInstanceName) throws MalformedObjectNameException {
     return MBeanRegistrar
@@ -109,6 +115,15 @@ public class ZkClientMonitor implements ZkClientMonitorMBean {
     return _dataChangeEventCounter;
   }
 
+  @Override
+  public long getPendingCallbackGauge() {
+    if (_zkEventThread != null) {
+      return _zkEventThread.getPendingEventsCount();
+    }
+
+    return -1;
+  }
+
   private void record(String path, int bytes, long latencyMilliSec, boolean isFailure,
       boolean isRead) {
     for (ZkClientPathMonitor.PredefinedPath predefinedPath : ZkClientPathMonitor.PredefinedPath

http://git-wip-us.apache.org/repos/asf/helix/blob/fde1a6a4/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitorMBean.java
index a260f71..17bfa25 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitorMBean.java
@@ -24,4 +24,5 @@ import org.apache.helix.monitoring.SensorNameProvider;
 public interface ZkClientMonitorMBean extends SensorNameProvider {
   long getStateChangeEventCounter();
   long getDataChangeEventCounter();
+  long getPendingCallbackGauge();
 }


[38/50] [abbrv] helix git commit: Remove all legacy dependencies on "controller" as the controller nodes' name.

Posted by jx...@apache.org.
Remove all legacy dependencies on "controller" as the controller nodes' name.

Should check instance type for the instance information instead of relying on the name. Name can be configured as any string.


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

Branch: refs/heads/master
Commit: 5ffab62fc96007cb636a6da97be0e62ddb63cd91
Parents: 89089b4
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Tue Nov 14 00:38:44 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:52 2018 -0800

----------------------------------------------------------------------
 .../DefaultSchedulerMessageHandlerFactory.java  |   4 +-
 .../messaging/DefaultMessagingService.java      |   4 +-
 .../handling/HelixStateTransitionHandler.java   |  21 ++-
 .../helix/messaging/handling/HelixTask.java     |  22 +--
 .../messaging/handling/HelixTaskExecutor.java   |  32 ++--
 .../java/org/apache/helix/model/Message.java    |   4 +-
 .../participant/HelixStateMachineEngine.java    |   2 +-
 .../org/apache/helix/util/StatusUpdateUtil.java | 149 ++++++++++++-------
 8 files changed, 141 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
index 51bb5c3..443df9f 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
@@ -80,7 +80,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
           + _timeout + " Ms");
 
       _statusUpdateUtil.logError(_originalMessage, SchedulerAsyncCallback.class, "Task timeout",
-          _manager.getHelixDataAccessor());
+          _manager);
       addSummary(_resultSummaryMap, _originalMessage, _manager, true);
     }
 
@@ -94,7 +94,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       if (this.isDone()) {
         _logger.info("Scheduler msg " + _originalMessage.getMsgId() + " completed");
         _statusUpdateUtil.logInfo(_originalMessage, SchedulerAsyncCallback.class,
-            "Scheduler task completed", _manager.getHelixDataAccessor());
+            "Scheduler task completed", _manager);
         addSummary(_resultSummaryMap, _originalMessage, _manager, false);
       }
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
index e776dc2..fb84fd7 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
@@ -204,7 +204,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
     Message newMessage = new Message(message.getRecord(), id);
     newMessage.setMsgId(id);
     newMessage.setSrcName(_manager.getInstanceName());
-    newMessage.setTgtName("Controller");
+    newMessage.setTgtName(InstanceType.CONTROLLER.name());
     messages.add(newMessage);
     return messages;
   }
@@ -298,7 +298,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
 
       if (_manager.getInstanceType() == InstanceType.CONTROLLER
           || _manager.getInstanceType() == InstanceType.CONTROLLER_PARTICIPANT) {
-        nopMsg.setTgtName("Controller");
+        nopMsg.setTgtName(InstanceType.CONTROLLER.name());
         accessor.setProperty(keyBuilder.controllerMessage(nopMsg.getId()), nopMsg);
       }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
index ece7ac7..9412dde 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
@@ -89,7 +89,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
               + Arrays.toString(Message.Attributes.values());
 
       _statusUpdateUtil.logError(_message, HelixStateTransitionHandler.class, errorMessage,
-          _manager.getHelixDataAccessor());
+          _manager);
       logger.error(errorMessage);
       throw new HelixException(errorMessage);
     }
@@ -119,7 +119,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
               + _message.getTgtName();
 
       _statusUpdateUtil.logError(_message, HelixStateTransitionHandler.class, errorMessage,
-          accessor);
+          _manager);
       logger.error(errorMessage);
       throw new HelixStateMismatchException(errorMessage);
     }
@@ -162,7 +162,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
                                  HelixStateTransitionHandler.class,
                                  e,
                                  "Error when removing " + CurrentState.CurrentStateProperty.REQUESTED_STATE.name() +  " from current state.",
-                                 accessor);
+                                 _manager);
     }
   }
 
@@ -286,7 +286,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
           new StateTransitionError(ErrorType.FRAMEWORK, ErrorCode.ERROR, e);
       _stateModel.rollbackOnError(_message, _notificationContext, error);
       _statusUpdateUtil.logError(_message, HelixStateTransitionHandler.class, e,
-          "Error when update current-state ", accessor);
+          "Error when update current-state ", _manager);
     }
   }
 
@@ -311,15 +311,14 @@ public class HelixStateTransitionHandler extends MessageHandler {
     synchronized (_stateModel) {
       HelixTaskResult taskResult = new HelixTaskResult();
       HelixManager manager = context.getManager();
-      HelixDataAccessor accessor = manager.getHelixDataAccessor();
 
       _statusUpdateUtil.logInfo(message, HelixStateTransitionHandler.class,
-          "Message handling task begin execute", accessor);
+          "Message handling task begin execute", manager);
       message.setExecuteStartTimeStamp(new Date().getTime());
 
       try {
         preHandleMessage();
-        invoke(accessor, context, taskResult, message);
+        invoke(manager, context, taskResult, message);
       } catch (HelixStateMismatchException e) {
         // Simply log error and return from here if State mismatch.
         // The current state of the state model is intact.
@@ -344,7 +343,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
           taskResult.setCancelled(true);
         } else {
           _statusUpdateUtil
-              .logError(message, HelixStateTransitionHandler.class, e, errorMessage, accessor);
+              .logError(message, HelixStateTransitionHandler.class, e, errorMessage, manager);
           taskResult.setSuccess(false);
           taskResult.setMessage(e.toString());
           taskResult.setException(e);
@@ -361,11 +360,11 @@ public class HelixStateTransitionHandler extends MessageHandler {
     }
   }
 
-  private void invoke(HelixDataAccessor accessor, NotificationContext context,
+  private void invoke(HelixManager manager, NotificationContext context,
       HelixTaskResult taskResult, Message message) throws IllegalAccessException,
       InvocationTargetException, InterruptedException, HelixRollbackException {
     _statusUpdateUtil.logInfo(message, HelixStateTransitionHandler.class,
-        "Message handling invoking", accessor);
+        "Message handling invoking", manager);
 
     // by default, we invoke state transition function in state model
     Method methodToInvoke = null;
@@ -417,7 +416,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
       taskResult.setSuccess(false);
 
       _statusUpdateUtil
-          .logError(message, HelixStateTransitionHandler.class, errorMessage, accessor);
+          .logError(message, HelixStateTransitionHandler.class, errorMessage, manager);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
index 2543d81..85665c1 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
@@ -79,7 +79,7 @@ public class HelixTask implements MessageTask {
     logger.info("handling task: " + getTaskId() + " begin, at: " + start);
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
     _statusUpdateUtil.logInfo(_message, HelixTask.class, "Message handling task begin execute",
-        accessor);
+        _manager);
     _message.setExecuteStartTimeStamp(new Date().getTime());
 
     // add a concurrent map to hold currentStateUpdates for sub-messages of a batch-message
@@ -99,7 +99,7 @@ public class HelixTask implements MessageTask {
       taskResult.setInterrupted(true);
 
       _statusUpdateUtil.logError(_message, HelixTask.class, e,
-          "State transition interrupted, timeout:" + _isTimeout, accessor);
+          "State transition interrupted, timeout:" + _isTimeout, _manager);
       logger.info("Message " + _message.getMsgId() + " is interrupted");
     } catch (Exception e) {
       taskResult = new HelixTaskResult();
@@ -110,7 +110,7 @@ public class HelixTask implements MessageTask {
           "Exception while executing a message. " + e + " msgId: " + _message.getMsgId()
               + " type: " + _message.getMsgType();
       logger.error(errorMessage, e);
-      _statusUpdateUtil.logError(_message, HelixTask.class, e, errorMessage, accessor);
+      _statusUpdateUtil.logError(_message, HelixTask.class, e, errorMessage, _manager);
     }
 
     // cancel timeout task
@@ -120,7 +120,7 @@ public class HelixTask implements MessageTask {
     try {
       if (taskResult.isSuccess()) {
         _statusUpdateUtil
-            .logInfo(_message, _handler.getClass(), "Message handling task completed successfully", accessor);
+            .logInfo(_message, _handler.getClass(), "Message handling task completed successfully", _manager);
         logger.info("Message " + _message.getMsgId() + " completed.");
         _executor.getParticipantMonitor().reportProcessedMessage(_message, ParticipantMessageMonitor.ProcessedMessageState.COMPLETED);
       } else {
@@ -134,7 +134,7 @@ public class HelixTask implements MessageTask {
             logger.info("Message timeout, retry count: " + retryCount + " msgId:"
                 + _message.getMsgId());
             _statusUpdateUtil.logInfo(_message, _handler.getClass(),
-                "Message handling task timeout, retryCount:" + retryCount, accessor);
+                "Message handling task timeout, retryCount:" + retryCount, _manager);
             // Notify the handler that timeout happens, and the number of retries left
             // In case timeout happens (time out and also interrupted)
             // we should retry the execution of the message by re-schedule it in
@@ -151,7 +151,7 @@ public class HelixTask implements MessageTask {
           type = null;
           _statusUpdateUtil
               .logInfo(_message, _handler.getClass(), "Cancellation completed successfully",
-                  accessor);
+                  _manager);
           _executor.getParticipantMonitor().reportProcessedMessage(
               _message, ParticipantMessageMonitor.ProcessedMessageState.DISCARDED);
         } else {// logging for errors
@@ -160,7 +160,7 @@ public class HelixTask implements MessageTask {
               "Message execution failed. msgId: " + getTaskId() + ", errorMsg: "
                   + taskResult.getMessage();
           logger.error(errorMsg);
-          _statusUpdateUtil.logError(_message, _handler.getClass(), errorMsg, accessor);
+          _statusUpdateUtil.logError(_message, _handler.getClass(), errorMsg, _manager);
           _executor.getParticipantMonitor().reportProcessedMessage(
               _message, ParticipantMessageMonitor.ProcessedMessageState.FAILED);
         }
@@ -185,7 +185,7 @@ public class HelixTask implements MessageTask {
       String errorMessage =
           "Exception after executing a message, msgId: " + _message.getMsgId() + e;
       logger.error(errorMessage, e);
-      _statusUpdateUtil.logError(_message, HelixTask.class, errorMessage, accessor);
+      _statusUpdateUtil.logError(_message, HelixTask.class, errorMessage, _manager);
     } finally {
       long end = System.currentTimeMillis();
       logger.info("msg: " + _message.getMsgId() + " handling task completed, results:"
@@ -206,7 +206,7 @@ public class HelixTask implements MessageTask {
   private void removeMessageFromZk(HelixDataAccessor accessor, Message message) {
     Builder keyBuilder = accessor.keyBuilder();
     PropertyKey msgKey;
-    if (message.getTgtName().equalsIgnoreCase("controller")) {
+    if (message.getTgtName().equalsIgnoreCase(InstanceType.CONTROLLER.name())) {
       msgKey = keyBuilder.controllerMessage(message.getMsgId());
     } else {
       msgKey = keyBuilder.message(_manager.getInstanceName(), message.getMsgId());
@@ -259,7 +259,7 @@ public class HelixTask implements MessageTask {
     if (_message.getCorrelationId() != null
         && !message.getMsgType().equals(MessageType.TASK_REPLY.name())) {
       logger.info("Sending reply for message " + message.getCorrelationId());
-      _statusUpdateUtil.logInfo(message, HelixTask.class, "Sending reply", accessor);
+      _statusUpdateUtil.logInfo(message, HelixTask.class, "Sending reply", _manager);
 
       taskResult.getTaskResultMap().put("SUCCESS", "" + taskResult.isSuccess());
       taskResult.getTaskResultMap().put("INTERRUPTED", "" + taskResult.isInterrupted());
@@ -280,7 +280,7 @@ public class HelixTask implements MessageTask {
         accessor.setProperty(keyBuilder.controllerMessage(replyMessage.getMsgId()), replyMessage);
       }
       _statusUpdateUtil.logInfo(message, HelixTask.class,
-          "1 msg replied to " + replyMessage.getTgtName(), accessor);
+          "1 msg replied to " + replyMessage.getTgtName(), _manager);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index c0be583..064b6fd 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -398,10 +398,11 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
     String taskId = task.getTaskId();
     Message message = task.getMessage();
     NotificationContext notificationContext = task.getNotificationContext();
+    HelixManager manager = notificationContext.getManager();
 
     try {
       // Check to see if dedicate thread pool for handling state transition messages is configured or provided.
-      updateStateTransitionMessageThreadPool(message, notificationContext.getManager());
+      updateStateTransitionMessageThreadPool(message, manager);
 
       LOG.info("Scheduling message: " + taskId);
       // System.out.println("sched msg: " + message.getPartitionName() + "-"
@@ -409,8 +410,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       // + message.getToState());
 
       _statusUpdateUtil.logInfo(message, HelixTaskExecutor.class,
-          "Message handling task scheduled", notificationContext.getManager()
-              .getHelixDataAccessor());
+          "Message handling task scheduled", manager);
 
       // this sync guarantees that ExecutorService.submit() task and put taskInfo into map are
       // sync'ed
@@ -441,23 +441,19 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
           } else {
             LOG.debug("Message does not have timeout. MsgId: " + task.getTaskId());
           }
-
           _taskMap.put(taskId, new MessageTaskInfo(task, future, timerTask));
 
           LOG.info("Message: " + taskId + " handling task scheduled");
-
           return true;
         } else {
           _statusUpdateUtil.logWarning(message, HelixTaskExecutor.class,
-              "Message handling task already sheduled for " + taskId, notificationContext
-                  .getManager().getHelixDataAccessor());
+              "Message handling task already sheduled for " + taskId, manager);
         }
       }
     } catch (Exception e) {
       LOG.error("Error while executing task. " + message, e);
-
       _statusUpdateUtil.logError(message, HelixTaskExecutor.class, e, "Error while executing task "
-          + e, notificationContext.getManager().getHelixDataAccessor());
+          + e, manager);
     }
     return false;
   }
@@ -480,25 +476,25 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         Future<HelixTaskResult> future = taskInfo.getFuture();
         removeMessageFromTaskAndFutureMap(message);
         _statusUpdateUtil.logInfo(message, HelixTaskExecutor.class, "Canceling task: " + taskId,
-            notificationContext.getManager().getHelixDataAccessor());
+            notificationContext.getManager());
 
         // If the thread is still running it will be interrupted if cancel(true)
         // is called. So state transition callbacks should implement logic to
         // return if it is interrupted.
         if (future.cancel(true)) {
           _statusUpdateUtil.logInfo(message, HelixTaskExecutor.class, "Canceled task: " + taskId,
-              notificationContext.getManager().getHelixDataAccessor());
+              notificationContext.getManager());
           _taskMap.remove(taskId);
           return true;
         } else {
           _statusUpdateUtil.logInfo(message, HelixTaskExecutor.class,
               "fail to cancel task: " + taskId,
-              notificationContext.getManager().getHelixDataAccessor());
+              notificationContext.getManager());
         }
       } else {
         _statusUpdateUtil.logWarning(message, HelixTaskExecutor.class,
             "fail to cancel task: " + taskId + ", future not found",
-            notificationContext.getManager().getHelixDataAccessor());
+            notificationContext.getManager());
       }
     }
     return false;
@@ -807,7 +803,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
                 + message.getMsgId();
         LOG.warn(warningMessage);
         reportAndRemoveMessage(message, accessor, instanceName, ProcessedMessageState.DISCARDED);
-        _statusUpdateUtil.logWarning(message, HelixStateMachineEngine.class, warningMessage, accessor);
+        _statusUpdateUtil.logWarning(message, HelixStateMachineEngine.class, warningMessage, manager);
 
         // Proactively send a session sync message from participant to controller
         // upon session mismatch after a new session is established
@@ -873,7 +869,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         String error =
             "Failed to create message handler for " + message.getMsgId() + ", exception: " + e;
 
-        _statusUpdateUtil.logError(message, HelixStateMachineEngine.class, e, error, accessor);
+        _statusUpdateUtil.logError(message, HelixStateMachineEngine.class, e, error, manager);
 
         message.setMsgState(MessageState.UNPROCESSABLE);
         removeMessageFromZK(accessor, message, instanceName);
@@ -882,7 +878,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         continue;
       }
 
-      markReadMessage(message, changeContext, accessor);
+      markReadMessage(message, changeContext, manager);
       readMsgs.add(message);
 
       // batch creation of all current state meta data
@@ -1003,12 +999,12 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
   }
 
   private void markReadMessage(Message message, NotificationContext context,
-      HelixDataAccessor accessor) {
+      HelixManager manager) {
     message.setMsgState(MessageState.READ);
     message.setReadTimeStamp(new Date().getTime());
     message.setExecuteSessionId(context.getManager().getSessionId());
 
-    _statusUpdateUtil.logInfo(message, HelixStateMachineEngine.class, "New Message", accessor);
+    _statusUpdateUtil.logInfo(message, HelixStateMachineEngine.class, "New Message", manager);
   }
 
   public MessageHandler createMessageHandler(Message message, NotificationContext changeContext) {

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/model/Message.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java
index d987c54..9f0054c 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Message.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Message.java
@@ -644,7 +644,7 @@ public class Message extends HelixProperty {
     replyMessage.setMsgState(MessageState.NEW);
     replyMessage.setSrcName(instanceName);
     if (srcMessage.getSrcInstanceType() == InstanceType.CONTROLLER) {
-      replyMessage.setTgtName("Controller");
+      replyMessage.setTgtName(InstanceType.CONTROLLER.name());
     } else {
       replyMessage.setTgtName(srcMessage.getMsgSrc());
     }
@@ -849,7 +849,7 @@ public class Message extends HelixProperty {
    * @return true if this is a controller message, false otherwise
    */
   public boolean isControlerMsg() {
-    return getTgtName().equalsIgnoreCase("controller");
+    return getTgtName().equalsIgnoreCase(InstanceType.CONTROLLER.name());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java b/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java
index cd424f8..3a13180 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/HelixStateMachineEngine.java
@@ -127,7 +127,7 @@ public class HelixStateMachineEngine implements StateMachineEngine {
 
         if (_manager.getInstanceType() == InstanceType.CONTROLLER
             || _manager.getInstanceType() == InstanceType.CONTROLLER_PARTICIPANT) {
-          nopMsg.setTgtName("Controller");
+          nopMsg.setTgtName(InstanceType.CONTROLLER.name());
           accessor.setProperty(keyBuilder.controllerMessage(nopMsg.getId()), nopMsg);
         }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/5ffab62f/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java b/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java
index 3eb20ef..90e0d24 100644
--- a/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/util/StatusUpdateUtil.java
@@ -32,13 +32,17 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.helix.*;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.InstanceType;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.model.Error;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.StatusUpdate;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.StatusUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -310,6 +314,18 @@ public class StatusUpdateUtil {
     return String.format("%4s %26s ", level.toString(), time) + recordId;
   }
 
+  @Deprecated
+  public void logMessageStatusUpdateRecord(Message message, Level level, Class classInfo,
+      String additionalInfo, HelixDataAccessor accessor) {
+    try {
+      ZNRecord record = createMessageStatusUpdateRecord(message, level, classInfo, additionalInfo);
+      publishStatusUpdateRecord(record, message, level, accessor,
+          message.getTgtName().equalsIgnoreCase(InstanceType.CONTROLLER.name()));
+    } catch (Exception e) {
+      _logger.error("Exception while logging status update", e);
+    }
+  }
+
   /**
    * Create a statusupdate that is related to a cluster manager message, then record it to
    * the zookeeper store.
@@ -321,14 +337,16 @@ public class StatusUpdateUtil {
    *          class info about the class that reports the status update
    * @param additionalInfo
    *          info the additional debug information
-   * @param accessor
-   *          the zookeeper data accessor that writes the status update to zookeeper
+   * @param manager
+   *          the HelixManager that writes the status update to zookeeper
    */
   public void logMessageStatusUpdateRecord(Message message, Level level, Class classInfo,
-      String additionalInfo, HelixDataAccessor accessor) {
+      String additionalInfo, HelixManager manager) {
     try {
       ZNRecord record = createMessageStatusUpdateRecord(message, level, classInfo, additionalInfo);
-      publishStatusUpdateRecord(record, message, level, accessor);
+      publishStatusUpdateRecord(record, message, level, manager.getHelixDataAccessor(),
+          manager.getInstanceType().equals(InstanceType.CONTROLLER) || manager.getInstanceType()
+              .equals(InstanceType.CONTROLLER_PARTICIPANT));
     } catch (Exception e) {
       _logger.error("Exception while logging status update", e);
     }
@@ -338,39 +356,67 @@ public class StatusUpdateUtil {
     RebalanceResourceFailure,
   }
 
-  public void logError(ErrorType errorType, Class classInfo, String additionalInfo,
-      HelixManager helixManager) {
+  public void logError(ErrorType errorType, Class classInfo, String additionalInfo, HelixManager helixManager) {
     if (helixManager != null) {
-      logError(errorType, "ErrorInfo", helixManager.getInstanceName(), helixManager.getSessionId(),
-          additionalInfo, classInfo, helixManager.getHelixDataAccessor());
+      logError(errorType, "ErrorInfo", helixManager.getInstanceName(), helixManager.getSessionId(), additionalInfo,
+          classInfo, helixManager.getHelixDataAccessor(),
+          helixManager.getInstanceType().equals(InstanceType.CONTROLLER) || helixManager.getInstanceType()
+              .equals(InstanceType.CONTROLLER_PARTICIPANT));
     } else {
       _logger.error("Exception while logging error. HelixManager is null.");
     }
   }
 
   private void logError(ErrorType errorType, String updateKey, String instanceName,
-      String sessionId, String additionalInfo, Class classInfo, HelixDataAccessor accessor) {
+      String sessionId, String additionalInfo, Class classInfo, HelixDataAccessor accessor,
+      boolean isController) {
     try {
       ZNRecord record = createEmptyStatusUpdateRecord(sessionId + "__" + instanceName);
 
-      Map<String, String> contentMap = new TreeMap<String, String>();
+      Map<String, String> contentMap = new TreeMap<>();
       contentMap.put("AdditionalInfo", additionalInfo);
       contentMap.put("Class", classInfo.toString());
       contentMap.put("SessionId", sessionId);
 
       record.setMapField(generateMapFieldId(Level.HELIX_ERROR, updateKey), contentMap);
 
-      publishErrorRecord(record, instanceName, errorType.name(), updateKey, sessionId, accessor);
+      publishErrorRecord(record, instanceName, errorType.name(), updateKey, sessionId, accessor,
+          isController);
     } catch (Exception e) {
       _logger.error("Exception while logging error", e);
     }
   }
 
+  public void logError(Message message, Class classInfo, String additionalInfo, HelixManager manager) {
+    logMessageStatusUpdateRecord(message, Level.HELIX_ERROR, classInfo, additionalInfo, manager);
+  }
+
+  public void logError(Message message, Class classInfo, Exception e, String additionalInfo,
+      HelixManager manager) {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    e.printStackTrace(pw);
+    logMessageStatusUpdateRecord(message, Level.HELIX_ERROR, classInfo,
+        additionalInfo + sw.toString(), manager);
+  }
+
+  public void logInfo(Message message, Class classInfo, String additionalInfo,
+      HelixManager manager) {
+    logMessageStatusUpdateRecord(message, Level.HELIX_INFO, classInfo, additionalInfo, manager);
+  }
+
+  public void logWarning(Message message, Class classInfo, String additionalInfo,
+      HelixManager manager) {
+    logMessageStatusUpdateRecord(message, Level.HELIX_WARNING, classInfo, additionalInfo, manager);
+  }
+
+  @Deprecated
   public void logError(Message message, Class classInfo, String additionalInfo,
       HelixDataAccessor accessor) {
     logMessageStatusUpdateRecord(message, Level.HELIX_ERROR, classInfo, additionalInfo, accessor);
   }
 
+  @Deprecated
   public void logError(Message message, Class classInfo, Exception e, String additionalInfo,
       HelixDataAccessor accessor) {
     StringWriter sw = new StringWriter();
@@ -380,16 +426,42 @@ public class StatusUpdateUtil {
         additionalInfo + sw.toString(), accessor);
   }
 
+  @Deprecated
   public void logInfo(Message message, Class classInfo, String additionalInfo,
       HelixDataAccessor accessor) {
     logMessageStatusUpdateRecord(message, Level.HELIX_INFO, classInfo, additionalInfo, accessor);
   }
 
+  @Deprecated
   public void logWarning(Message message, Class classInfo, String additionalInfo,
       HelixDataAccessor accessor) {
     logMessageStatusUpdateRecord(message, Level.HELIX_WARNING, classInfo, additionalInfo, accessor);
   }
 
+  private String getStatusUpdateKey(Message message) {
+    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
+      return message.getPartitionName();
+    }
+    return message.getMsgId();
+  }
+
+  /**
+   * Generate the sub-path under STATUSUPDATE or ERROR path for a status update
+   */
+  String getStatusUpdateSubPath(Message message) {
+    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
+      return message.getResourceName();
+    }
+    return message.getMsgType();
+  }
+
+  String getStatusUpdateRecordName(Message message) {
+    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
+      return message.getTgtSessionId() + "__" + message.getResourceName();
+    }
+    return message.getMsgId();
+  }
+
   /**
    * Write a status update record to zookeeper to the zookeeper store.
    * @param record
@@ -400,9 +472,11 @@ public class StatusUpdateUtil {
    *          the error level of the message update
    * @param accessor
    *          the zookeeper data accessor that writes the status update to zookeeper
+   * @param isController
+   *          if the update is for a controller instance or not
    */
   void publishStatusUpdateRecord(ZNRecord record, Message message, Level level,
-      HelixDataAccessor accessor) {
+      HelixDataAccessor accessor, boolean isController) {
     String instanceName = message.getTgtName();
     String statusUpdateSubPath = getStatusUpdateSubPath(message);
     String statusUpdateKey = getStatusUpdateKey(message);
@@ -416,11 +490,10 @@ public class StatusUpdateUtil {
 
     Builder keyBuilder = accessor.keyBuilder();
     if (!_recordedMessages.containsKey(message.getMsgId())) {
-      // TODO instanceName of a controller might be any string
-      if (instanceName.equalsIgnoreCase("Controller")) {
-        accessor.updateProperty(
-            keyBuilder.controllerTaskStatus(statusUpdateSubPath, statusUpdateKey),
-            new StatusUpdate(createMessageLogRecord(message)));
+      if (isController) {
+        accessor
+            .updateProperty(keyBuilder.controllerTaskStatus(statusUpdateSubPath, statusUpdateKey),
+                new StatusUpdate(createMessageLogRecord(message)));
 
       } else {
 
@@ -442,7 +515,7 @@ public class StatusUpdateUtil {
       _recordedMessages.put(message.getMsgId(), message.getMsgId());
     }
 
-    if (instanceName.equalsIgnoreCase("Controller")) {
+    if (isController) {
       accessor.updateProperty(
           keyBuilder.controllerTaskStatus(statusUpdateSubPath, statusUpdateKey), new StatusUpdate(
               record));
@@ -462,35 +535,10 @@ public class StatusUpdateUtil {
     // If the error level is ERROR, also write the record to "ERROR" ZNode
     if (Level.HELIX_ERROR == level) {
       publishErrorRecord(record, instanceName, statusUpdateSubPath, statusUpdateKey, sessionId,
-          accessor);
+          accessor, isController);
     }
   }
 
-  private String getStatusUpdateKey(Message message) {
-    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
-      return message.getPartitionName();
-    }
-    return message.getMsgId();
-  }
-
-  /**
-   * Generate the sub-path under STATUSUPDATE or ERROR path for a status update
-   */
-  String getStatusUpdateSubPath(Message message) {
-    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
-      return message.getResourceName();
-    } else {
-      return message.getMsgType();
-    }
-  }
-
-  String getStatusUpdateRecordName(Message message) {
-    if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.name())) {
-      return message.getTgtSessionId() + "__" + message.getResourceName();
-    }
-    return message.getMsgId();
-  }
-
   /**
    * Write an error record to zookeeper to the zookeeper store.
    * @param record
@@ -505,12 +553,13 @@ public class StatusUpdateUtil {
    *          the session id
    * @param accessor
    *          the zookeeper data accessor that writes the status update to zookeeper
+   * @param isController
+   *          if the error log is for a controller instance or not
    */
   void publishErrorRecord(ZNRecord record, String instanceName, String updateSubPath,
-      String updateKey, String sessionId, HelixDataAccessor accessor) {
+      String updateKey, String sessionId, HelixDataAccessor accessor, boolean isController) {
     Builder keyBuilder = accessor.keyBuilder();
-    // TODO remove the hard code: "controller"
-    if (instanceName.toLowerCase().startsWith("controller")) {
+    if (isController) {
       // TODO need to fix: ERRORS_CONTROLLER doesn't have a form of
       // ../{sessionId}/{subPath}
       accessor.setProperty(keyBuilder.controllerTaskError(updateSubPath), new Error(record));


[21/50] [abbrv] helix git commit: Improve the data load in Helix Spectator (RoutingTableProvider), which includes: 1) Put event callback handler in a spearate thread so other ZK event callbacks won't be blocked. 2) Deduplicate the callbacks from same eve

Posted by jx...@apache.org.
Improve the data load in Helix Spectator (RoutingTableProvider), which includes:
1) Put event callback handler in a spearate thread so other ZK event callbacks won't be blocked.
2) Deduplicate the callbacks from same event type, always keep just one latest copy of event callback in the event queue.
3) Add methods to return all instances and liveInstances in the cluster.


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

Branch: refs/heads/master
Commit: 093f7ab949b1d6d77f8589b96e7a1a10762f7e6b
Parents: 2f3a56f
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Nov 8 14:56:09 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:51 2018 -0800

----------------------------------------------------------------------
 .../helix/common/ClusterEventBlockingQueue.java | 126 ++++++
 .../helix/common/ClusterEventProcessor.java     |  57 +++
 .../controller/GenericHelixController.java      |   7 +-
 .../stages/ClusterEventBlockingQueue.java       | 124 -----
 .../helix/spectator/RoutingDataCache.java       | 161 +++++++
 .../apache/helix/spectator/RoutingTable.java    | 437 ++++++++++++++++++
 .../helix/spectator/RoutingTableProvider.java   | 453 ++++++-------------
 .../java/org/apache/helix/TestRoutingTable.java |   3 +-
 .../stages/TestClusterEventBlockingQueue.java   |   1 +
 .../Spectator/TestRoutingTableProvider.java     | 173 +++++++
 .../helix/integration/TestBasicSpectator.java   |   1 +
 .../integration/TestResourceGroupEndtoEnd.java  |  29 +-
 .../common/ZkIntegrationTestBase.java           |  39 ++
 .../manager/MockParticipantManager.java         |   6 +-
 .../messaging/TestP2PMessageSemiAuto.java       |  18 +-
 15 files changed, 1147 insertions(+), 488 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/common/ClusterEventBlockingQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/common/ClusterEventBlockingQueue.java b/helix-core/src/main/java/org/apache/helix/common/ClusterEventBlockingQueue.java
new file mode 100644
index 0000000..075edf9
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/common/ClusterEventBlockingQueue.java
@@ -0,0 +1,126 @@
+package org.apache.helix.common;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+
+import org.apache.helix.controller.stages.ClusterEvent;
+import org.apache.helix.controller.stages.ClusterEventType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * A blocking queue of ClusterEvent objects to be used by the controller pipeline. This prevents
+ * multiple events of the same type from flooding the controller and preventing progress from being
+ * made. This queue has no capacity. This class is meant to be a limited implementation of the
+ * {@link BlockingQueue} interface.
+ */
+public class ClusterEventBlockingQueue {
+  private static final Logger LOG = LoggerFactory.getLogger(ClusterEventBlockingQueue.class);
+  private final Map<ClusterEventType, ClusterEvent> _eventMap;
+  private final Queue<ClusterEvent> _eventQueue;
+
+  /**
+   * Instantiate the queue
+   */
+  public ClusterEventBlockingQueue() {
+    _eventMap = Maps.newHashMap();
+    _eventQueue = Lists.newLinkedList();
+  }
+
+  /**
+   * Remove all events from the queue
+   */
+  public synchronized void clear() {
+    _eventMap.clear();
+    _eventQueue.clear();
+  }
+
+  /**
+   * Add a single event to the queue, overwriting events with the same name
+   * @param event ClusterEvent event to add
+   */
+  public synchronized void put(ClusterEvent event) {
+    if (!_eventMap.containsKey(event.getEventType())) {
+      // only insert if there isn't a same-named event already present
+      boolean result = _eventQueue.offer(event);
+      if (!result) {
+        return;
+      }
+    }
+    // always overwrite in case this is a FINALIZE
+    _eventMap.put(event.getEventType(), event);
+    LOG.debug("Putting event " + event.getEventType());
+    LOG.debug("Event queue size: " + _eventQueue.size());
+    notify();
+  }
+
+  /**
+   * Remove an element from the front of the queue, blocking if none is available. This method
+   * will return the most recent event seen with the oldest enqueued event name.
+   * @return ClusterEvent at the front of the queue
+   * @throws InterruptedException if the wait for elements was interrupted
+   */
+  public synchronized ClusterEvent take() throws InterruptedException {
+    while (_eventQueue.isEmpty()) {
+      wait();
+    }
+    ClusterEvent queuedEvent = _eventQueue.poll();
+    if (queuedEvent != null) {
+      LOG.debug("Taking event " + queuedEvent.getEventType());
+      LOG.debug("Event queue size: " + _eventQueue.size());
+      return _eventMap.remove(queuedEvent.getEventType());
+    }
+    return null;
+  }
+
+  /**
+   * Get at the head of the queue without removing it
+   * @return ClusterEvent at the front of the queue, or null if none available
+   */
+  public synchronized ClusterEvent peek() {
+    ClusterEvent queuedEvent = _eventQueue.peek();
+    if (queuedEvent != null) {
+      return _eventMap.get(queuedEvent.getEventType());
+    }
+    return queuedEvent;
+  }
+
+  /**
+   * Get the queue size
+   * @return integer size of the queue
+   */
+  public int size() {
+    return _eventQueue.size();
+  }
+
+  /**
+   * Check if the queue is empty
+   * @return true if events are not present, false otherwise
+   */
+  public boolean isEmpty() {
+    return _eventQueue.isEmpty();
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/common/ClusterEventProcessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/common/ClusterEventProcessor.java b/helix-core/src/main/java/org/apache/helix/common/ClusterEventProcessor.java
new file mode 100644
index 0000000..6001edc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/common/ClusterEventProcessor.java
@@ -0,0 +1,57 @@
+package org.apache.helix.common;
+
+import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.apache.helix.controller.stages.ClusterEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A generic extended single-thread class to handle ClusterEvent (multiple-producer/single consumer
+ * style).
+ */
+public abstract class ClusterEventProcessor extends Thread {
+  private static final Logger logger = LoggerFactory.getLogger(ClusterEventProcessor.class);
+
+  protected final ClusterEventBlockingQueue _eventQueue;
+  protected final String _clusterName;
+  protected final String _processorName;
+
+  public ClusterEventProcessor(String clusterName) {
+    this(clusterName, "Helix-ClusterEventProcessor");
+  }
+
+  public ClusterEventProcessor(String clusterName, String processorName) {
+    super(processorName + "-" + clusterName);
+    _processorName = processorName;
+    _eventQueue = new ClusterEventBlockingQueue();
+    _clusterName = clusterName;
+  }
+
+  @Override
+  public void run() {
+    logger.info("START " + _processorName + " thread for cluster " + _clusterName);
+    while (!isInterrupted()) {
+      try {
+        ClusterEvent event = _eventQueue.take();
+        handleEvent(event);
+      } catch (InterruptedException e) {
+        logger.warn(_processorName + " thread interrupted", e);
+        interrupt();
+      } catch (ZkInterruptedException e) {
+        logger.warn(_processorName + " thread caught a ZK connection interrupt", e);
+        interrupt();
+      } catch (ThreadDeath death) {
+        throw death;
+      } catch (Throwable t) {
+        logger.error(_processorName + " thread failed while running the controller pipeline", t);
+      }
+    }
+    logger.info("END " + _processorName + " thread");
+  }
+
+  protected abstract void handleEvent(ClusterEvent event);
+
+  public void queueEvent(ClusterEvent event) {
+    _eventQueue.put(event);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index c182ada..6d1af7c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -53,7 +53,7 @@ import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.controller.stages.ClusterEventBlockingQueue;
+import org.apache.helix.common.ClusterEventBlockingQueue;
 import org.apache.helix.controller.stages.ClusterEventType;
 import org.apache.helix.controller.stages.CompatibilityCheckStage;
 import org.apache.helix.controller.stages.CurrentStateComputationStage;
@@ -581,7 +581,6 @@ public class GenericHelixController implements IdealStateChangeListener,
         .info("END: GenericClusterController.onClusterConfigChange() for cluster " + _clusterName);
   }
 
-
   private void notifyCaches(NotificationContext context, ChangeType changeType) {
     if (context == null || context.getType() != Type.CALLBACK) {
       _cache.requireFullRefresh();
@@ -759,13 +758,15 @@ public class GenericHelixController implements IdealStateChangeListener,
     }
   }
 
+
+  // TODO: refactor this to use common/ClusterEventProcessor.
   private class ClusterEventProcessor extends Thread {
     private final ClusterDataCache _cache;
     private final ClusterEventBlockingQueue _eventBlockingQueue;
 
     public ClusterEventProcessor(ClusterDataCache cache,
         ClusterEventBlockingQueue eventBlockingQueue) {
-      super("GerenricHelixController-event_process");
+      super("GenericHelixController-event_process");
       _cache = cache;
       _eventBlockingQueue = eventBlockingQueue;
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventBlockingQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventBlockingQueue.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventBlockingQueue.java
deleted file mode 100644
index 521c43e..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterEventBlockingQueue.java
+++ /dev/null
@@ -1,124 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.BlockingQueue;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-/**
- * A blocking queue of ClusterEvent objects to be used by the controller pipeline. This prevents
- * multiple events of the same type from flooding the controller and preventing progress from being
- * made. This queue has no capacity. This class is meant to be a limited implementation of the
- * {@link BlockingQueue} interface.
- */
-public class ClusterEventBlockingQueue {
-  private static final Logger LOG = LoggerFactory.getLogger(ClusterEventBlockingQueue.class);
-  private final Map<ClusterEventType, ClusterEvent> _eventMap;
-  private final Queue<ClusterEvent> _eventQueue;
-
-  /**
-   * Instantiate the queue
-   */
-  public ClusterEventBlockingQueue() {
-    _eventMap = Maps.newHashMap();
-    _eventQueue = Lists.newLinkedList();
-  }
-
-  /**
-   * Remove all events from the queue
-   */
-  public synchronized void clear() {
-    _eventMap.clear();
-    _eventQueue.clear();
-  }
-
-  /**
-   * Add a single event to the queue, overwriting events with the same name
-   * @param event ClusterEvent event to add
-   */
-  public synchronized void put(ClusterEvent event) {
-    if (!_eventMap.containsKey(event.getEventType())) {
-      // only insert if there isn't a same-named event already present
-      boolean result = _eventQueue.offer(event);
-      if (!result) {
-        return;
-      }
-    }
-    // always overwrite in case this is a FINALIZE
-    _eventMap.put(event.getEventType(), event);
-    LOG.debug("Putting event " + event.getEventType());
-    LOG.debug("Event queue size: " + _eventQueue.size());
-    notify();
-  }
-
-  /**
-   * Remove an element from the front of the queue, blocking if none is available. This method
-   * will return the most recent event seen with the oldest enqueued event name.
-   * @return ClusterEvent at the front of the queue
-   * @throws InterruptedException if the wait for elements was interrupted
-   */
-  public synchronized ClusterEvent take() throws InterruptedException {
-    while (_eventQueue.isEmpty()) {
-      wait();
-    }
-    ClusterEvent queuedEvent = _eventQueue.poll();
-    if (queuedEvent != null) {
-      LOG.debug("Taking event " + queuedEvent.getEventType());
-      LOG.debug("Event queue size: " + _eventQueue.size());
-      return _eventMap.remove(queuedEvent.getEventType());
-    }
-    return null;
-  }
-
-  /**
-   * Get at the head of the queue without removing it
-   * @return ClusterEvent at the front of the queue, or null if none available
-   */
-  public synchronized ClusterEvent peek() {
-    ClusterEvent queuedEvent = _eventQueue.peek();
-    if (queuedEvent != null) {
-      return _eventMap.get(queuedEvent.getEventType());
-    }
-    return queuedEvent;
-  }
-
-  /**
-   * Get the queue size
-   * @return integer size of the queue
-   */
-  public int size() {
-    return _eventQueue.size();
-  }
-
-  /**
-   * Check if the queue is empty
-   * @return true if events are not present, false otherwise
-   */
-  public boolean isEmpty() {
-    return _eventQueue.isEmpty();
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/spectator/RoutingDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingDataCache.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingDataCache.java
new file mode 100644
index 0000000..2da0f97
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingDataCache.java
@@ -0,0 +1,161 @@
+package org.apache.helix.spectator;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Cache the cluster data that are needed by RoutingTableProvider.
+ */
+public class RoutingDataCache {
+  private static final Logger LOG = LoggerFactory.getLogger(RoutingDataCache.class.getName());
+
+  private Map<String, LiveInstance> _liveInstanceMap;
+  private Map<String, InstanceConfig> _instanceConfigMap;
+  private Map<String, ExternalView> _externalViewMap;
+  String _clusterName;
+
+  private Map<HelixConstants.ChangeType, Boolean> _propertyDataChangedMap;
+
+  public RoutingDataCache(String clusterName) {
+    _propertyDataChangedMap = new ConcurrentHashMap<>();
+    for (HelixConstants.ChangeType type : HelixConstants.ChangeType.values()) {
+      _propertyDataChangedMap.put(type, Boolean.valueOf(true));
+    }
+    _clusterName = clusterName;
+  }
+
+  /**
+   * This refreshes the cluster data by re-fetching the data from zookeeper in an efficient way
+   *
+   * @param accessor
+   *
+   * @return
+   */
+  public synchronized void refresh(HelixDataAccessor accessor) {
+    LOG.info("START: ClusterDataCache.refresh()");
+    long startTime = System.currentTimeMillis();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
+
+    if (_propertyDataChangedMap.get(HelixConstants.ChangeType.EXTERNAL_VIEW)) {
+      long start = System.currentTimeMillis();
+      _propertyDataChangedMap.put(HelixConstants.ChangeType.EXTERNAL_VIEW, Boolean.valueOf(false));
+      _externalViewMap = accessor.getChildValuesMap(keyBuilder.externalViews());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Reload ExternalViews: " + _externalViewMap.keySet() + ". Takes " + (
+            System.currentTimeMillis() - start) + " ms");
+      }
+    }
+
+    if (_propertyDataChangedMap.get(HelixConstants.ChangeType.LIVE_INSTANCE)) {
+      _propertyDataChangedMap.put(HelixConstants.ChangeType.LIVE_INSTANCE, Boolean.valueOf(false));
+      _liveInstanceMap = accessor.getChildValuesMap(keyBuilder.liveInstances());
+      LOG.debug("Reload LiveInstances: " + _liveInstanceMap.keySet());
+    }
+
+    if (_propertyDataChangedMap.get(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
+      _propertyDataChangedMap
+          .put(HelixConstants.ChangeType.INSTANCE_CONFIG, Boolean.valueOf(false));
+      _instanceConfigMap = accessor.getChildValuesMap(keyBuilder.instanceConfigs());
+      LOG.debug("Reload InstanceConfig: " + _instanceConfigMap.keySet());
+    }
+
+    long endTime = System.currentTimeMillis();
+    LOG.info(
+        "END: RoutingDataCache.refresh() for cluster " + _clusterName + ", took " + (endTime
+            - startTime) + " ms");
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("LiveInstances: " + _liveInstanceMap.keySet());
+      for (LiveInstance instance : _liveInstanceMap.values()) {
+        LOG.debug("live instance: " + instance.getInstanceName() + " " + instance.getSessionId());
+      }
+      LOG.debug("ExternalViews: " + _externalViewMap.keySet());
+      LOG.debug("InstanceConfigs: " + _instanceConfigMap.keySet());
+    }
+  }
+
+  /**
+   * Retrieves the ExternalView for all resources
+   *
+   * @return
+   */
+  public Map<String, ExternalView> getExternalViews() {
+    return Collections.unmodifiableMap(_externalViewMap);
+  }
+
+  /**
+   * Returns the LiveInstances for each of the instances that are curretnly up and running
+   *
+   * @return
+   */
+  public Map<String, LiveInstance> getLiveInstances() {
+    return Collections.unmodifiableMap(_liveInstanceMap);
+  }
+
+  /**
+   * Returns the instance config map
+   *
+   * @return
+   */
+  public Map<String, InstanceConfig> getInstanceConfigMap() {
+    return Collections.unmodifiableMap(_instanceConfigMap);
+  }
+
+  /**
+   * Notify the cache that some part of the cluster data has been changed.
+   */
+  public void notifyDataChange(HelixConstants.ChangeType changeType, String pathChanged) {
+    _propertyDataChangedMap.put(changeType, Boolean.valueOf(true));
+  }
+
+  /**
+   * Indicate that a full read should be done on the next refresh
+   */
+  public synchronized void requireFullRefresh() {
+    for(HelixConstants.ChangeType type : HelixConstants.ChangeType.values()) {
+      _propertyDataChangedMap.put(type, Boolean.valueOf(true));
+    }
+  }
+
+  /**
+   * toString method to print the data cache state
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("liveInstaceMap:" + _liveInstanceMap).append("\n");
+    sb.append("externalViewMap:" + _externalViewMap).append("\n");
+    sb.append("instanceConfigMap:" + _instanceConfigMap).append("\n");
+
+    return sb.toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/spectator/RoutingTable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTable.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTable.java
new file mode 100644
index 0000000..564a218
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTable.java
@@ -0,0 +1,437 @@
+package org.apache.helix.spectator;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A class to consume ExternalViews of a cluster and provide {resource, partition, state} to
+ * {instances} map function.
+ */
+class RoutingTable {
+  private static final Logger logger = LoggerFactory.getLogger(RoutingTable.class);
+
+  // mapping a resourceName to the ResourceInfo
+  private final Map<String, ResourceInfo> _resourceInfoMap;
+  // mapping a resource group name to a resourceGroupInfo
+  private final Map<String, ResourceGroupInfo> _resourceGroupInfoMap;
+  private final Collection<LiveInstance> _liveInstances;
+  private final Collection<InstanceConfig> _instanceConfigs;
+
+  public RoutingTable() {
+    this(Collections.<ExternalView>emptyList(), Collections.<InstanceConfig>emptyList(),
+        Collections.<LiveInstance>emptyList());
+  }
+
+  public RoutingTable(Collection<ExternalView> externalViews, Collection<InstanceConfig> instanceConfigs,
+      Collection<LiveInstance> liveInstances) {
+    _resourceInfoMap = new HashMap<>();
+    _resourceGroupInfoMap = new HashMap<>();
+    _liveInstances = liveInstances;
+    _instanceConfigs = instanceConfigs;
+    refresh(externalViews, instanceConfigs);
+  }
+
+  private void addEntry(String resourceName, String partitionName, String state,
+      InstanceConfig config) {
+    if (!_resourceInfoMap.containsKey(resourceName)) {
+      _resourceInfoMap.put(resourceName, new ResourceInfo());
+    }
+    ResourceInfo resourceInfo = _resourceInfoMap.get(resourceName);
+    resourceInfo.addEntry(partitionName, state, config);
+  }
+
+  /**
+   * add an entry with a resource with resourceGrouping enabled.
+   */
+  private void addEntry(String resourceName, String resourceGroupName, String resourceTag,
+      String partitionName, String state, InstanceConfig config) {
+    addEntry(resourceName, partitionName, state, config);
+
+    if (!_resourceGroupInfoMap.containsKey(resourceGroupName)) {
+      _resourceGroupInfoMap.put(resourceGroupName, new ResourceGroupInfo());
+    }
+
+    ResourceGroupInfo resourceGroupInfo = _resourceGroupInfoMap.get(resourceGroupName);
+    resourceGroupInfo.addEntry(resourceTag, partitionName, state, config);
+  }
+
+  ResourceInfo get(String resourceName) {
+    return _resourceInfoMap.get(resourceName);
+  }
+
+  ResourceGroupInfo getResourceGroup(String resourceGroupName) {
+    return _resourceGroupInfoMap.get(resourceGroupName);
+  }
+
+  /**
+   * returns all instances for {resource} that are in a specific {state}.
+   * @param resourceName
+   * @param state
+   * @return empty list if there is no instance in a given state
+   */
+  public Set<InstanceConfig> getInstancesForResource(String resourceName, String state) {
+    Set<InstanceConfig> instanceSet = null;
+    ResourceInfo resourceInfo = get(resourceName);
+    if (resourceInfo != null) {
+      instanceSet = resourceInfo.getInstances(state);
+    }
+    if (instanceSet == null) {
+      instanceSet = Collections.emptySet();
+    }
+    return instanceSet;
+  }
+
+  /**
+   * returns all instances for all resources in {resource group} that are in a specific {state}
+   *
+   * @param resourceGroupName
+   * @param state
+   *
+   * @return empty list if there is no instance in a given state
+   */
+  public Set<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String state) {
+    Set<InstanceConfig> instanceSet = null;
+    ResourceGroupInfo resourceGroupInfo = getResourceGroup(resourceGroupName);
+    if (resourceGroupInfo != null) {
+      instanceSet = resourceGroupInfo.getInstances(state);
+    }
+    if (instanceSet == null) {
+      instanceSet = Collections.emptySet();
+    }
+    return instanceSet;
+  }
+
+  /**
+   * returns all instances for resources contains any given tags in {resource group} that are in a
+   * specific {state}
+   *
+   * @param resourceGroupName
+   * @param state
+   *
+   * @return empty list if there is no instance in a given state
+   */
+  public Set<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String state,
+      List<String> resourceTags) {
+    Set<InstanceConfig> instanceSet = null;
+    ResourceGroupInfo resourceGroupInfo = getResourceGroup(resourceGroupName);
+    if (resourceGroupInfo != null) {
+      instanceSet = new HashSet<>();
+      for (String tag : resourceTags) {
+        Set<InstanceConfig> instances = resourceGroupInfo.getInstances(state, tag);
+        if (instances != null) {
+          instanceSet.addAll(resourceGroupInfo.getInstances(state, tag));
+        }
+      }
+    }
+    if (instanceSet == null) {
+      return Collections.emptySet();
+    }
+    return instanceSet;
+  }
+
+  /**
+   * returns the instances for {resource,partition} pair that are in a specific
+   * {state}
+   * @param resourceName
+   * @param partitionName
+   * @param state
+   * @return empty list if there is no instance in a given state
+   */
+  public List<InstanceConfig> getInstancesForResource(String resourceName, String partitionName,
+      String state) {
+    List<InstanceConfig> instanceList = null;
+    ResourceInfo resourceInfo = get(resourceName);
+    if (resourceInfo != null) {
+      PartitionInfo keyInfo = resourceInfo.get(partitionName);
+      if (keyInfo != null) {
+        instanceList = keyInfo.get(state);
+      }
+    }
+    if (instanceList == null) {
+      instanceList = Collections.emptyList();
+    }
+    return instanceList;
+  }
+
+  /**
+   * returns the instances for {resource group,partition} pair in all resources belongs to the given
+   * resource group that are in a specific {state}.
+   *
+   * The return results aggregate all partition states from all the resources in the given resource
+   * group.
+   *
+   * @param resourceGroupName
+   * @param partitionName
+   * @param state
+   *
+   * @return empty list if there is no instance in a given state
+   */
+  public List<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName,
+      String partitionName, String state) {
+    List<InstanceConfig> instanceList = null;
+    ResourceGroupInfo resourceGroupInfo = getResourceGroup(resourceGroupName);
+    if (resourceGroupInfo != null) {
+      PartitionInfo keyInfo = resourceGroupInfo.get(partitionName);
+      if (keyInfo != null) {
+        instanceList = keyInfo.get(state);
+      }
+    }
+    if (instanceList == null) {
+      instanceList = Collections.emptyList();
+    }
+    return instanceList;
+  }
+
+  /**
+   * Return all liveInstances in the cluster now.
+   * @return
+   */
+  protected Collection<LiveInstance> getLiveInstances() {
+    return _liveInstances;
+  }
+
+  /**
+   * Return all instance's config in this cluster.
+   * @return
+   */
+  protected Collection<InstanceConfig> getInstanceConfigs() {
+    return _instanceConfigs;
+  }
+
+  /**
+   * returns the instances for {resource group,partition} pair contains any of the given tags
+   * that are in a specific {state}.
+   *
+   * Find all resources belongs to the given resource group that have any of the given resource tags
+   * and return the aggregated partition states from all these resources.
+   *
+   * @param resourceGroupName
+   * @param partitionName
+   * @param state
+   * @param resourceTags
+   *
+   * @return empty list if there is no instance in a given state
+   */
+  public List<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String partitionName,
+      String state, List<String> resourceTags) {
+    ResourceGroupInfo resourceGroupInfo = getResourceGroup(resourceGroupName);
+    List<InstanceConfig> instanceList = null;
+    if (resourceGroupInfo != null) {
+      instanceList = new ArrayList<>();
+      for (String tag : resourceTags) {
+        RoutingTable.PartitionInfo keyInfo = resourceGroupInfo.get(partitionName, tag);
+        if (keyInfo != null && keyInfo.containsState(state)) {
+          instanceList.addAll(keyInfo.get(state));
+        }
+      }
+    }
+    if (instanceList == null) {
+      return Collections.emptyList();
+    }
+
+    return instanceList;
+  }
+
+  private void refresh(Collection<ExternalView> externalViewList,
+      Collection<InstanceConfig> instanceConfigList) {
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    for (InstanceConfig config : instanceConfigList) {
+      instanceConfigMap.put(config.getId(), config);
+    }
+    if (externalViewList != null) {
+      for (ExternalView extView : externalViewList) {
+        String resourceName = extView.getId();
+        for (String partitionName : extView.getPartitionSet()) {
+          Map<String, String> stateMap = extView.getStateMap(partitionName);
+          for (String instanceName : stateMap.keySet()) {
+            String currentState = stateMap.get(instanceName);
+            if (instanceConfigMap.containsKey(instanceName)) {
+              InstanceConfig instanceConfig = instanceConfigMap.get(instanceName);
+              if (extView.isGroupRoutingEnabled()) {
+                addEntry(resourceName, extView.getResourceGroupName(),
+                    extView.getInstanceGroupTag(), partitionName, currentState, instanceConfig);
+              } else {
+                addEntry(resourceName, partitionName, currentState, instanceConfig);
+              }
+            } else {
+              logger.error("Invalid instance name. " + instanceName
+                  + " .Not found in /cluster/configs/. instanceName: ");
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Class to store instances, partitions and their states for each resource.
+   */
+  class ResourceInfo {
+    // store PartitionInfo for each partition
+    Map<String, PartitionInfo> partitionInfoMap;
+    // stores the Set of Instances in a given state
+    Map<String, Set<InstanceConfig>> stateInfoMap;
+
+    public ResourceInfo() {
+      partitionInfoMap = new HashMap<>();
+      stateInfoMap = new HashMap<>();
+    }
+
+    public void addEntry(String stateUnitKey, String state, InstanceConfig config) {
+      if (!stateInfoMap.containsKey(state)) {
+        stateInfoMap.put(state, new TreeSet<>(INSTANCE_CONFIG_COMPARATOR));
+      }
+      Set<InstanceConfig> set = stateInfoMap.get(state);
+      set.add(config);
+
+      if (!partitionInfoMap.containsKey(stateUnitKey)) {
+        partitionInfoMap.put(stateUnitKey, new PartitionInfo());
+      }
+      PartitionInfo stateUnitKeyInfo = partitionInfoMap.get(stateUnitKey);
+      stateUnitKeyInfo.addEntry(state, config);
+    }
+
+    public Set<InstanceConfig> getInstances(String state) {
+      return stateInfoMap.get(state);
+    }
+
+    PartitionInfo get(String stateUnitKey) {
+      return partitionInfoMap.get(stateUnitKey);
+    }
+  }
+
+  /**
+   * Class to store instances, partitions and their states for each resource group.
+   */
+  class ResourceGroupInfo {
+    // aggregated partitions and instances info for all resources in the resource group.
+    ResourceInfo aggregatedResourceInfo;
+
+    // <ResourceTag, ResourceInfo> maps resource tag to the resource with the tag
+    // in this resource group.
+    // Each ResourceInfo saves only partitions and instances for that resource.
+    Map<String, ResourceInfo> tagToResourceMap;
+
+    public ResourceGroupInfo() {
+      aggregatedResourceInfo = new ResourceInfo();
+      tagToResourceMap = new HashMap<>();
+    }
+
+    public void addEntry(String resourceTag, String stateUnitKey, String state, InstanceConfig config) {
+      // add the new entry to the aggregated resource info
+      aggregatedResourceInfo.addEntry(stateUnitKey, state, config);
+
+      // add the entry to the resourceInfo with given tag
+      if (!tagToResourceMap.containsKey(resourceTag)) {
+        tagToResourceMap.put(resourceTag, new ResourceInfo());
+      }
+      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
+      resourceInfo.addEntry(stateUnitKey, state, config);
+    }
+
+    public Set<InstanceConfig> getInstances(String state) {
+      return aggregatedResourceInfo.getInstances(state);
+    }
+
+    public Set<InstanceConfig> getInstances(String state, String resourceTag) {
+      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
+      if (resourceInfo != null) {
+        return resourceInfo.getInstances(state);
+      }
+
+      return null;
+    }
+
+    PartitionInfo get(String stateUnitKey) {
+      return aggregatedResourceInfo.get(stateUnitKey);
+    }
+
+    PartitionInfo get(String stateUnitKey, String resourceTag) {
+      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
+      if (resourceInfo == null) {
+        return null;
+      }
+
+      return resourceInfo.get(stateUnitKey);
+    }
+  }
+
+  class PartitionInfo {
+    Map<String, List<InstanceConfig>> stateInfoMap;
+
+    public PartitionInfo() {
+      stateInfoMap = new HashMap<>();
+    }
+
+    public void addEntry(String state, InstanceConfig config) {
+      if (!stateInfoMap.containsKey(state)) {
+        stateInfoMap.put(state, new ArrayList<InstanceConfig>());
+      }
+      List<InstanceConfig> list = stateInfoMap.get(state);
+      list.add(config);
+    }
+
+    List<InstanceConfig> get(String state) {
+      return stateInfoMap.get(state);
+    }
+
+    boolean containsState(String state) {
+      return stateInfoMap.containsKey(state);
+    }
+  }
+
+  private static Comparator<InstanceConfig> INSTANCE_CONFIG_COMPARATOR =
+      new Comparator<InstanceConfig>() {
+        @Override
+        public int compare(InstanceConfig o1, InstanceConfig o2) {
+          if (o1 == o2) {
+            return 0;
+          }
+          if (o1 == null) {
+            return -1;
+          }
+          if (o2 == null) {
+            return 1;
+          }
+
+          int compareTo = o1.getHostName().compareTo(o2.getHostName());
+          if (compareTo == 0) {
+            return o1.getPort().compareTo(o2.getPort());
+          }
+
+          return compareTo;
+        }
+      };
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
index 79a8ed0..a89636b 100644
--- a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
@@ -19,34 +19,60 @@ package org.apache.helix.spectator;
  * under the License.
  */
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.helix.ConfigChangeListener;
-import org.apache.helix.ExternalViewChangeListener;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.listeners.ConfigChangeListener;
+import org.apache.helix.api.listeners.InstanceConfigChangeListener;
+import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.listeners.LiveInstanceChangeListener;
+import org.apache.helix.api.listeners.PreFetch;
+import org.apache.helix.common.ClusterEventProcessor;
+import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.ClusterEvent;
+import org.apache.helix.controller.stages.ClusterEventType;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class RoutingTableProvider implements ExternalViewChangeListener, ConfigChangeListener {
+public class RoutingTableProvider implements ExternalViewChangeListener, InstanceConfigChangeListener,
+    ConfigChangeListener, LiveInstanceChangeListener {
   private static final Logger logger = LoggerFactory.getLogger(RoutingTableProvider.class);
   private final AtomicReference<RoutingTable> _routingTableRef;
+  private final HelixManager _helixManager;
+  private final RouterUpdater _routerUpdater;
 
   public RoutingTableProvider() {
-    _routingTableRef = new AtomicReference<RoutingTableProvider.RoutingTable>(new RoutingTable());
+    this(null);
+  }
 
+  public RoutingTableProvider(HelixManager helixManager) throws HelixException {
+    _routingTableRef = new AtomicReference<>(new RoutingTable());
+    _helixManager = helixManager;
+    String clusterName = null;
+    if (_helixManager != null) {
+      clusterName = _helixManager.getClusterName();
+      try {
+        _helixManager.addExternalViewChangeListener(this);
+        _helixManager.addInstanceConfigChangeListener(this);
+        _helixManager.addLiveInstanceChangeListener(this);
+      } catch (Exception e) {
+        logger.error("Failed to attach listeners to HelixManager!");
+        throw new HelixException("Failed to attach listeners to HelixManager!", e);
+      }
+    }
+    _routerUpdater = new RouterUpdater(clusterName);
+    _routerUpdater.start();
   }
 
   /**
@@ -75,19 +101,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    * @return empty list if there is no instance in a given state
    */
   public List<InstanceConfig> getInstancesForResource(String resourceName, String partitionName, String state) {
-    List<InstanceConfig> instanceList = null;
-    RoutingTable _routingTable = _routingTableRef.get();
-    ResourceInfo resourceInfo = _routingTable.get(resourceName);
-    if (resourceInfo != null) {
-      PartitionInfo keyInfo = resourceInfo.get(partitionName);
-      if (keyInfo != null) {
-        instanceList = keyInfo.get(state);
-      }
-    }
-    if (instanceList == null) {
-      instanceList = Collections.emptyList();
-    }
-    return instanceList;
+    return _routingTableRef.get().getInstancesForResource(resourceName, partitionName, state);
   }
 
   /**
@@ -105,19 +119,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    */
   public List<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName,
       String partitionName, String state) {
-    List<InstanceConfig> instanceList = null;
-    RoutingTable _routingTable = _routingTableRef.get();
-    ResourceGroupInfo resourceGroupInfo = _routingTable.getResourceGroup(resourceGroupName);
-    if (resourceGroupInfo != null) {
-      PartitionInfo keyInfo = resourceGroupInfo.get(partitionName);
-      if (keyInfo != null) {
-        instanceList = keyInfo.get(state);
-      }
-    }
-    if (instanceList == null) {
-      instanceList = Collections.emptyList();
-    }
-    return instanceList;
+    return _routingTableRef.get().getInstancesForResourceGroup(resourceGroupName, partitionName, state);
   }
 
   /**
@@ -134,25 +136,10 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    *
    * @return empty list if there is no instance in a given state
    */
-  public List<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String partitionName,
-      String state, List<String> resourceTags) {
-    RoutingTable _routingTable = _routingTableRef.get();
-    ResourceGroupInfo resourceGroupInfo = _routingTable.getResourceGroup(resourceGroupName);
-    List<InstanceConfig> instanceList = null;
-    if (resourceGroupInfo != null) {
-      instanceList = new ArrayList<InstanceConfig>();
-      for (String tag : resourceTags) {
-        PartitionInfo keyInfo = resourceGroupInfo.get(partitionName, tag);
-        if (keyInfo != null && keyInfo.containsState(state)) {
-          instanceList.addAll(keyInfo.get(state));
-        }
-      }
-    }
-    if (instanceList == null) {
-      return Collections.emptyList();
-    }
-
-    return instanceList;
+  public List<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName,
+      String partitionName, String state, List<String> resourceTags) {
+    return _routingTableRef.get()
+        .getInstancesForResourceGroup(resourceGroupName, partitionName, state, resourceTags);
   }
 
   /**
@@ -175,16 +162,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    * @return empty list if there is no instance in a given state
    */
   public Set<InstanceConfig> getInstancesForResource(String resourceName, String state) {
-    Set<InstanceConfig> instanceSet = null;
-    RoutingTable routingTable = _routingTableRef.get();
-    ResourceInfo resourceInfo = routingTable.get(resourceName);
-    if (resourceInfo != null) {
-      instanceSet = resourceInfo.getInstances(state);
-    }
-    if (instanceSet == null) {
-      instanceSet = Collections.emptySet();
-    }
-    return instanceSet;
+    return _routingTableRef.get().getInstancesForResource(resourceName, state);
   }
 
   /**
@@ -196,16 +174,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    * @return empty list if there is no instance in a given state
    */
   public Set<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String state) {
-    Set<InstanceConfig> instanceSet = null;
-    RoutingTable _routingTable = _routingTableRef.get();
-    ResourceGroupInfo resourceGroupInfo = _routingTable.getResourceGroup(resourceGroupName);
-    if (resourceGroupInfo != null) {
-      instanceSet = resourceGroupInfo.getInstances(state);
-    }
-    if (instanceSet == null) {
-      instanceSet = Collections.emptySet();
-    }
-    return instanceSet;
+    return _routingTableRef.get().getInstancesForResourceGroup(resourceGroupName, state);
   }
 
   /**
@@ -219,274 +188,122 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
    */
   public Set<InstanceConfig> getInstancesForResourceGroup(String resourceGroupName, String state,
       List<String> resourceTags) {
-    Set<InstanceConfig> instanceSet = null;
-    RoutingTable _routingTable = _routingTableRef.get();
-    ResourceGroupInfo resourceGroupInfo = _routingTable.getResourceGroup(resourceGroupName);
-    if (resourceGroupInfo != null) {
-      instanceSet = new HashSet<InstanceConfig>();
-      for (String tag : resourceTags) {
-        Set<InstanceConfig> instances = resourceGroupInfo.getInstances(state, tag);
-        if (instances != null) {
-          instanceSet.addAll(resourceGroupInfo.getInstances(state, tag));
-        }
-      }
-    }
-    if (instanceSet == null) {
-      return Collections.emptySet();
-    }
-    return instanceSet;
+    return _routingTableRef.get().getInstancesForResourceGroup(resourceGroupName, state, resourceTags);
+  }
+
+  /**
+   * Return all liveInstances in the cluster now.
+   * @return
+   */
+  public Collection<LiveInstance> getLiveInstances() {
+    return _routingTableRef.get().getLiveInstances();
+  }
+
+  /**
+   * Return all instance's config in this cluster.
+   * @return
+   */
+  public Collection<InstanceConfig> getInstanceConfigs() {
+    return _routingTableRef.get().getInstanceConfigs();
   }
 
   @Override
+  @PreFetch(enabled = false)
   public void onExternalViewChange(List<ExternalView> externalViewList,
       NotificationContext changeContext) {
-    // session has expired clean up the routing table
-    if (changeContext.getType() == NotificationContext.Type.FINALIZE) {
-      logger.info("Resetting the routing table. ");
-      RoutingTable newRoutingTable = new RoutingTable();
-      _routingTableRef.set(newRoutingTable);
-      return;
+    // Refresh with full list of external view.
+    // keep this here for back-compatibility
+    if (externalViewList != null && externalViewList.size() > 0) {
+      refresh(externalViewList, changeContext);
+    } else {
+      _routerUpdater.queueEvent(changeContext, ClusterEventType.ExternalViewChange,
+          HelixConstants.ChangeType.EXTERNAL_VIEW);
     }
-    refresh(externalViewList, changeContext);
   }
 
   @Override
-  public void onConfigChange(List<InstanceConfig> configs, NotificationContext changeContext) {
-    // session has expired clean up the routing table
-    if (changeContext.getType() == NotificationContext.Type.FINALIZE) {
-      logger.info("Resetting the routing table. ");
-      RoutingTable newRoutingTable = new RoutingTable();
-      _routingTableRef.set(newRoutingTable);
-      return;
-    }
+  @PreFetch(enabled = false)
+  public void onInstanceConfigChange(List<InstanceConfig> configs,
+      NotificationContext changeContext) {
+    _routerUpdater.queueEvent(changeContext, ClusterEventType.InstanceConfigChange,
+        HelixConstants.ChangeType.INSTANCE_CONFIG);
+  }
 
-    HelixDataAccessor accessor = changeContext.getManager().getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-    List<ExternalView> externalViewList = accessor.getChildValues(keyBuilder.externalViews());
-    refresh(externalViewList, changeContext);
+  @Override
+  @PreFetch(enabled = false)
+  public void onConfigChange(List<InstanceConfig> configs,
+      NotificationContext changeContext) {
+    onInstanceConfigChange(configs, changeContext);
   }
 
-  private void refresh(List<ExternalView> externalViewList, NotificationContext changeContext) {
-    HelixDataAccessor accessor = changeContext.getManager().getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
+  @Override
+  @PreFetch(enabled = false)
+  public void onLiveInstanceChange(List<LiveInstance> liveInstances,
+      NotificationContext changeContext) {
+    _routerUpdater.queueEvent(changeContext, ClusterEventType.LiveInstanceChange,
+        HelixConstants.ChangeType.LIVE_INSTANCE);
+  }
 
-    List<InstanceConfig> configList = accessor.getChildValues(keyBuilder.instanceConfigs());
-    Map<String, InstanceConfig> instanceConfigMap = new HashMap<String, InstanceConfig>();
-    for (InstanceConfig config : configList) {
-      instanceConfigMap.put(config.getId(), config);
-    }
+  private void reset() {
+    logger.info("Resetting the routing table.");
     RoutingTable newRoutingTable = new RoutingTable();
-    if (externalViewList != null) {
-      for (ExternalView extView : externalViewList) {
-        String resourceName = extView.getId();
-        for (String partitionName : extView.getPartitionSet()) {
-          Map<String, String> stateMap = extView.getStateMap(partitionName);
-          for (String instanceName : stateMap.keySet()) {
-            String currentState = stateMap.get(instanceName);
-            if (instanceConfigMap.containsKey(instanceName)) {
-              InstanceConfig instanceConfig = instanceConfigMap.get(instanceName);
-              if (extView.isGroupRoutingEnabled()) {
-                newRoutingTable.addEntry(resourceName, extView.getResourceGroupName(),
-                    extView.getInstanceGroupTag(), partitionName, currentState, instanceConfig);
-              } else {
-                newRoutingTable.addEntry(resourceName, partitionName, currentState, instanceConfig);
-              }
-            } else {
-              logger.error("Invalid instance name." + instanceName
-                  + " .Not found in /cluster/configs/. instanceName: ");
-            }
-          }
-        }
-      }
-    }
     _routingTableRef.set(newRoutingTable);
   }
 
-  class RoutingTable {
-    // mapping a resourceName to the ResourceInfo
-    private final Map<String, ResourceInfo> resourceInfoMap;
-
-    // mapping a resource group name to a resourceGroupInfo
-    private final Map<String, ResourceGroupInfo> resourceGroupInfoMap;
-
-    public RoutingTable() {
-      resourceInfoMap = new HashMap<String, ResourceInfo>();
-      resourceGroupInfoMap = new HashMap<String, ResourceGroupInfo>();
-    }
-
-    public void addEntry(String resourceName, String partitionName, String state,
-        InstanceConfig config) {
-      if (!resourceInfoMap.containsKey(resourceName)) {
-        resourceInfoMap.put(resourceName, new ResourceInfo());
-      }
-      ResourceInfo resourceInfo = resourceInfoMap.get(resourceName);
-      resourceInfo.addEntry(partitionName, state, config);
-    }
-
-    /**
-     * add an entry with a resource with resourceGrouping enabled.
-     */
-    public void addEntry(String resourceName, String resourceGroupName, String resourceTag,
-        String partitionName, String state, InstanceConfig config) {
-      addEntry(resourceName, partitionName, state, config);
-
-      if (!resourceGroupInfoMap.containsKey(resourceGroupName)) {
-        resourceGroupInfoMap.put(resourceGroupName, new ResourceGroupInfo());
-      }
-
-      ResourceGroupInfo resourceGroupInfo = resourceGroupInfoMap.get(resourceGroupName);
-      resourceGroupInfo.addEntry(resourceTag, partitionName, state, config);
-    }
-
-    ResourceInfo get(String resourceName) {
-      return resourceInfoMap.get(resourceName);
-    }
+  public void refresh(List<ExternalView> externalViewList, NotificationContext changeContext) {
+    HelixDataAccessor accessor = changeContext.getManager().getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = accessor.keyBuilder();
 
-    ResourceGroupInfo getResourceGroup(String resourceGroupName) {
-      return resourceGroupInfoMap.get(resourceGroupName);
-    }
+    List<InstanceConfig> configList = accessor.getChildValues(keyBuilder.instanceConfigs());
+    List<LiveInstance> liveInstances = accessor.getChildValues(keyBuilder.liveInstances());
+    refresh(externalViewList, configList, liveInstances);
   }
 
-  private static Comparator<InstanceConfig> INSTANCE_CONFIG_COMPARATOR =
-      new Comparator<InstanceConfig>() {
-        @Override
-        public int compare(InstanceConfig o1, InstanceConfig o2) {
-          if (o1 == o2) {
-            return 0;
-          }
-          if (o1 == null) {
-            return -1;
-          }
-          if (o2 == null) {
-            return 1;
-          }
-
-          int compareTo = o1.getHostName().compareTo(o2.getHostName());
-          if (compareTo == 0) {
-            return o1.getPort().compareTo(o2.getPort());
-          } else {
-            return compareTo;
-          }
-
-        }
-      };
-
-  /**
-   * Class to store instances, partitions and their states for each resource.
-   */
-  class ResourceInfo {
-    // store PartitionInfo for each partition
-    Map<String, PartitionInfo> partitionInfoMap;
-    // stores the Set of Instances in a given state
-    Map<String, Set<InstanceConfig>> stateInfoMap;
-
-    public ResourceInfo() {
-      partitionInfoMap = new HashMap<String, RoutingTableProvider.PartitionInfo>();
-      stateInfoMap = new HashMap<String, Set<InstanceConfig>>();
-    }
-
-    public void addEntry(String stateUnitKey, String state, InstanceConfig config) {
-      // add
-      if (!stateInfoMap.containsKey(state)) {
-        stateInfoMap.put(state, new TreeSet<InstanceConfig>(INSTANCE_CONFIG_COMPARATOR));
-      }
-      Set<InstanceConfig> set = stateInfoMap.get(state);
-      set.add(config);
-
-      if (!partitionInfoMap.containsKey(stateUnitKey)) {
-        partitionInfoMap.put(stateUnitKey, new PartitionInfo());
-      }
-      PartitionInfo stateUnitKeyInfo = partitionInfoMap.get(stateUnitKey);
-      stateUnitKeyInfo.addEntry(state, config);
-    }
-
-    public Set<InstanceConfig> getInstances(String state) {
-      Set<InstanceConfig> instanceSet = stateInfoMap.get(state);
-      return instanceSet;
-    }
-
-    PartitionInfo get(String stateUnitKey) {
-      return partitionInfoMap.get(stateUnitKey);
-    }
+  public void refresh(Collection<ExternalView> externalViews,
+      Collection<InstanceConfig> instanceConfigs, Collection<LiveInstance> liveInstances) {
+    RoutingTable newRoutingTable = new RoutingTable(externalViews, instanceConfigs, liveInstances);
+    _routingTableRef.set(newRoutingTable);
   }
 
-  /**
-   * Class to store instances, partitions and their states for each resource group.
-   */
-  class ResourceGroupInfo {
-    // aggregated partitions and instances info for all resources in the resource group.
-    ResourceInfo aggregatedResourceInfo;
-
-    // <ResourceTag, ResourceInfo> maps resource tag to the resource with the tag
-    // in this resource group.
-    // Each ResourceInfo saves only partitions and instances for that resource.
-    Map<String, ResourceInfo> tagToResourceMap;
-
-    public ResourceGroupInfo() {
-      aggregatedResourceInfo = new ResourceInfo();
-      tagToResourceMap = new HashMap<String, ResourceInfo>();
-    }
-
-    public void addEntry(String resourceTag, String stateUnitKey, String state, InstanceConfig config) {
-      // add the new entry to the aggregated resource info
-      aggregatedResourceInfo.addEntry(stateUnitKey, state, config);
-
-      // add the entry to the resourceInfo with given tag
-      if (!tagToResourceMap.containsKey(resourceTag)) {
-        tagToResourceMap.put(resourceTag, new ResourceInfo());
-      }
-      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
-      resourceInfo.addEntry(stateUnitKey, state, config);
-    }
-
-    public Set<InstanceConfig> getInstances(String state) {
-      return aggregatedResourceInfo.getInstances(state);
-    }
-
-    public Set<InstanceConfig> getInstances(String state, String resourceTag) {
-      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
-      if (resourceInfo != null) {
-        return resourceInfo.getInstances(state);
-      }
-
-      return null;
-    }
-
-    PartitionInfo get(String stateUnitKey) {
-      return aggregatedResourceInfo.get(stateUnitKey);
-    }
-
-    PartitionInfo get(String stateUnitKey, String resourceTag) {
-      ResourceInfo resourceInfo = tagToResourceMap.get(resourceTag);
-      if (resourceInfo == null) {
-        return null;
+  private class RouterUpdater extends ClusterEventProcessor {
+    private final RoutingDataCache _dataCache;
+
+    public RouterUpdater(String clusterName) {
+      super("Helix-RouterUpdater-event_process");
+      _dataCache = new RoutingDataCache(clusterName);
+    }
+
+    @Override
+    protected void handleEvent(ClusterEvent event) {
+      NotificationContext changeContext = event.getAttribute(AttributeName.changeContext.name());
+      // session has expired clean up the routing table
+      if (changeContext.getType() == NotificationContext.Type.FINALIZE) {
+        reset();
+      } else {
+        // refresh routing table.
+        HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+        if (manager == null) {
+          logger.error("HelixManager is null for router update event : " + event);
+          throw new HelixException("HelixManager is null for router update event.");
+        }
+        _dataCache.refresh(manager.getHelixDataAccessor());
+        refresh(_dataCache.getExternalViews().values(), _dataCache.getInstanceConfigMap().values(),
+            _dataCache.getLiveInstances().values());
       }
-
-      return resourceInfo.get(stateUnitKey);
     }
-  }
 
-  class PartitionInfo {
-    Map<String, List<InstanceConfig>> stateInfoMap;
-
-    public PartitionInfo() {
-      stateInfoMap = new HashMap<String, List<InstanceConfig>>();
-    }
-
-    public void addEntry(String state, InstanceConfig config) {
-      if (!stateInfoMap.containsKey(state)) {
-        stateInfoMap.put(state, new ArrayList<InstanceConfig>());
+    public void queueEvent(NotificationContext context, ClusterEventType eventType,
+        HelixConstants.ChangeType changeType) {
+      ClusterEvent event = new ClusterEvent(_clusterName, eventType);
+      if (context == null || context.getType() != NotificationContext.Type.CALLBACK) {
+        _dataCache.requireFullRefresh();
+      } else {
+        _dataCache.notifyDataChange(changeType, context.getPathChanged());
       }
-      List<InstanceConfig> list = stateInfoMap.get(state);
-      list.add(config);
-    }
-
-    List<InstanceConfig> get(String state) {
-      return stateInfoMap.get(state);
-    }
 
-    boolean containsState(String state) {
-      return stateInfoMap.containsKey(state);
+      event.addAttribute(AttributeName.helixmanager.name(), context.getManager());
+      event.addAttribute(AttributeName.changeContext.name(), context);
+      queueEvent(event);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/TestRoutingTable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestRoutingTable.java b/helix-core/src/test/java/org/apache/helix/TestRoutingTable.java
index 9fc0b33..11f3701 100644
--- a/helix-core/src/test/java/org/apache/helix/TestRoutingTable.java
+++ b/helix-core/src/test/java/org/apache/helix/TestRoutingTable.java
@@ -138,7 +138,7 @@ public class TestRoutingTable {
   }
 
   @Test()
-  public void testStateUnitGroupDeletion() {
+  public void testStateUnitGroupDeletion() throws InterruptedException {
     List<InstanceConfig> instances;
     RoutingTableProvider routingTable = new RoutingTableProvider();
 
@@ -155,6 +155,7 @@ public class TestRoutingTable {
 
     externalViewList.clear();
     routingTable.onExternalViewChange(externalViewList, changeContext);
+    Thread.sleep(100);
     instances = routingTable.getInstances("TESTDB", "TESTDB_0", "MASTER");
     AssertJUnit.assertNotNull(instances);
     AssertJUnit.assertEquals(instances.size(), 0);

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/controller/stages/TestClusterEventBlockingQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestClusterEventBlockingQueue.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestClusterEventBlockingQueue.java
index 802b39c..da86495 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestClusterEventBlockingQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestClusterEventBlockingQueue.java
@@ -25,6 +25,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.helix.common.ClusterEventBlockingQueue;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/integration/Spectator/TestRoutingTableProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/Spectator/TestRoutingTableProvider.java b/helix-core/src/test/java/org/apache/helix/integration/Spectator/TestRoutingTableProvider.java
new file mode 100644
index 0000000..aa731e5
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/Spectator/TestRoutingTableProvider.java
@@ -0,0 +1,173 @@
+package org.apache.helix.integration.Spectator;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.spectator.RoutingTableProvider;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
+import org.mockito.internal.util.collections.Sets;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestRoutingTableProvider extends ZkIntegrationTestBase {
+
+  static final String STATE_MODEL = BuiltInStateModelDefinitions.MasterSlave.name();
+  static final String TEST_DB = "TestDB";
+  static final String CLASS_NAME = TestRoutingTableProvider.class.getSimpleName();
+  static final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  static final int PARTICIPANT_NUMBER = 3;
+  static final int PARTICIPANT_START_PORT = 12918;
+
+  static final int PARTITION_NUMBER = 20;
+  static final int REPLICA_NUMBER = 3;
+
+  private HelixManager _spectator;
+  private List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  private List<String> _instances = new ArrayList<>();
+  private ClusterControllerManager _controller;
+  private HelixClusterVerifier _clusterVerifier;
+  private RoutingTableProvider _routingTableProvider;
+  private RoutingTableProvider _routingTableProvider2;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println(
+        "START " + getShortClassName() + " at " + new Date(System.currentTimeMillis()));
+
+    // setup storage cluster
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < PARTICIPANT_NUMBER; i++) {
+      String instance = PARTICIPANT_PREFIX + "_" + (PARTICIPANT_START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instance);
+      _instances.add(instance);
+    }
+
+    // start dummy participants
+    for (int i = 0; i < PARTICIPANT_NUMBER; i++) {
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _instances.get(i));
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    createDBInSemiAuto(_gSetupTool, CLUSTER_NAME, TEST_DB, _instances,
+        STATE_MODEL, PARTITION_NUMBER, REPLICA_NUMBER);
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // start speculator
+    _routingTableProvider = new RoutingTableProvider();
+    _spectator = HelixManagerFactory
+        .getZKHelixManager(CLUSTER_NAME, "spectator", InstanceType.SPECTATOR, ZK_ADDR);
+    _spectator.connect();
+    _spectator.addExternalViewChangeListener(_routingTableProvider);
+    _spectator.addLiveInstanceChangeListener(_routingTableProvider);
+    _spectator.addInstanceConfigChangeListener(_routingTableProvider);
+
+    _routingTableProvider2 = new RoutingTableProvider(_spectator);
+
+    _clusterVerifier = new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkClient(_gZkClient).build();
+    Assert.assertTrue(_clusterVerifier.verify());
+  }
+
+  @AfterClass
+  public void afterClass() {
+    // stop participants
+    for (MockParticipantManager p : _participants) {
+      p.syncStop();
+    }
+    _controller.syncStop();
+    _spectator.disconnect();
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+  }
+
+  @Test
+  public void testRoutingTable() {
+    Assert.assertEquals(_routingTableProvider.getLiveInstances().size(), _instances.size());
+    Assert.assertEquals(_routingTableProvider.getInstanceConfigs().size(), _instances.size());
+
+    Assert.assertEquals(_routingTableProvider2.getLiveInstances().size(), _instances.size());
+    Assert.assertEquals(_routingTableProvider2.getInstanceConfigs().size(), _instances.size());
+
+    validateRoutingTable(_routingTableProvider, Sets.newSet(_instances.get(0)),
+        Sets.newSet(_instances.get(1), _instances.get(2)));
+    validateRoutingTable(_routingTableProvider2, Sets.newSet(_instances.get(0)),
+        Sets.newSet(_instances.get(1), _instances.get(2)));
+  }
+
+  @Test(dependsOnMethods = { "testRoutingTable" })
+  public void testDisableInstance() throws InterruptedException {
+    // disable the master instance
+    String prevMasterInstance = _instances.get(0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, false);
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    validateRoutingTable(_routingTableProvider, Sets.newSet(_instances.get(1)),
+        Sets.newSet(_instances.get(2)));
+    validateRoutingTable(_routingTableProvider2, Sets.newSet(_instances.get(1)),
+        Sets.newSet(_instances.get(2)));
+  }
+
+  @Test(dependsOnMethods = { "testDisableInstance" })
+  public void testShutdownInstance() throws InterruptedException {
+    // reenable the first instance
+    String prevMasterInstance = _instances.get(0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, true);
+
+    // shutdown second instance.
+    _participants.get(1).syncStop();
+
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    Assert.assertEquals(_routingTableProvider.getLiveInstances().size(), _instances.size() - 1);
+    Assert.assertEquals(_routingTableProvider.getInstanceConfigs().size(), _instances.size());
+
+    Assert.assertEquals(_routingTableProvider2.getLiveInstances().size(), _instances.size() - 1);
+    Assert.assertEquals(_routingTableProvider2.getInstanceConfigs().size(), _instances.size());
+
+    validateRoutingTable(_routingTableProvider, Sets.newSet(_instances.get(0)),
+        Sets.newSet(_instances.get(2)));
+    validateRoutingTable(_routingTableProvider2, Sets.newSet(_instances.get(0)),
+        Sets.newSet(_instances.get(2)));
+  }
+
+  private void validateRoutingTable(RoutingTableProvider routingTableProvider,
+      Set<String> masterNodes, Set<String> slaveNodes) {
+    IdealState is =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, TEST_DB);
+    for (String p : is.getPartitionSet()) {
+      Set<String> masterInstances = new HashSet<>();
+      for (InstanceConfig config : routingTableProvider.getInstances(TEST_DB, p, "MASTER")) {
+        masterInstances.add(config.getInstanceName());
+      }
+
+      Set<String> slaveInstances = new HashSet<>();
+      for (InstanceConfig config : routingTableProvider.getInstances(TEST_DB, p, "SLAVE")) {
+        slaveInstances.add(config.getInstanceName());
+      }
+
+      Assert.assertEquals(masterInstances, masterNodes);
+      Assert.assertEquals(slaveInstances, slaveNodes);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/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 a24ded4..0c61b27 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
@@ -52,6 +52,7 @@ public class TestBasicSpectator extends ZkStandAloneCMTestBase implements
     boolean result =
         ClusterStateVerifier.verifyByPolling(new ClusterStateVerifier.BestPossAndExtViewZkVerifier(
             ZK_ADDR, CLUSTER_NAME));
+
     Assert.assertTrue(result);
 
     Assert.assertTrue(_externalViewChanges.containsKey("NextDB"));

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/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 e8640f4..47f8af9 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
@@ -33,6 +33,7 @@ import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.mock.participant.DummyProcess;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.OnlineOfflineSMD;
@@ -90,9 +91,9 @@ public class TestResourceGroupEndtoEnd extends ZkIntegrationTestBase {
 
     for (String tag : instanceGroupTags) {
       List<String> instances = _admin.getInstancesInClusterWithTag(CLUSTER_NAME, tag);
-      IdealState idealState =
-          createIdealState(TEST_DB, tag, instances, PARTITIONS, _replica,
-              IdealState.RebalanceMode.CUSTOMIZED.toString());
+      IdealState idealState = createIdealState(TEST_DB, tag, instances, PARTITIONS, _replica,
+          IdealState.RebalanceMode.CUSTOMIZED.toString(),
+          BuiltInStateModelDefinitions.OnlineOffline.name());
       _gSetupTool.addResourceToCluster(CLUSTER_NAME, idealState.getResourceName(), idealState);
     }
 
@@ -140,28 +141,6 @@ public class TestResourceGroupEndtoEnd extends ZkIntegrationTestBase {
     _spectator.disconnect();
   }
 
-  public IdealState createIdealState(String resourceGroupName, String instanceGroupTag,
-      List<String> instanceNames, int numPartition, int replica, String rebalanceMode) {
-    IdealState is =
-        _gSetupTool.createIdealStateForResourceGroup(resourceGroupName, instanceGroupTag,
-            numPartition, replica, rebalanceMode, "OnlineOffline");
-
-    // setup initial partition->instance mapping.
-    int nodeIdx = 0;
-    int numNode = instanceNames.size();
-    assert (numNode >= replica);
-    for (int i = 0; i < numPartition; i++) {
-      String partitionName = resourceGroupName + "_" + i;
-      for (int j = 0; j < replica; j++) {
-        is.setPartitionState(partitionName, instanceNames.get((nodeIdx + j) % numNode),
-            OnlineOfflineSMD.States.ONLINE.toString());
-      }
-      nodeIdx++;
-    }
-
-    return is;
-  }
-
   private void addInstanceGroup(String clusterName, String instanceTag, int numInstance) {
     List<String> instances = new ArrayList<String>();
     for (int i = 0; i < numInstance; i++) {

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
index 4e49502..cee331e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
@@ -21,6 +21,7 @@ package org.apache.helix.integration.common;
 
 import java.lang.reflect.Method;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
@@ -48,6 +49,7 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.OnlineOfflineSMD;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.model.builder.ConfigScopeBuilder;
 import org.apache.helix.tools.ClusterSetup;
@@ -217,6 +219,43 @@ public class ZkIntegrationTestBase {
     return idealState;
   }
 
+  protected IdealState createIdealState(String resourceGroupName, String instanceGroupTag,
+      List<String> instanceNames, int numPartition, int replica, String rebalanceMode,
+      String stateModelDef) {
+    IdealState is = _gSetupTool
+        .createIdealStateForResourceGroup(resourceGroupName, instanceGroupTag, numPartition,
+            replica, rebalanceMode, stateModelDef);
+
+    // setup initial partition->instance mapping.
+    int nodeIdx = 0;
+    int numNode = instanceNames.size();
+    assert (numNode >= replica);
+    for (int i = 0; i < numPartition; i++) {
+      String partitionName = resourceGroupName + "_" + i;
+      for (int j = 0; j < replica; j++) {
+        is.setPartitionState(partitionName, instanceNames.get((nodeIdx + j) % numNode),
+            OnlineOfflineSMD.States.ONLINE.toString());
+      }
+      nodeIdx++;
+    }
+
+    return is;
+  }
+
+  protected void createDBInSemiAuto(ClusterSetup clusterSetup, String clusterName, String dbName,
+      List<String> preferenceList, String stateModelDef, int numPartition, int replica) {
+    clusterSetup.addResourceToCluster(clusterName, dbName, numPartition, stateModelDef,
+        IdealState.RebalanceMode.SEMI_AUTO.toString());
+    clusterSetup.rebalanceStorageCluster(clusterName, dbName, replica);
+
+    IdealState is =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, dbName);
+    for (String p : is.getPartitionSet()) {
+      is.setPreferenceList(p, preferenceList);
+    }
+    clusterSetup.getClusterManagementTool().setResourceIdealState(clusterName, dbName, is);
+  }
+
   /**
    * Validate there should be always minimal active replica and top state replica for each partition.
    * Also make sure there is always some partitions with only active replica count.

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
index 6f7f369..cc168e9 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/MockParticipantManager.java
@@ -40,9 +40,9 @@ import org.slf4j.LoggerFactory;
 public class MockParticipantManager extends ZKHelixManager implements Runnable, ZkTestManager {
   private static Logger LOG = LoggerFactory.getLogger(MockParticipantManager.class);
 
-  private CountDownLatch _startCountDown = new CountDownLatch(1);
-  private CountDownLatch _stopCountDown = new CountDownLatch(1);
-  private CountDownLatch _waitStopCompleteCountDown = new CountDownLatch(1);
+  protected CountDownLatch _startCountDown = new CountDownLatch(1);
+  protected CountDownLatch _stopCountDown = new CountDownLatch(1);
+  protected CountDownLatch _waitStopCompleteCountDown = new CountDownLatch(1);
 
   protected MockMSModelFactory _msModelFactory = new MockMSModelFactory(null);
   protected DummyLeaderStandbyStateModelFactory _lsModelFactory =

http://git-wip-us.apache.org/repos/asf/helix/blob/093f7ab9/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
index e0f83e1..d7ea7a3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
@@ -95,8 +95,10 @@ public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
       _participants.add(participant);
     }
 
-    createDBInSemiAuto(DB_NAME_1, _instances);
-    createDBInSemiAuto(DB_NAME_2, _instances);
+    createDBInSemiAuto(_gSetupTool, CLUSTER_NAME, DB_NAME_1, _instances,
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITION_NUMBER, REPLICA_NUMBER);
+    createDBInSemiAuto(_gSetupTool, CLUSTER_NAME, DB_NAME_2, _instances,
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITION_NUMBER, REPLICA_NUMBER);
 
     // start controller
     String controllerName = CONTROLLER_PREFIX + "_0";
@@ -110,18 +112,6 @@ public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
     _accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
   }
 
-  private void createDBInSemiAuto(String dbName, List<String> preferenceList) {
-    _gSetupTool.addResourceToCluster(CLUSTER_NAME, dbName, PARTITION_NUMBER,
-        BuiltInStateModelDefinitions.MasterSlave.name(), IdealState.RebalanceMode.SEMI_AUTO.toString());
-    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, REPLICA_NUMBER);
-
-    IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
-    for (String p : is.getPartitionSet()) {
-      is.setPreferenceList(p, preferenceList);
-    }
-    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
-  }
-
   @Test
   public void testP2PStateTransitionDisabled() {
     // disable the master instance


[06/50] [abbrv] helix git commit: Add P2P (Participant-to-Participant) state-transition message support in Helix controller.

Posted by jx...@apache.org.
Add P2P (Participant-to-Participant) state-transition message support in Helix controller.


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

Branch: refs/heads/master
Commit: d0a3c0d1ea3a467564ed5ffd20bcd6a8e5a56678
Parents: 5117022
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Oct 9 14:52:19 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:30 2018 -0800

----------------------------------------------------------------------
 .../helix/api/config/HelixConfigProperty.java   |  43 ++++
 .../rebalancer/DelayedAutoRebalancer.java       |   2 +-
 .../controller/stages/ClusterDataCache.java     |  62 ++++-
 .../controller/stages/CurrentStateOutput.java   |  16 +-
 .../stages/MessageGenerationPhase.java          |   3 +-
 .../stages/MessageSelectionStage.java           |  82 +++----
 .../stages/ResourceComputationStage.java        |   4 +-
 .../controller/stages/TaskAssignmentStage.java  |  10 +-
 .../messaging/handling/BatchMessageHandler.java |   1 +
 .../handling/HelixStateTransitionHandler.java   |  16 +-
 .../helix/messaging/handling/HelixTask.java     |  52 +++-
 .../messaging/handling/HelixTaskExecutor.java   |   9 +-
 .../messaging/handling/HelixTaskResult.java     |   9 +
 .../org/apache/helix/model/ClusterConfig.java   |  23 ++
 .../org/apache/helix/model/CurrentState.java    |  11 +-
 .../java/org/apache/helix/model/Message.java    | 184 +++++++++++++-
 .../java/org/apache/helix/model/Resource.java   |  40 +++-
 .../org/apache/helix/model/ResourceConfig.java  |  33 ++-
 .../helix/model/StateModelDefinition.java       |  30 ++-
 .../helix/task/FixedTargetTaskRebalancer.java   |   3 +-
 .../helix/controller/stages/BaseStageTest.java  |  43 +++-
 .../stages/TestMsgSelectionStage.java           |   8 +-
 .../stages/TestResourceComputationStage.java    |   1 -
 .../helix/integration/TestZkReconnect.java      |   2 +-
 .../common/ZkIntegrationTestBase.java           |  20 ++
 .../manager/TestZkCallbackHandlerLeak.java      |   6 +-
 .../messaging/TestP2PMessageSemiAuto.java       | 240 +++++++++++++++++++
 .../paticipant/TestNodeOfflineTimeStamp.java    |   2 +
 .../TestCrushAutoRebalance.java                 |   7 +-
 .../org/apache/helix/manager/zk/TestZKUtil.java |   4 -
 .../helix/manager/zk/TestZkClusterManager.java  |   3 -
 .../TestP2PStateTransitionMessages.java         | 176 ++++++++++++++
 .../apache/helix/mock/MockBaseDataAccessor.java |   6 +-
 .../org/apache/helix/mock/MockHelixAdmin.java   |  14 +-
 .../mbeans/TestDisableResourceMbean.java        |   9 +-
 .../mbeans/TestTopStateHandoffMetrics.java      |   4 +-
 36 files changed, 1065 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/api/config/HelixConfigProperty.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/HelixConfigProperty.java b/helix-core/src/main/java/org/apache/helix/api/config/HelixConfigProperty.java
new file mode 100644
index 0000000..eff63cb
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/HelixConfigProperty.java
@@ -0,0 +1,43 @@
+package org.apache.helix.api.config;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
+
+/**
+ * This class defines all possible configuration options
+ * and its applicable config scopes (eg, Cluster/Resource/Instance/Partition config).
+ */
+public enum HelixConfigProperty {
+  P2P_MESSAGE_ENABLED(ConfigScopeProperty.CLUSTER, ConfigScopeProperty.RESOURCE);
+
+  Set<ConfigScopeProperty> _applicableScopes;
+
+  HelixConfigProperty(ConfigScopeProperty ...configScopeProperties) {
+    _applicableScopes = new HashSet<>(Arrays.asList(configScopeProperties));
+  }
+
+  public Set<ConfigScopeProperty> applicableScopes() {
+    return _applicableScopes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index a44aa11..adac235 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -310,7 +310,7 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
   private boolean isDelayRebalanceEnabled(IdealState idealState, ClusterConfig clusterConfig) {
     long delay = getRebalanceDelay(idealState, clusterConfig);
     return (delay > 0 && idealState.isDelayRebalanceEnabled() && clusterConfig
-        .isDelayRebalaceEnabled());
+        . isDelayRebalaceEnabled());
   }
 
   private ZNRecord getFinalDelayedMapping(IdealState idealState, ZNRecord newIdealMapping,

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index 8999ed7..1dd862d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -188,16 +188,20 @@ public class ClusterDataCache {
         accessor.getChildValuesMap(keyBuilder.stateModelDefs());
     _stateModelDefMap = new ConcurrentHashMap<>(stateDefMap);
     _constraintMap = accessor.getChildValuesMap(keyBuilder.constraints());
+    _clusterConfig = accessor.getProperty(keyBuilder.clusterConfig());
 
     refreshMessages(accessor);
     refreshCurrentStates(accessor);
 
-    _clusterConfig = accessor.getProperty(keyBuilder.clusterConfig());
+    // current state must be refreshed before refreshing relay messages
+    // because we need to use current state to validate all relay messages.
+    updateRelayMessages(_messageMap);
+
     if (_clusterConfig != null) {
       _idealStateRuleMap = _clusterConfig.getIdealStateRules();
     } else {
       _idealStateRuleMap = Maps.newHashMap();
-      LOG.error("Cluster config is null!");
+      LOG.warn("Cluster config is null!");
     }
 
     long endTime = System.currentTimeMillis();
@@ -275,6 +279,7 @@ public class ClusterDataCache {
         }
       }
     }
+
     _messageMap = Collections.unmodifiableMap(msgMap);
 
     if (LOG.isDebugEnabled()) {
@@ -284,6 +289,59 @@ public class ClusterDataCache {
     }
   }
 
+  // update all valid relay messages attached to existing state transition messages into message map.
+  private void updateRelayMessages(Map<String, Map<String, Message>> messageMap) {
+    List<Message> relayMessages = new ArrayList<>();
+    for (String instance : messageMap.keySet()) {
+      Map<String, Message> instanceMessages = messageMap.get(instance);
+      Map<String, Map<String, CurrentState>> instanceCurrentStateMap = _currentStateMap.get(instance);
+      if (instanceCurrentStateMap == null) {
+        continue;
+      }
+
+      for (Message message : instanceMessages.values()) {
+        if (message.hasRelayMessages()) {
+          String sessionId = message.getTgtSessionId();
+          String resourceName = message.getResourceName();
+          String partitionName = message.getPartitionName();
+          String targetState = message.getToState();
+          String instanceSessionId = _liveInstanceMap.get(instance).getSessionId();
+
+          if (!instanceSessionId.equals(sessionId)) {
+            continue;
+          }
+
+          Map<String, CurrentState> sessionCurrentStateMap = instanceCurrentStateMap.get(sessionId);
+          if (sessionCurrentStateMap == null) {
+            continue;
+          }
+          CurrentState currentState = sessionCurrentStateMap.get(resourceName);
+          if (currentState == null || !targetState.equals(currentState.getState(partitionName))) {
+            continue;
+          }
+          long transitionCompleteTime = currentState.getEndTime(partitionName);
+
+          for (Message msg : message.getRelayMessages().values()) {
+            msg.setRelayTime(transitionCompleteTime);
+            if (!message.isExpired()) {
+              relayMessages.add(msg);
+            }
+          }
+        }
+      }
+    }
+
+    for (Message message : relayMessages) {
+      String instance = message.getTgtName();
+      Map<String, Message> instanceMessages = messageMap.get(instance);
+      if (instanceMessages == null) {
+        instanceMessages = new HashMap<>();
+        messageMap.put(instance, instanceMessages);
+      }
+      instanceMessages.put(message.getId(), message);
+    }
+  }
+
   private void refreshCurrentStates(HelixDataAccessor accessor) {
     refreshCurrentStatesCache(accessor);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java
index 3821303..97420e5 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateOutput.java
@@ -54,13 +54,13 @@ public class CurrentStateOutput {
   private final Map<String, CurrentState> _curStateMetaMap;
 
   public CurrentStateOutput() {
-    _currentStateMap = new HashMap<String, Map<Partition, Map<String, String>>>();
-    _pendingStateMap = new HashMap<String, Map<Partition, Map<String, Message>>>();
-    _cancellationStateMap = new HashMap<String, Map<Partition, Map<String, Message>>>();
-    _resourceStateModelMap = new HashMap<String, String>();
-    _curStateMetaMap = new HashMap<String, CurrentState>();
-    _requestedStateMap = new HashMap<String, Map<Partition, Map<String, String>>>();
-    _infoMap = new HashMap<String, Map<Partition, Map<String, String>>>();
+    _currentStateMap = new HashMap<>();
+    _pendingStateMap = new HashMap<>();
+    _cancellationStateMap = new HashMap<>();
+    _resourceStateModelMap = new HashMap<>();
+    _curStateMetaMap = new HashMap<>();
+    _requestedStateMap = new HashMap<>();
+    _infoMap = new HashMap<>();
   }
 
   public void setResourceStateModelDef(String resourceName, String stateModelDefName) {
@@ -331,7 +331,7 @@ public class CurrentStateOutput {
 
   private Map<String, Integer> getPartitionCountWithState(String resourceStateModel, String state,
       Map<String, Map<Partition, Map<String, Object>>> stateMap) {
-    Map<String, Integer> currentPartitionCount = new HashMap<String, Integer>();
+    Map<String, Integer> currentPartitionCount = new HashMap<>();
     for (String resource : stateMap.keySet()) {
       String stateModel = _resourceStateModelMap.get(resource);
       if ((stateModel != null && stateModel.equals(resourceStateModel)) || (stateModel == null

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
index ca0adfe..d60c20e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
-import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
@@ -187,7 +186,7 @@ public class MessageGenerationPhase extends AbstractBaseStage {
               message.setExecutionTimeout(timeout);
             }
 
-            message.getRecord().setSimpleField("ClusterEventName", event.getEventType().name());
+            message.setAttribute(Message.Attributes.ClusterEventName, event.getEventType().name());
             // output.addMessage(resourceName, partition, message);
             if (!messageMap.containsKey(desiredState)) {
               messageMap.put(desiredState, new ArrayList<Message>());

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
index 94b645d..1a94dcb 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
@@ -22,17 +22,20 @@ package org.apache.helix.controller.stages;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,7 +43,7 @@ import org.slf4j.LoggerFactory;
 public class MessageSelectionStage extends AbstractBaseStage {
   private static final Logger LOG = LoggerFactory.getLogger(MessageSelectionStage.class);
 
-  public static class Bounds {
+  protected static class Bounds {
     private int upper;
     private int lower;
 
@@ -49,26 +52,6 @@ public class MessageSelectionStage extends AbstractBaseStage {
       this.upper = upper;
     }
 
-    public void increaseUpperBound() {
-      upper++;
-    }
-
-    public void increaseLowerBound() {
-      lower++;
-    }
-
-    public void decreaseUpperBound() {
-      upper--;
-    }
-
-    public void decreaseLowerBound() {
-      lower--;
-    }
-
-    public int getLowerBound() {
-      return lower;
-    }
-
     public int getUpperBound() {
       return upper;
     }
@@ -100,11 +83,11 @@ public class MessageSelectionStage extends AbstractBaseStage {
           computeStateConstraints(stateModelDef, idealState, cache);
       for (Partition partition : resource.getPartitions()) {
         List<Message> messages = messageGenOutput.getMessages(resourceName, partition);
-        List<Message> selectedMessages =
-            selectMessages(cache.getLiveInstances(),
-                currentStateOutput.getCurrentStateMap(resourceName, partition),
-                currentStateOutput.getPendingMessageMap(resourceName, partition), messages,
-                stateConstraints, stateTransitionPriorities, stateModelDef.getInitialState());
+        List<Message> selectedMessages = selectMessages(cache.getLiveInstances(),
+            currentStateOutput.getCurrentStateMap(resourceName, partition),
+            currentStateOutput.getPendingMessageMap(resourceName, partition), messages,
+            stateConstraints, stateTransitionPriorities, stateModelDef,
+            resource.isP2PMessageEnabled());
         output.addMessages(resourceName, partition, selectedMessages);
       }
     }
@@ -124,34 +107,36 @@ public class MessageSelectionStage extends AbstractBaseStage {
   }
 
   // TODO: This method deserves its own class. The class should not understand helix but
-  // just be
-  // able to solve the problem using the algo. I think the method is following that but if
-  // we don't move it to another class its quite easy to break that contract
+  // just be able to solve the problem using the algo. I think the method is following that
+  // but if we don't move it to another class its quite easy to break that contract
   /**
    * greedy message selection algorithm: 1) calculate CS+PS state lower/upper-bounds 2)
    * group messages by state transition and sorted by priority 3) from highest priority to
    * lowest, for each message group with the same transition add message one by one and
    * make sure state constraint is not violated update state lower/upper-bounds when a new
-   * message is selected
+   * message is selected.
+   *
+   * @param liveInstances
    * @param currentStates
-   * @param pendingStates
+   * @param pendingMessages
    * @param messages
    * @param stateConstraints
-   *          : STATE -> bound (lower:upper)
    * @param stateTransitionPriorities
-   *          : FROME_STATE-TO_STATE -> priority
-   * @return: selected messages
+   * @param stateModelDef
+   * @return
    */
   List<Message> selectMessages(Map<String, LiveInstance> liveInstances,
       Map<String, String> currentStates, Map<String, Message> pendingMessages,
       List<Message> messages, Map<String, Bounds> stateConstraints,
-      final Map<String, Integer> stateTransitionPriorities, String initialState) {
+      final Map<String, Integer> stateTransitionPriorities, StateModelDefinition stateModelDef,
+      boolean p2pMessageEnabled) {
     if (messages == null || messages.isEmpty()) {
       return Collections.emptyList();
     }
-    List<Message> selectedMessages = new ArrayList<Message>();
-    Map<String, Integer> stateCnts = new HashMap<String, Integer>();
+    List<Message> selectedMessages = new ArrayList<>();
+    Map<String, Integer> stateCnts = new HashMap<>();
 
+    String initialState = stateModelDef.getInitialState();
     // count currentState, if no currentState, count as in initialState
     for (String instance : liveInstances.keySet()) {
       String state = initialState;
@@ -171,7 +156,10 @@ public class MessageSelectionStage extends AbstractBaseStage {
 
     // group messages based on state transition priority
     Map<Integer, List<Message>> messagesGroupByStateTransitPriority =
-        new TreeMap<Integer, List<Message>>();
+        new TreeMap<>();
+
+    /* record all state transition messages that transition a replica from top-state */
+    List<Message> fromTopStateMessages = new LinkedList<>();
     for (Message message : messages) {
       if (message.getMsgType().equals(Message.MessageType.STATE_TRANSITION_CANCELLATION.name())) {
         selectedMessages.add(message);
@@ -190,6 +178,10 @@ public class MessageSelectionStage extends AbstractBaseStage {
         messagesGroupByStateTransitPriority.put(priority, new ArrayList<Message>());
       }
       messagesGroupByStateTransitPriority.get(priority).add(message);
+
+      if (fromState.equals(stateModelDef.getTopState())) {
+        fromTopStateMessages.add(message);
+      }
     }
 
     // select messages
@@ -200,8 +192,18 @@ public class MessageSelectionStage extends AbstractBaseStage {
         if (stateConstraints.containsKey(toState)) {
           int newCnt = (stateCnts.containsKey(toState) ? stateCnts.get(toState) + 1 : 1);
           if (newCnt > stateConstraints.get(toState).getUpperBound()) {
-            LOG.info("Reach upper_bound: " + stateConstraints.get(toState).getUpperBound()
-                + ", not send message: " + message);
+            if (p2pMessageEnabled && toState.equals(stateModelDef.getTopState())
+                && stateModelDef.isSingleTopStateModel()) {
+              // attach this message as a relay message to the message to transition off current top-state replica
+              if (fromTopStateMessages.size() > 0) {
+                Message fromTopStateMsg = fromTopStateMessages.get(0);
+                fromTopStateMsg.attachRelayMessage(message.getTgtName(), message);
+                fromTopStateMessages.remove(0);
+              }
+            } else {
+              // reach upper-bound of message for the topState, will not send the message
+              LOG.info("Reach upper_bound: " + stateConstraints.get(toState).getUpperBound() + ", not send message: " + message);
+            }
             continue;
           }
         }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
index c6b4054..a6c2779 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
@@ -56,7 +56,6 @@ public class ResourceComputationStage extends AbstractBaseStage {
     Map<String, Resource> resourceMap = new LinkedHashMap<String, Resource>();
     Map<String, Resource> resourceToRebalance = new LinkedHashMap<>();
 
-
     if (idealStates != null && idealStates.size() > 0) {
       for (IdealState idealState : idealStates.values()) {
         if (idealState == null) {
@@ -65,7 +64,8 @@ public class ResourceComputationStage extends AbstractBaseStage {
         Set<String> partitionSet = idealState.getPartitionSet();
         String resourceName = idealState.getResourceName();
         if (!resourceMap.containsKey(resourceName)) {
-          Resource resource = new Resource(resourceName);
+          Resource resource = new Resource(resourceName, cache.getClusterConfig(),
+              cache.getResourceConfig(resourceName));
           resourceMap.put(resourceName, resource);
 
           if (!idealState.isValid() && !cache.isTaskCache()

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
index ee0f1e5..dfa747e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/TaskAssignmentStage.java
@@ -144,7 +144,15 @@ public class TaskAssignmentStage extends AbstractBaseStage {
           "Sending Message " + message.getMsgId() + " to " + message.getTgtName() + " transit "
               + message.getResourceName() + "." + message.getPartitionName() + "|" + message
               .getPartitionNames() + " from:" + message.getFromState() + " to:" + message
-              .getToState() + ", Message type ");
+              .getToState() + ", relayMessages: " + message.getRelayMessages().size());
+      if (message.hasRelayMessages()) {
+        for (Message msg : message.getRelayMessages().values()) {
+          logger.info("Sending Relay Message " + msg.getMsgId() + " to " + msg.getTgtName() + " transit "
+              + msg.getResourceName() + "." + msg.getPartitionName() + "|" + msg.getPartitionNames() + " from:"
+              + msg.getFromState() + " to:" + msg.getToState() + ", relayFrom: " + msg.getRelaySrcHost()
+              + ", attached to message: " + message.getMsgId());
+        }
+      }
 
       keys.add(keyBuilder.message(message.getTgtName(), message.getId()));
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
index 9fed082..3979a4b 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
@@ -197,6 +197,7 @@ public class BatchMessageHandler extends MessageHandler {
         result.setSuccess(isBatchTaskSucceed);
       }
 
+      result.setCompleteTime(System.currentTimeMillis());
       // pass task-result to post-handle-msg
       _notificationContext.add(MapKey.HELIX_TASK_RESULT.toString(), result);
       postHandleMessage();

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
index 26df112..ece7ac7 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
@@ -94,6 +94,12 @@ public class HelixStateTransitionHandler extends MessageHandler {
       throw new HelixException(errorMessage);
     }
 
+    logger.info(
+        "handling message: " + _message.getMsgId() + " transit " + _message.getResourceName()
+            + "." + _message.getPartitionName() + "|" + _message.getPartitionNames() + " from:"
+            + _message.getFromState() + " to:" + _message.getToState() + ", relayedFrom: "
+            + _message.getRelaySrcHost());
+
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
 
     String partitionName = _message.getPartitionName();
@@ -188,9 +194,16 @@ public class HelixStateTransitionHandler extends MessageHandler {
 
     // Set the INFO property and mark the end time, previous state of the state transition
     _currentStateDelta.setInfo(partitionKey, taskResult.getInfo());
-    _currentStateDelta.setEndTime(partitionKey, System.currentTimeMillis());
+    _currentStateDelta.setEndTime(partitionKey, taskResult.getCompleteTime());
     _currentStateDelta.setPreviousState(partitionKey, _message.getFromState());
 
+    // add host name this state transition is triggered by.
+    if (Message.MessageType.RELAYED_MESSAGE.name().equals(_message.getMsgSubType())) {
+      _currentStateDelta.setTriggerHost(partitionKey, _message.getRelaySrcHost());
+    } else {
+      _currentStateDelta.setTriggerHost(partitionKey, _message.getMsgSrc());
+    }
+
     if (taskResult.isSuccess()) {
       // String fromState = message.getFromState();
       String toState = _message.getToState();
@@ -339,6 +352,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
         }
       }
 
+      taskResult.setCompleteTime(System.currentTimeMillis());
       // add task result to context for postHandling
       context.add(MapKey.HELIX_TASK_RESULT.toString(), taskResult);
       postHandleMessage();

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
index 3272ca4..7b1853f 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
@@ -20,15 +20,19 @@ package org.apache.helix.messaging.handling;
  */
 
 import java.util.Date;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.helix.AccessOption;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRollbackException;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
+import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.PropertyPathBuilder;
 import org.apache.helix.messaging.handling.MessageHandler.ErrorCode;
 import org.apache.helix.messaging.handling.MessageHandler.ErrorType;
 import org.apache.helix.model.Message;
@@ -37,6 +41,7 @@ import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.monitoring.StateTransitionContext;
 import org.apache.helix.monitoring.StateTransitionDataPoint;
 import org.apache.helix.monitoring.mbeans.ParticipantMessageMonitor;
+import org.apache.helix.task.TaskResult;
 import org.apache.helix.util.StatusUpdateUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -161,8 +166,12 @@ public class HelixTask implements MessageTask {
         }
       }
 
+      // forward relay messages attached to this message to other participants
+      if (taskResult.isSuccess()) {
+        forwardRelayMessages(accessor, _message, taskResult.getCompleteTime());
+      }
+
       if (_message.getAttribute(Attributes.PARENT_MSG_ID) == null) {
-        // System.err.println("\t[dbg]remove msg: " + getTaskId());
         removeMessageFromZk(accessor, _message);
         reportMessageStat(_manager, _message, taskResult);
         sendReply(accessor, _message, taskResult);
@@ -196,11 +205,46 @@ public class HelixTask implements MessageTask {
 
   private void removeMessageFromZk(HelixDataAccessor accessor, Message message) {
     Builder keyBuilder = accessor.keyBuilder();
+    PropertyKey msgKey;
     if (message.getTgtName().equalsIgnoreCase("controller")) {
-      // TODO: removeProperty returns boolean
-      accessor.removeProperty(keyBuilder.controllerMessage(message.getMsgId()));
+      msgKey = keyBuilder.controllerMessage(message.getMsgId());
     } else {
-      accessor.removeProperty(keyBuilder.message(_manager.getInstanceName(), message.getMsgId()));
+      msgKey = keyBuilder.message(_manager.getInstanceName(), message.getMsgId());
+    }
+    boolean success = accessor.removeProperty(msgKey);
+    if (!success) {
+      logger.warn("Failed to delete message " + message.getId() + " from zk!");
+    }
+  }
+
+  private void forwardRelayMessages(HelixDataAccessor accessor, Message message,
+      long taskCompletionTime) {
+    if (message.hasRelayMessages()) {
+      Map<String, Message> relayMessages = message.getRelayMessages();
+      Builder keyBuilder = accessor.keyBuilder();
+
+      // Ignore all relay messages if participant's session has changed.
+      if (!_manager.getSessionId().equals(message.getTgtSessionId())) {
+        return;
+      }
+
+      for (String instance : relayMessages.keySet()) {
+        Message msg = relayMessages.get(instance);
+        if (msg.getMsgSubType().equals(MessageType.RELAYED_MESSAGE.name())) {
+          msg.setRelayTime(taskCompletionTime);
+          if (msg.isExpired()) {
+            logger.info(
+                "Relay message expired, ignore it! " + msg.getId() + " to instance " + instance);
+            continue;
+          }
+          PropertyKey msgKey = keyBuilder.message(instance, msg.getId());
+          boolean success = accessor.getBaseDataAccessor()
+              .create(msgKey.getPath(), msg.getRecord(), AccessOption.PERSISTENT);
+          if (!success) {
+            logger.warn("Failed to send relay message " + msg.getId() + " to " + instance);
+          }
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index 4c67a5e..c0be583 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -790,8 +790,15 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         continue;
       }
 
-      String tgtSessionId = message.getTgtSessionId();
+      if (message.isExpired()) {
+        LOG.info(
+            "Dropping expired message. mid: " + message.getId() + ", from: " + message.getMsgSrc() + " relayed from: "
+                + message.getRelaySrcHost());
+        reportAndRemoveMessage(message, accessor, instanceName, ProcessedMessageState.DISCARDED);
+        continue;
+      }
 
+      String tgtSessionId = message.getTgtSessionId();
       // sessionId mismatch normally means message comes from expired session, just remove it
       if (!sessionId.equals(tgtSessionId) && !tgtSessionId.equals("*")) {
         String warningMessage =

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
index 5ed6140..da96eaf 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskResult.java
@@ -31,6 +31,7 @@ public class HelixTaskResult {
   private Map<String, String> _taskResultMap = new HashMap<String, String>();
   private boolean _interrupted = false;
   Exception _exception = null;
+  private long _completeTime = -1;
 
   public boolean isSuccess() {
     return _success;
@@ -83,4 +84,12 @@ public class HelixTaskResult {
   public Exception getException() {
     return _exception;
   }
+
+  public long getCompleteTime() {
+    return _completeTime;
+  }
+
+  public void setCompleteTime(long completeTime) {
+    _completeTime = completeTime;
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
index 2a97145..fad253d 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.config.HelixConfigProperty;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
 
@@ -492,6 +493,28 @@ public class ClusterConfig extends HelixProperty {
   }
 
   /**
+   * Whether the P2P state transition message is enabled for all resources in this cluster. By
+   * default it is disabled if not set.
+   *
+   * @return
+   */
+  public boolean isP2PMessageEnabled() {
+    return _record.getBooleanField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name(), false);
+  }
+
+  /**
+   * Enable P2P state transition message for all resources in this cluster. P2P State Transition
+   * message can reduce the top-state replica unavailable time during top-state handoff period. This
+   * only applies for those resources with state models that only have a single top-state replica,
+   * such as MasterSlave or LeaderStandy models. By default P2P message is disabled if not set.
+   *
+   * @param enabled
+   */
+  public void enableP2PMessage(boolean enabled) {
+    _record.setBooleanField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name(), enabled);
+  }
+
+  /**
    * Get IdealState rules defined in the cluster config.
    *
    * @return

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
index 34b525e..c227060 100644
--- a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
@@ -47,7 +47,8 @@ public class CurrentState extends HelixProperty {
     RESOURCE,
     START_TIME,
     END_TIME,
-    PREVIOUS_STATE // ,
+    PREVIOUS_STATE,
+    TRIGGERED_BY// ,
              // BUCKET_SIZE
   }
 
@@ -134,6 +135,10 @@ public class CurrentState extends HelixProperty {
     return endTime == null ? -1L : Long.parseLong(endTime);
   }
 
+  public String getTriggerHost(String partitionName) {
+    return getProperty(partitionName, CurrentStateProperty.TRIGGERED_BY);
+  }
+
   public String getPreviousState(String partitionName) {
     return getProperty(partitionName, CurrentStateProperty.PREVIOUS_STATE);
   }
@@ -187,6 +192,10 @@ public class CurrentState extends HelixProperty {
     setProperty(partitionName, CurrentStateProperty.END_TIME, String.valueOf(endTime));
   }
 
+  public void setTriggerHost(String partitionName, String triggerHost) {
+    setProperty(partitionName, CurrentStateProperty.TRIGGERED_BY, triggerHost);
+  }
+
   public void setPreviousState(String partitionName, String state) {
     setProperty(partitionName, CurrentStateProperty.PREVIOUS_STATE, state);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/Message.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java
index 668242a..d987c54 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Message.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Message.java
@@ -19,12 +19,16 @@ package org.apache.helix.model;
  * under the License.
  */
 
+import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 
 import org.apache.helix.HelixException;
@@ -50,7 +54,9 @@ public class Message extends HelixProperty {
     TASK_REPLY,
     NO_OP,
     PARTICIPANT_ERROR_REPORT,
-    PARTICIPANT_SESSION_CHANGE
+    PARTICIPANT_SESSION_CHANGE,
+    CHAINED_MESSAGE, // this is a message subtype
+    RELAYED_MESSAGE
   }
 
   /**
@@ -58,6 +64,7 @@ public class Message extends HelixProperty {
    */
   public enum Attributes {
     MSG_ID,
+    RELAY_MSG_ID,
     SRC_SESSION_ID,
     TGT_SESSION_ID,
     SRC_NAME,
@@ -84,7 +91,12 @@ public class Message extends HelixProperty {
     STATE_MODEL_FACTORY_NAME,
     BUCKET_SIZE,
     PARENT_MSG_ID, // used for group message mode
-    INNER_MESSAGE
+    ClusterEventName,
+    INNER_MESSAGE,
+    RELAY_PARTICIPANTS,
+    RELAY_TIME,
+    RELAY_FROM,
+    EXPIRY_PERIOD
   }
 
   /**
@@ -96,6 +108,9 @@ public class Message extends HelixProperty {
     UNPROCESSABLE // get exception when create handler
   }
 
+  // default expiry time period for a relay message.
+  public static final long RELAY_MESSAGE_DEFAULT_EXPIRY = 5 * 1000;  //5 second
+
   /**
    * Compares the creation time of two Messages
    */
@@ -665,6 +680,171 @@ public class Message extends HelixProperty {
   }
 
   /**
+   * Get the completion time of previous task associated with this message.
+   * This applies only when this is a relay message,
+   * which specified the completion time of the task running on the participant that sent this relay message.
+   *
+   * @return
+   */
+  public long getRelayTime() {
+    return _record.getLongField(Attributes.RELAY_TIME.name(), -1);
+  }
+
+  /**
+   * Set the completion time of previous task associated with this message.
+   * This applies only when this is a relay message,
+   * which specified the completion time of the task running on the participant that sent this relay message.
+   *
+   * @param completionTime
+   */
+  public void setRelayTime(long completionTime) {
+    _record.setLongField(Attributes.RELAY_TIME.name(), completionTime);
+  }
+
+  /**
+   * Attach a relayed message and its destination participant to this message.
+   *
+   * WARNNING: only content in SimpleFields of relayed message will be carried over and sent,
+   * all contents in either ListFields or MapFields will be ignored.
+   *
+   * @param instance destination participant name
+   * @param message relayed message.
+   */
+  public void attachRelayMessage(String instance, Message message) {
+    List<String> relayList = _record.getListField(Attributes.RELAY_PARTICIPANTS.name());
+    if (relayList == null) {
+      relayList = Collections.EMPTY_LIST;
+    }
+    Set<String> relayParticipants = new LinkedHashSet<>(relayList);
+    relayParticipants.add(instance);
+    Map<String, String> messageInfo = message.getRecord().getSimpleFields();
+    messageInfo.put(Attributes.RELAY_MSG_ID.name(), message.getId());
+    messageInfo.put(Attributes.MSG_SUBTYPE.name(), MessageType.RELAYED_MESSAGE.name());
+    messageInfo.put(Attributes.RELAY_FROM.name(), getTgtName());
+    messageInfo
+        .put(Attributes.EXPIRY_PERIOD.name(), String.valueOf(RELAY_MESSAGE_DEFAULT_EXPIRY));
+    _record.setMapField(instance, messageInfo);
+    _record.setListField(Attributes.RELAY_PARTICIPANTS.name(),
+        Lists.newArrayList(relayParticipants));
+  }
+
+  /**
+   * Get relay message attached for the given instance.
+   *
+   * @param instance
+   * @return null if no message for the instance
+   */
+  public Message getRelayMessage(String instance) {
+    Map<String, String> messageInfo = _record.getMapField(instance);
+    if (messageInfo != null) {
+      String id = messageInfo.get(Attributes.RELAY_MSG_ID.name());
+      if (id == null) {
+        id = messageInfo.get(Attributes.MSG_ID.name());
+        if (id == null) {
+          return null;
+        }
+      }
+      ZNRecord record = new ZNRecord(id);
+      record.setSimpleFields(messageInfo);
+      return new Message(record);
+    }
+
+    return null;
+  }
+
+  public String getRelaySrcHost() {
+    return _record.getSimpleField(Attributes.RELAY_FROM.name());
+  }
+
+  /**
+   * Get all relay messages attached to this message as a map (instance->message).
+   *
+   * @return map of instanceName->message, empty map if none.
+   */
+  public Map<String, Message> getRelayMessages() {
+    Map<String, Message> relayMessageMap = new HashMap<>();
+    List<String> relayParticipants = _record.getListField(Attributes.RELAY_PARTICIPANTS.name());
+    if (relayParticipants != null) {
+      for (String p : relayParticipants) {
+        Message msg = getRelayMessage(p);
+        if (p != null) {
+          relayMessageMap.put(p, msg);
+        }
+      }
+    }
+
+    return relayMessageMap;
+  }
+
+  /**
+   * Whether there are any relay message attached to this message.
+   *
+   * @return
+   */
+  public boolean hasRelayMessages() {
+    List<String> relayHosts = _record.getListField(Attributes.RELAY_PARTICIPANTS.name());
+    return (relayHosts != null && relayHosts.size() > 0);
+  }
+
+  /**
+   * Whether this message is a relay message.
+   * @return
+   */
+  public boolean isRelayMessage() {
+    String subType = _record.getStringField(Attributes.MSG_SUBTYPE.name(), null);
+    String relayFrom = _record.getStringField(Attributes.RELAY_FROM.name(), null);
+    return MessageType.RELAYED_MESSAGE.name().equals(subType) && (relayFrom != null);
+  }
+
+  /**
+   * Whether a message is expired.
+   *
+   * A message is expired if:
+   *   1) creationTime + expiryPeriod > current time
+   *   or
+   *   2) relayTime + expiryPeriod > current time iff it is relay message.
+   *
+   * @return
+   */
+  public boolean isExpired() {
+    long expiry = getExpiryPeriod();
+    if (expiry < 0) {
+      return false;
+    }
+
+    long current = System.currentTimeMillis();
+
+    // use relay time if this is a relay message
+    if (isRelayMessage()) {
+      long relayTime = getRelayTime();
+      return relayTime <= 0 || (relayTime + expiry < current);
+    }
+
+    return getCreateTimeStamp() + expiry < current;
+  }
+
+  /**
+   * Get the expiry period (in milliseconds)
+   *
+   * @return
+   */
+  public long getExpiryPeriod() {
+    return _record.getLongField(Attributes.EXPIRY_PERIOD.name(), -1);
+  }
+
+  /**
+   * Set expiry period for this message.
+   * A message will be expired after this period of time from either its 1) creationTime or 2)
+   * relayTime if it is relay message.
+   * Default is -1 if it is not set.
+   *
+   * @param expiry
+   */
+  public void setExpiryPeriod(long expiry) {
+    _record.setLongField(Attributes.EXPIRY_PERIOD.name(), expiry);
+  }
+
+  /**
    * Check if this message is targetted for a controller
    * @return true if this is a controller message, false otherwise
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/Resource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Resource.java b/helix-core/src/main/java/org/apache/helix/model/Resource.java
index 3af830a..b911244 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Resource.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Resource.java
@@ -24,16 +24,14 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.helix.HelixConstants;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import org.apache.helix.api.config.HelixConfigProperty;
 /**
  * A resource contains a set of partitions and its replicas are managed by a state model
  */
 public class Resource {
-  private static Logger LOG = LoggerFactory.getLogger(Resource.class);
-
   private final String _resourceName;
+  private ClusterConfig _clusterConfig;
+  private ResourceConfig _resourceConfig;
   private final Map<String, Partition> _partitionMap;
   private String _stateModelDefRef;
   private String _stateModelFactoryName;
@@ -48,7 +46,18 @@ public class Resource {
    */
   public Resource(String resourceName) {
     this._resourceName = resourceName;
-    this._partitionMap = new LinkedHashMap<String, Partition>();
+    this._partitionMap = new LinkedHashMap<>();
+  }
+
+  /**
+   * Instantiate a resource by its name
+   *
+   * @param resourceName the name of the resource that identifies it
+   */
+  public Resource(String resourceName, ClusterConfig clusterConfig, ResourceConfig resourceConfig) {
+    this(resourceName);
+    _clusterConfig = clusterConfig;
+    _resourceConfig = resourceConfig;
   }
 
   /**
@@ -185,6 +194,25 @@ public class Resource {
     _resourceGroupName = resourceGroupName;
   }
 
+  /**
+   * Whether P2P state transition message is enabled.
+   *
+   * @return
+   */
+  public boolean isP2PMessageEnabled() {
+    String enabledInResource = _resourceConfig != null ?
+        _resourceConfig.getRecord().getSimpleField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name()) : null;
+
+    if (enabledInResource != null) {
+      return Boolean.valueOf(enabledInResource);
+    }
+
+    String enabledInCluster = _clusterConfig != null ?
+        _clusterConfig.getRecord().getSimpleField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name()) : null;
+
+    return enabledInCluster != null ? Boolean.valueOf(enabledInCluster) : false;
+  }
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
index 8278be3..274640c 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
@@ -20,13 +20,13 @@ package org.apache.helix.model;
  */
 
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import java.util.TreeMap;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.config.HelixConfigProperty;
 import org.apache.helix.api.config.RebalanceConfig;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
 import org.slf4j.Logger;
@@ -61,7 +61,6 @@ public class ResourceConfig extends HelixProperty {
   }
 
   private static final Logger _logger = LoggerFactory.getLogger(ResourceConfig.class.getName());
-
   /**
    * Instantiate for a specific instance
    *
@@ -95,13 +94,18 @@ public class ResourceConfig extends HelixProperty {
       Boolean helixEnabled, String resourceGroupName, String resourceType,
       Boolean groupRoutingEnabled, Boolean externalViewDisabled,
       RebalanceConfig rebalanceConfig, StateTransitionTimeoutConfig stateTransitionTimeoutConfig,
-      Map<String, List<String>> listFields, Map<String, Map<String, String>> mapFields) {
+      Map<String, List<String>> listFields, Map<String, Map<String, String>> mapFields,
+      Boolean p2pMessageEnabled) {
     super(resourceId);
 
     if (monitorDisabled != null) {
       _record.setBooleanField(ResourceConfigProperty.MONITORING_DISABLED.name(), monitorDisabled);
     }
 
+    if (p2pMessageEnabled != null) {
+      _record.setBooleanField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name(), p2pMessageEnabled);
+    }
+
     if (numPartitions > 0) {
       _record.setIntField(ResourceConfigProperty.NUM_PARTITIONS.name(), numPartitions);
     }
@@ -180,6 +184,15 @@ public class ResourceConfig extends HelixProperty {
     return _record.getBooleanField(ResourceConfigProperty.MONITORING_DISABLED.toString(), false);
   }
 
+  /**
+   * Whether the P2P state transition message is enabled for this resource.
+   * By default it is disabled if not set.
+   *
+   * @return
+   */
+  public boolean isP2PMessageEnabled() {
+    return _record.getBooleanField(HelixConfigProperty.P2P_MESSAGE_ENABLED.name(), false);
+  }
 
   /**
    * Get the associated resource
@@ -458,6 +471,7 @@ public class ResourceConfig extends HelixProperty {
     private String _resourceType;
     private Boolean _groupRoutingEnabled;
     private Boolean _externalViewDisabled;
+    private Boolean _p2pMessageEnabled;
     private RebalanceConfig _rebalanceConfig;
     private StateTransitionTimeoutConfig _stateTransitionTimeoutConfig;
     private Map<String, List<String>> _preferenceLists;
@@ -472,6 +486,17 @@ public class ResourceConfig extends HelixProperty {
       return this;
     }
 
+    /**
+     * Enable/Disable the p2p state transition message for this resource.
+     * By default it is disabled if not set.
+     *
+     * @param enabled
+     */
+    public Builder setP2PMessageEnabled(boolean enabled) {
+      _p2pMessageEnabled = enabled;
+      return this;
+    }
+
     public Boolean isMonitorDisabled() {
       return _monitorDisabled;
     }
@@ -693,7 +718,7 @@ public class ResourceConfig extends HelixProperty {
           _stateModelFactoryName, _numReplica, _minActiveReplica, _maxPartitionsPerInstance,
           _instanceGroupTag, _helixEnabled, _resourceGroupName, _resourceType, _groupRoutingEnabled,
           _externalViewDisabled, _rebalanceConfig, _stateTransitionTimeoutConfig, _preferenceLists,
-          _mapFields);
+          _mapFields, _p2pMessageEnabled);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
index 01a3746..2fbbfcb 100644
--- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
+++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
@@ -95,8 +95,8 @@ public class StateModelDefinition extends HelixProperty {
         record.getListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString());
     _stateTransitionPriorityList =
         record.getListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString());
-    _stateTransitionTable = new HashMap<String, Map<String, String>>();
-    _statesCountMap = new HashMap<String, String>();
+    _stateTransitionTable = new HashMap<>();
+    _statesCountMap = new HashMap<>();
     if (_statesPriorityList != null) {
       int priority = 1;
       for (String state : _statesPriorityList) {
@@ -203,6 +203,22 @@ public class StateModelDefinition extends HelixProperty {
   }
 
   /**
+   * Whether this state model allows at most a single replica in the top-state?
+   *
+   * @return
+   */
+  public boolean isSingleTopStateModel() {
+    int topStateCount = 0;
+    try {
+      topStateCount = Integer.valueOf(_statesCountMap.get(getTopState()));
+    } catch (NumberFormatException ex) {
+
+    }
+
+    return topStateCount == 1;
+  }
+
+  /**
    * Get the second top states, which need one step transition to top state
    * @return a set of second top states
    */
@@ -244,9 +260,9 @@ public class StateModelDefinition extends HelixProperty {
      */
     public Builder(String name) {
       this._statemodelName = name;
-      statesMap = new HashMap<String, Integer>();
-      transitionMap = new HashMap<Transition, Integer>();
-      stateConstraintMap = new HashMap<String, String>();
+      statesMap = new HashMap<>();
+      transitionMap = new HashMap<>();
+      stateConstraintMap = new HashMap<>();
     }
 
     /**
@@ -368,7 +384,7 @@ public class StateModelDefinition extends HelixProperty {
         }
       };
       Collections.sort(transitionList, c2);
-      List<String> transitionPriorityList = new ArrayList<String>(transitionList.size());
+      List<String> transitionPriorityList = new ArrayList<>(transitionList.size());
       for (Transition t : transitionList) {
         transitionPriorityList.add(t.toString());
       }
@@ -383,7 +399,7 @@ public class StateModelDefinition extends HelixProperty {
       StateTransitionTableBuilder stateTransitionTableBuilder = new StateTransitionTableBuilder();
       Map<String, Map<String, String>> transitionTable =
           stateTransitionTableBuilder.buildTransitionTable(statePriorityList,
-              new ArrayList<Transition>(transitionMap.keySet()));
+              new ArrayList<>(transitionMap.keySet()));
       for (String state : transitionTable.keySet()) {
         record.setMapField(state + ".next", transitionTable.get(state));
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
index 1589c1a..2af4151 100644
--- a/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/FixedTargetTaskRebalancer.java
@@ -36,7 +36,8 @@ import org.apache.helix.model.ResourceAssignment;
 /**
  * This rebalancer is deprecated, left here only for back-compatible. *
  */
-@Deprecated public class FixedTargetTaskRebalancer extends DeprecatedTaskRebalancer {
+@Deprecated
+public class FixedTargetTaskRebalancer extends DeprecatedTaskRebalancer {
   private FixedTargetTaskAssignmentCalculator taskAssignmentCalculator =
       new FixedTargetTaskAssignmentCalculator();
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java b/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java
index 1aa82f4..e2d09ba 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/BaseStageTest.java
@@ -19,6 +19,7 @@ package org.apache.helix.controller.stages;
  * under the License.
  */
 
+import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.HashMap;
@@ -43,9 +44,13 @@ import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.tools.StateModelConfigGenerator;
+import org.codehaus.jackson.annotate.JsonAnySetter;
+import org.testng.ITestContext;
 import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 
@@ -73,7 +78,6 @@ public class BaseStageTest {
         + new Date(System.currentTimeMillis()));
   }
 
-  @BeforeMethod()
   public void setup() {
     _clusterName = "testCluster-" + UUID.randomUUID().toString();
     manager = new MockManager(_clusterName);
@@ -85,6 +89,21 @@ public class BaseStageTest {
     admin.addCluster(_clusterName);
   }
 
+  @BeforeMethod
+  public void beforeTest(Method testMethod, ITestContext testContext){
+    long startTime = System.currentTimeMillis();
+    System.out.println("START " + testMethod.getName() + " at " + new Date(startTime));
+    testContext.setAttribute("StartTime", System.currentTimeMillis());
+    setup();
+  }
+
+  @AfterMethod
+  public void endTest(Method testMethod, ITestContext testContext) {
+    Long startTime = (Long) testContext.getAttribute("StartTime");
+    long endTime = System.currentTimeMillis();
+    System.out.println("END " + testMethod.getName() + " at " + new Date(endTime) + ", took: " + (endTime - startTime) + "ms.");
+  }
+
   protected List<IdealState> setupIdealState(int nodes, String[] resources, int partitions,
       int replicas, RebalanceMode rebalanceMode, String stateModelName, String rebalanceClassName,
       String rebalanceStrategyName) {
@@ -137,14 +156,18 @@ public class BaseStageTest {
       stateModelName, rebalanceClassName, null);
   }
 
-  protected void setupLiveInstances(int numLiveInstances) {
+  protected List<String> setupLiveInstances(int numLiveInstances) {
+    List<String> instances = new ArrayList<>();
     for (int i = 0; i < numLiveInstances; i++) {
       LiveInstance liveInstance = new LiveInstance(HOSTNAME_PREFIX + i);
       liveInstance.setSessionId(SESSION_PREFIX + i);
 
       Builder keyBuilder = accessor.keyBuilder();
       accessor.setProperty(keyBuilder.liveInstance(HOSTNAME_PREFIX + i), liveInstance);
+      instances.add(liveInstance.getInstanceName());
     }
+
+    return instances;
   }
 
   protected void setupInstances(int numInstances) {
@@ -234,4 +257,20 @@ public class BaseStageTest {
 
     return resourceMap;
   }
+
+  protected Map<String, Resource> getResourceMap(String[] resources, int partitions,
+      String stateModel, ClusterConfig clusterConfig, ResourceConfig resourceConfig) {
+    Map<String, Resource> resourceMap = new HashMap<String, Resource>();
+
+    for (String r : resources) {
+      Resource testResource = new Resource(r, clusterConfig, resourceConfig);
+      testResource.setStateModelDefRef(stateModel);
+      for (int i = 0; i < partitions; i++) {
+        testResource.addPartition(r + "_" + i);
+      }
+      resourceMap.put(r, testResource);
+    }
+
+    return resourceMap;
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
index 994e2fa..4e20116 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
@@ -27,8 +27,8 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.controller.stages.MessageSelectionStage;
 import org.apache.helix.controller.stages.MessageSelectionStage.Bounds;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
@@ -86,7 +86,8 @@ public class TestMsgSelectionStage {
 
     List<Message> selectedMsg =
         new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingMessages,
-            messages, stateConstraints, stateTransitionPriorities, "OFFLINE");
+            messages, stateConstraints, stateTransitionPriorities,
+            BuiltInStateModelDefinitions.MasterSlave.getStateModelDefinition(), false);
 
     Assert.assertEquals(selectedMsg.size(), 1);
     Assert.assertEquals(selectedMsg.get(0).getMsgId(), "msgId_1");
@@ -123,7 +124,8 @@ public class TestMsgSelectionStage {
 
     List<Message> selectedMsg =
         new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingMessages,
-            messages, stateConstraints, stateTransitionPriorities, "OFFLINE");
+            messages, stateConstraints, stateTransitionPriorities,
+            BuiltInStateModelDefinitions.MasterSlave.getStateModelDefinition(), false);
 
     Assert.assertEquals(selectedMsg.size(), 0);
     System.out.println("END testMasterXferAfterMasterResume at "

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
index ae7f2f5..47de2d3 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
@@ -182,7 +182,6 @@ public class TestResourceComputationStage extends BaseStageTest {
 
   @Test
   public void testNull() {
-    ClusterEvent event = new ClusterEvent(ClusterEventType.Unknown);
     ResourceComputationStage stage = new ResourceComputationStage();
     StageContext context = new StageContext();
     stage.init(context);

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java
index 21878ac..85ecb0c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestZkReconnect.java
@@ -49,7 +49,7 @@ import org.testng.annotations.Test;
 public class TestZkReconnect {
   private static final Logger LOG = LoggerFactory.getLogger(TestZkReconnect.class);
 
-  @Test
+  @Test (enabled = false)
   public void testZKReconnect() throws Exception {
     final AtomicReference<ZkServer> zkServerRef = new AtomicReference<ZkServer>();
     final int zkPort = TestHelper.getRandomPort();

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
index ba2196c..4e49502 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/common/ZkIntegrationTestBase.java
@@ -19,6 +19,8 @@ package org.apache.helix.integration.common;
  * under the License.
  */
 
+import java.lang.reflect.Method;
+import java.util.Date;
 import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
@@ -54,7 +56,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
 import org.testng.AssertJUnit;
+import org.testng.ITestContext;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.AfterSuite;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.BeforeSuite;
 
 public class ZkIntegrationTestBase {
@@ -95,6 +100,21 @@ public class ZkIntegrationTestBase {
     TestHelper.stopZkServer(_zkServer);
   }
 
+  @BeforeMethod
+  public void beforeTest(Method testMethod, ITestContext testContext){
+    long startTime = System.currentTimeMillis();
+    System.out.println("START " + testMethod.getName() + " at " + new Date(startTime));
+    testContext.setAttribute("StartTime", System.currentTimeMillis());
+  }
+
+  @AfterMethod
+  public void endTest(Method testMethod, ITestContext testContext) {
+    Long startTime = (Long) testContext.getAttribute("StartTime");
+    long endTime = System.currentTimeMillis();
+    System.out.println(
+        "END " + testMethod.getName() + " at " + new Date(endTime) + ", took: " + (endTime - startTime) + "ms.");
+  }
+
   protected String getShortClassName() {
     return this.getClass().getSimpleName();
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
index ec2dd1b..1448453 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestZkCallbackHandlerLeak.java
@@ -93,7 +93,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
     Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers.");
@@ -119,7 +119,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
     // printHandlers(participantManagerToExpire);
     int controllerHandlerNb = controller.getHandlers().size();
     int particHandlerNb = participantManagerToExpire.getHandlers().size();
-    Assert.assertEquals(controllerHandlerNb, (6 + 2 * n),
+    Assert.assertEquals(controllerHandlerNb, (7 + 2 * n),
         "HelixController should have 9 (5+2n) callback handlers for 2 (n) participant");
     Assert.assertEquals(particHandlerNb, 1,
         "HelixParticipant should have 1 (msg->HelixTaskExecutor) callback handlers");
@@ -149,7 +149,7 @@ public class TestZkCallbackHandlerLeak extends ZkUnitTestBase {
 
         // controller should have 5 + 2n + m + (m+2)n zk-watchers
         // where n is number of nodes and m is number of resources
-        return watchPaths.size() == (7 + 5 * n);
+        return watchPaths.size() == (8 + 5 * n);
       }
     }, 500);
     Assert.assertTrue(result, "Controller should have 8 + 5*n zk-watchers after session expiry.");

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
new file mode 100644
index 0000000..e0f83e1
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/messaging/TestP2PMessageSemiAuto.java
@@ -0,0 +1,240 @@
+package org.apache.helix.integration.messaging;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.config.HelixConfigProperty;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.mock.participant.MockTransition;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MasterSlaveSMD;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestP2PMessageSemiAuto extends ZkIntegrationTestBase {
+  final String CLASS_NAME = getShortClassName();
+  final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+
+  static final int PARTICIPANT_NUMBER = 3;
+  static final int PARTICIPANT_START_PORT = 12918;
+
+  static final String DB_NAME_1 = "TestDB_1";
+  static final String DB_NAME_2 = "TestDB_2";
+
+  static final int PARTITION_NUMBER = 20;
+  static final int REPLICA_NUMBER = 3;
+
+  List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  List<String> _instances = new ArrayList<>();
+  ClusterControllerManager _controller;
+
+  HelixClusterVerifier _clusterVerifier;
+  ConfigAccessor _configAccessor;
+  HelixDataAccessor _accessor;
+
+  @BeforeClass
+  public void beforeClass()
+      throws InterruptedException {
+    System.out.println(
+        "START " + getShortClassName() + " at " + new Date(System.currentTimeMillis()));
+
+    // setup storage cluster
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < PARTICIPANT_NUMBER; i++) {
+      String instance = PARTICIPANT_PREFIX + "_" + (PARTICIPANT_START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instance);
+      _instances.add(instance);
+    }
+
+    // start dummy participants
+    for (int i = 0; i < PARTICIPANT_NUMBER; i++) {
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _instances.get(i));
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    createDBInSemiAuto(DB_NAME_1, _instances);
+    createDBInSemiAuto(DB_NAME_2, _instances);
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    _clusterVerifier = new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkClient(_gZkClient).build();
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    _configAccessor = new ConfigAccessor(_gZkClient);
+    _accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+  }
+
+  private void createDBInSemiAuto(String dbName, List<String> preferenceList) {
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, dbName, PARTITION_NUMBER,
+        BuiltInStateModelDefinitions.MasterSlave.name(), IdealState.RebalanceMode.SEMI_AUTO.toString());
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, REPLICA_NUMBER);
+
+    IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+    for (String p : is.getPartitionSet()) {
+      is.setPreferenceList(p, preferenceList);
+    }
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
+  }
+
+  @Test
+  public void testP2PStateTransitionDisabled() {
+    // disable the master instance
+    String prevMasterInstance = _instances.get(0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, false);
+
+    Assert.assertTrue(_clusterVerifier.verify());
+    verifyP2PMessage(DB_NAME_1,_instances.get(1), MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+    verifyP2PMessage(DB_NAME_2,_instances.get(1), MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+
+
+    //re-enable the old master
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, true);
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    verifyP2PMessage(DB_NAME_1, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+    verifyP2PMessage(DB_NAME_2, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+  }
+
+  @Test (dependsOnMethods = {"testP2PStateTransitionDisabled"})
+  public void testP2PStateTransitionEnabledInCluster() {
+    enableP2PInCluster(true);
+    enableP2PInResource(DB_NAME_1,false);
+    enableP2PInResource(DB_NAME_2,false);
+
+    // disable the master instance
+    String prevMasterInstance = _instances.get(0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, false);
+
+    Assert.assertTrue(_clusterVerifier.verify());
+    verifyP2PMessage(DB_NAME_1, _instances.get(1), MasterSlaveSMD.States.MASTER.name(), prevMasterInstance);
+    verifyP2PMessage(DB_NAME_2, _instances.get(1), MasterSlaveSMD.States.MASTER.name(), prevMasterInstance);
+
+    //re-enable the old master
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, true);
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    verifyP2PMessage(DB_NAME_1, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _instances.get(1));
+    verifyP2PMessage(DB_NAME_2, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _instances.get(1));
+  }
+
+  @Test (dependsOnMethods = {"testP2PStateTransitionDisabled"})
+  public void testP2PStateTransitionEnabledInResource() {
+    enableP2PInCluster(false);
+    enableP2PInResource(DB_NAME_1,true);
+    enableP2PInResource(DB_NAME_2,false);
+
+
+    // disable the master instance
+    String prevMasterInstance = _instances.get(0);
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, false);
+
+    Assert.assertTrue(_clusterVerifier.verify());
+    verifyP2PMessage(DB_NAME_1, _instances.get(1), MasterSlaveSMD.States.MASTER.name(), prevMasterInstance);
+    verifyP2PMessage(DB_NAME_2, _instances.get(1), MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+
+
+    //re-enable the old master
+    _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME, prevMasterInstance, true);
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    verifyP2PMessage(DB_NAME_1, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _instances.get(1));
+    verifyP2PMessage(DB_NAME_2, prevMasterInstance, MasterSlaveSMD.States.MASTER.name(), _controller.getInstanceName());
+  }
+
+  private void enableP2PInCluster(boolean enable) {
+    // enable p2p message in cluster.
+    if (enable) {
+      ClusterConfig clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
+      clusterConfig.enableP2PMessage(true);
+      _configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+    } else {
+      ClusterConfig clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
+      clusterConfig.getRecord().getSimpleFields().remove(HelixConfigProperty.P2P_MESSAGE_ENABLED.name());
+      _configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+    }
+  }
+
+  private void enableP2PInResource(String dbName, boolean enable) {
+    if (enable) {
+      ResourceConfig resourceConfig = new ResourceConfig.Builder(dbName).setP2PMessageEnabled(true).build();
+      _configAccessor.setResourceConfig(CLUSTER_NAME, dbName, resourceConfig);
+    } else {
+      // remove P2P Message in resource config
+      ResourceConfig resourceConfig = _configAccessor.getResourceConfig(CLUSTER_NAME, dbName);
+      if (resourceConfig != null) {
+        resourceConfig.getRecord().getSimpleFields().remove(HelixConfigProperty.P2P_MESSAGE_ENABLED.name());
+        _configAccessor.setResourceConfig(CLUSTER_NAME, dbName, resourceConfig);
+      }
+    }
+  }
+
+  private void verifyP2PMessage(String dbName, String instance, String expectedState, String expectedTriggerHost) {
+    ClusterDataCache dataCache = new ClusterDataCache(CLUSTER_NAME);
+    dataCache.refresh(_accessor);
+
+    Map<String, LiveInstance> liveInstanceMap = dataCache.getLiveInstances();
+    LiveInstance liveInstance = liveInstanceMap.get(instance);
+
+    Map<String, CurrentState> currentStateMap = dataCache.getCurrentState(instance, liveInstance.getSessionId());
+    Assert.assertNotNull(currentStateMap);
+    CurrentState currentState = currentStateMap.get(dbName);
+    Assert.assertNotNull(currentState);
+    Assert.assertEquals(currentState.getPartitionStateMap().size(), PARTITION_NUMBER);
+
+    for (String partition : currentState.getPartitionStateMap().keySet()) {
+      String state = currentState.getState(partition);
+      Assert.assertEquals(state, expectedState,
+          dbName + " Partition " + partition + "'s state is different as expected!");
+      String triggerHost = currentState.getTriggerHost(partition);
+      Assert.assertEquals(triggerHost, expectedTriggerHost,
+          "Partition " + partition + "'s transition to Master was not triggered by expected host!");
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNodeOfflineTimeStamp.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNodeOfflineTimeStamp.java b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNodeOfflineTimeStamp.java
index fb1c090..ceda6e5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNodeOfflineTimeStamp.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/paticipant/TestNodeOfflineTimeStamp.java
@@ -45,8 +45,10 @@ public class TestNodeOfflineTimeStamp extends ZkStandAloneCMTestBase {
     Assert.assertTrue(Math.abs(shutdownTime - recordTime) <= 500L);
 
     _participants[0].reset();
+    Thread.sleep(50);
     _participants[0].syncStart();
 
+    Thread.sleep(50);
     history = getInstanceHistory(_participants[0].getInstanceName());
     Assert.assertEquals(history.getLastOfflineTime(), ParticipantHistory.ONLINE);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
index 967175f..d886e44 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/CrushRebalancers/TestCrushAutoRebalance.java
@@ -109,17 +109,16 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
   }
 
   @DataProvider(name = "rebalanceStrategies")
-  public static String [][] rebalanceStrategies() {
+  public static Object [][] rebalanceStrategies() {
     return new String[][] { {"CrushRebalanceStrategy", CrushRebalanceStrategy.class.getName()},
         {"MultiRoundCrushRebalanceStrategy", MultiRoundCrushRebalanceStrategy.class.getName()}
     };
   }
 
-  @Test(dataProvider = "rebalanceStrategies", enabled=true)
+  @Test(dataProvider = "rebalanceStrategies")
   public void testZoneIsolation(String rebalanceStrategyName, String rebalanceStrategyClass)
       throws Exception {
     System.out.println("testZoneIsolation " + rebalanceStrategyName);
-
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + rebalanceStrategyName + "-" + i++;
@@ -143,7 +142,7 @@ public class TestCrushAutoRebalance extends ZkIntegrationTestBase {
     }
   }
 
-  @Test(dataProvider = "rebalanceStrategies", enabled=true)
+  @Test(dataProvider = "rebalanceStrategies")
   public void testZoneIsolationWithInstanceTag(
       String rebalanceStrategyName, String rebalanceStrategyClass) throws Exception {
     Set<String> tags = new HashSet<String>(_nodeToTagMap.values());

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKUtil.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKUtil.java
index 7b7ec2c..a2d3f1d 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKUtil.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZKUtil.java
@@ -22,17 +22,13 @@ package org.apache.helix.manager.zk;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
-
 import java.util.Map;
 import org.apache.helix.PropertyPathBuilder;
-import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.AssertJUnit;

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
index fcca597..6e68d1a 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
@@ -28,10 +28,8 @@ import java.util.Map;
 
 import org.apache.helix.AccessOption;
 import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
@@ -44,7 +42,6 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.ZkUnitTestBase;
 import org.apache.helix.manager.MockListener;
-import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.zookeeper.data.Stat;


[15/50] [abbrv] helix git commit: HELIX-614: add gauge for failed workflow

Posted by jx...@apache.org.
HELIX-614: add gauge for failed workflow


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

Branch: refs/heads/master
Commit: 62752b2cfc7841ca234ae5071964b65efe38569d
Parents: 79c96ba
Author: hrzhang <hr...@linkedin.com>
Authored: Mon Nov 13 13:39:32 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:26 2018 -0800

----------------------------------------------------------------------
 .../monitoring/mbeans/WorkflowMonitor.java      |  10 ++
 .../monitoring/mbeans/WorkflowMonitorMBean.java |   5 +
 .../helix/monitoring/TestWorkflowMonitor.java   | 148 +++++++++++++++++++
 3 files changed, 163 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/62752b2c/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
index 5789677..00f75d4 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
@@ -29,6 +29,7 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
 
   private long _successfulWorkflowCount;
   private long _failedWorkflowCount;
+  private long _failedWorkflowGauge;
   private long _existingWorkflowGauge;
   private long _queuedWorkflowGauge;
   private long _runningWorkflowGauge;
@@ -39,6 +40,7 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
     _workflowType = workflowType;
     _successfulWorkflowCount = 0L;
     _failedWorkflowCount = 0L;
+    _failedWorkflowGauge = 0L;
     _existingWorkflowGauge = 0L;
     _queuedWorkflowGauge = 0L;
     _runningWorkflowGauge = 0L;
@@ -55,6 +57,11 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
   }
 
   @Override
+  public long getFailedWorkflowGauge() {
+    return _failedWorkflowGauge;
+  }
+
+  @Override
   public long getExistingWorkflowGauge() {
     return _existingWorkflowGauge;
   }
@@ -93,6 +100,7 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
    * Reset gauges
    */
   public void resetGauges() {
+    _failedWorkflowGauge = 0L;
     _existingWorkflowGauge = 0L;
     _runningWorkflowGauge = 0L;
     _queuedWorkflowGauge = 0L;
@@ -107,6 +115,8 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
       _queuedWorkflowGauge++;
     } else if (current.equals(TaskState.IN_PROGRESS)) {
       _runningWorkflowGauge++;
+    } else if (current.equals(TaskState.FAILED)) {
+      _failedWorkflowGauge++;
     }
     _existingWorkflowGauge++;
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/62752b2c/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
index 8d7076c..dcd633d 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
@@ -39,6 +39,11 @@ public interface WorkflowMonitorMBean extends SensorNameProvider {
   public long getFailedWorkflowCount();
 
   /**
+   * Get number of current failed workflows
+   */
+  public long getFailedWorkflowGauge();
+
+  /**
    * Get number of current existing workflows
    * @return
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/62752b2c/helix-core/src/test/java/org/apache/helix/monitoring/TestWorkflowMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestWorkflowMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestWorkflowMonitor.java
new file mode 100644
index 0000000..978e574
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestWorkflowMonitor.java
@@ -0,0 +1,148 @@
+package org.apache.helix.monitoring;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.lang.management.ManagementFactory;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectInstance;
+import javax.management.ObjectName;
+import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
+import org.apache.helix.monitoring.mbeans.WorkflowMonitor;
+import org.apache.helix.task.TaskState;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestWorkflowMonitor {
+  private static final String TEST_CLUSTER_NAME = "TestCluster";
+  private static final String TEST_WORKFLOW_TYPE = "WorkflowTestType";
+  private static final String TEST_WORKFLOW_MBEAN_NAME = String
+      .format("%s=%s, %s=%s", "cluster", TEST_CLUSTER_NAME, "workflowType", TEST_WORKFLOW_TYPE);
+  private static final MBeanServer beanServer = ManagementFactory.getPlatformMBeanServer();
+
+  @Test
+  public void testRun() throws Exception {
+    WorkflowMonitor wm = new WorkflowMonitor(TEST_CLUSTER_NAME, TEST_WORKFLOW_TYPE);
+    registerMbean(wm, getObjectName());
+    Set<ObjectInstance> existingInstances = beanServer.queryMBeans(
+        new ObjectName(MonitorDomainNames.ClusterStatus.name() + ":" + TEST_WORKFLOW_MBEAN_NAME),
+        null);
+    HashSet<String> expectedAttr = new HashSet<>(Arrays
+        .asList("SuccessfulWorkflowCount", "FailedWorkflowCount", "FailedWorkflowGauge",
+            "ExistingWorkflowGauge", "QueuedWorkflowGauge", "RunningWorkflowGauge"));
+    for (ObjectInstance i : existingInstances) {
+      for (MBeanAttributeInfo info : beanServer.getMBeanInfo(i.getObjectName()).getAttributes()) {
+        expectedAttr.remove(info.getName());
+      }
+    }
+    Assert.assertTrue(expectedAttr.isEmpty());
+
+    int successfulWfCnt = 10;
+    int failedWfCnt = 10;
+    int queuedWfCnt = 10;
+    int runningWfCnt = 10;
+
+    for (int i = 0; i < successfulWfCnt; i++) {
+      wm.updateWorkflowCounters(TaskState.COMPLETED);
+      wm.updateWorkflowGauges(TaskState.COMPLETED);
+    }
+
+    for (int i = 0; i < failedWfCnt; i++) {
+      wm.updateWorkflowCounters(TaskState.FAILED);
+      wm.updateWorkflowGauges(TaskState.FAILED);
+    }
+
+    for (int i = 0; i < queuedWfCnt; i++) {
+      wm.updateWorkflowGauges(TaskState.NOT_STARTED);
+    }
+
+    for (int i = 0; i < runningWfCnt; i++) {
+      wm.updateWorkflowGauges(TaskState.IN_PROGRESS);
+    }
+
+    // Test gauges
+    Assert.assertEquals(wm.getExistingWorkflowGauge(),
+        successfulWfCnt + failedWfCnt + queuedWfCnt + runningWfCnt);
+    Assert.assertEquals(wm.getFailedWorkflowGauge(), failedWfCnt);
+    Assert.assertEquals(wm.getQueuedWorkflowGauge(), queuedWfCnt);
+    Assert.assertEquals(wm.getRunningWorkflowGauge(), runningWfCnt);
+
+    // Test counts
+    Assert.assertEquals(wm.getFailedWorkflowCount(), failedWfCnt);
+    Assert.assertEquals(wm.getSuccessfulWorkflowCount(), successfulWfCnt);
+
+    wm.resetGauges();
+
+    for (int i = 0; i < successfulWfCnt; i++) {
+      wm.updateWorkflowCounters(TaskState.COMPLETED);
+      wm.updateWorkflowGauges(TaskState.COMPLETED);
+    }
+
+    for (int i = 0; i < failedWfCnt; i++) {
+      wm.updateWorkflowCounters(TaskState.FAILED);
+      wm.updateWorkflowGauges(TaskState.FAILED);
+    }
+
+    for (int i = 0; i < queuedWfCnt; i++) {
+      wm.updateWorkflowGauges(TaskState.NOT_STARTED);
+    }
+
+    for (int i = 0; i < runningWfCnt; i++) {
+      wm.updateWorkflowGauges(TaskState.IN_PROGRESS);
+    }
+
+    // After reset, counters should be accumulative, but gauges should be reset
+    Assert.assertEquals(wm.getExistingWorkflowGauge(),
+        successfulWfCnt + failedWfCnt + queuedWfCnt + runningWfCnt);
+    Assert.assertEquals(wm.getFailedWorkflowGauge(), failedWfCnt);
+    Assert.assertEquals(wm.getQueuedWorkflowGauge(), queuedWfCnt);
+    Assert.assertEquals(wm.getRunningWorkflowGauge(), runningWfCnt);
+    Assert.assertEquals(wm.getFailedWorkflowCount(), failedWfCnt * 2);
+    Assert.assertEquals(wm.getSuccessfulWorkflowCount(), successfulWfCnt * 2);
+
+  }
+
+  private ObjectName getObjectName() throws MalformedObjectNameException {
+    return new ObjectName(
+        String.format("%s:%s", MonitorDomainNames.ClusterStatus.name(), TEST_WORKFLOW_MBEAN_NAME));
+  }
+
+  private void registerMbean(Object bean, ObjectName name) {
+    try {
+      if (beanServer.isRegistered(name)) {
+        beanServer.unregisterMBean(name);
+      }
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      System.out.println("Register MBean: " + name);
+      beanServer.registerMBean(bean, name);
+    } catch (Exception e) {
+      System.out.println("Could not register MBean: " + name + e.toString());
+    }
+  }
+
+}


[26/50] [abbrv] helix git commit: [helix-front] Added Back button on top of cluster list panel

Posted by jx...@apache.org.
[helix-front] Added Back button on top of cluster list panel


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

Branch: refs/heads/master
Commit: 4d630e0a6b3953d26594abbd09bef314cfb49683
Parents: ae58e79
Author: Vivo Xu <vx...@linkedin.com>
Authored: Thu Dec 7 17:26:48 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:10 2018 -0800

----------------------------------------------------------------------
 helix-front/client/app/app.component.scss                     | 1 +
 .../app/cluster/cluster-list/cluster-list.component.html      | 7 +++++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4d630e0a/helix-front/client/app/app.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/app.component.scss b/helix-front/client/app/app.component.scss
index 5f2d7d5..816aac0 100644
--- a/helix-front/client/app/app.component.scss
+++ b/helix-front/client/app/app.component.scss
@@ -25,6 +25,7 @@
 
 .main-container {
   height: calc(100vh - 64px - 50px);
+  overflow-y: scroll;
 
   &.no-header {
     height: 100vh;

http://git-wip-us.apache.org/repos/asf/helix/blob/4d630e0a/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/cluster/cluster-list/cluster-list.component.html b/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
index 480bd51..7df5531 100644
--- a/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
+++ b/helix-front/client/app/cluster/cluster-list/cluster-list.component.html
@@ -5,10 +5,13 @@
     </md-spinner>
   </section>
   <md-nav-list *ngIf="!isLoading && !errorMessage">
+    <button md-button routerLink="/">
+      <md-icon>arrow_back</md-icon> Back to Index
+    </button>
     <button md-mini-fab *ngIf="can" (click)="createCluster()">
       <md-icon>add</md-icon>
     </button>
-    <h3 md-subheader>Clusters</h3>
+    <h3 md-subheader>Clusters ({{ clusters.length }})</h3>
     <a *ngFor="let cluster of clusters"
       md-list-item
       [routerLink]="[cluster.name]"
@@ -18,7 +21,7 @@
     </a>
     <div *ngIf="clusters.length == 0" class="empty">
       There's no cluster here.
-      <a md-button (click)="createCluster()">Create one?</a>
+      <a md-button *ngIf="can" (click)="createCluster()">Create one?</a>
     </div>
   </md-nav-list>
   <section class="error-message" *ngIf="errorMessage">


[07/50] [abbrv] helix git commit: Temporary disable logging rebalance error before HELIX-631 is resolved.

Posted by jx...@apache.org.
Temporary disable logging rebalance error before HELIX-631 is resolved.

An issue is found that legacy code assumes all controllers' instance names start with "controller". However, this assumption is no longer valid now.
This results in log to be written to a wrong path.

Before we resolve the problem cleanly, disable the error log.


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

Branch: refs/heads/master
Commit: 401ada6391883f5cfcd7e04858dc94ee2b227ff7
Parents: d0a3c0d
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Tue Nov 7 16:12:42 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:33 2018 -0800

----------------------------------------------------------------------
 .../helix/controller/stages/BestPossibleStateCalcStage.java    | 3 +++
 .../helix/integration/TestAlertingRebalancerFailure.java       | 6 +++---
 .../TestPauseClusterWhenReachingOfflineInstancesLimit.java     | 3 +++
 3 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/401ada63/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 20ebb8e..c43b96c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -148,11 +148,14 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       @Override
       public Object call() {
         try {
+          // TODO re-enable logging error after ticket HELIX-631 is resolved
+          /*
           if (hasFailure && _statusUpdateUtil != null) {
             _statusUpdateUtil
                 .logError(StatusUpdateUtil.ErrorType.RebalanceResourceFailure, this.getClass(),
                     errorMessage, helixManager);
           }
+          */
           if (clusterStatusMonitor != null) {
             clusterStatusMonitor.setRebalanceFailureGauge(hasFailure);
           }

http://git-wip-us.apache.org/repos/asf/helix/blob/401ada63/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java b/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
index 0df817d..012760c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
@@ -106,7 +106,7 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     accessor.removeProperty(errorNodeKey);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testParticipantUnavailable() {
     _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
@@ -145,7 +145,7 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     }
   }
 
-  @Test
+  @Test (enabled = false)
   public void testTagSetIncorrect() {
     _setupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
         BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
@@ -164,7 +164,7 @@ public class TestAlertingRebalancerFailure extends ZkStandAloneCMTestBase {
     _setupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, testDb);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testWithDomainId() throws InterruptedException {
     int replicas = 2;
     ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);

http://git-wip-us.apache.org/repos/asf/helix/blob/401ada63/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
index 02668e0..9c2c434 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
@@ -167,6 +167,8 @@ public class TestPauseClusterWhenReachingOfflineInstancesLimit extends ZkIntegra
     Assert.assertNotNull(pauseSignal);
     Assert.assertNotNull(pauseSignal.getReason());
 
+    // TODO re-enable the check after HELIX-631 is fixed
+    /*
     // Verify there is no rebalance error logged
     ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
     PropertyKey errorNodeKey =
@@ -177,6 +179,7 @@ public class TestPauseClusterWhenReachingOfflineInstancesLimit extends ZkIntegra
         (Long) _server.getAttribute(getMbeanName(CLUSTER_NAME), "RebalanceFailureGauge");
     Assert.assertNotNull(value);
     Assert.assertTrue(value.longValue() > 0);
+    */
   }
 
   @AfterClass


[27/50] [abbrv] helix git commit: HELIX-674: change slf4j-log4j12 to use jar instead of ivy in IVY file

Posted by jx...@apache.org.
HELIX-674: change slf4j-log4j12 to use jar instead of ivy in IVY file


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

Branch: refs/heads/master
Commit: d9696cc16af8bdfc042c488fe3c28cb3b31413eb
Parents: 4d630e0
Author: hrzhang <hr...@linkedin.com>
Authored: Mon Dec 11 15:02:33 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:13 2018 -0800

----------------------------------------------------------------------
 helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy | 2 +-
 helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy               | 2 +-
 helix-core/helix-core-0.6.10-SNAPSHOT.ivy                 | 2 +-
 helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy                 | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d9696cc1/helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy b/helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy
index 6232300..36674ab 100644
--- a/helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy
+++ b/helix-admin-webapp/helix-admin-webapp-0.6.10-SNAPSHOT.ivy
@@ -41,7 +41,7 @@ under the License.
         <artifact name="slf4j-api" ext="jar"/>
     </dependency>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
-        <artifact name="slf4j-log4j12" ext="ivy"/>
+        <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
 		<dependency org="org.apache.helix" name="helix-core" rev="0.6.10-SNAPSHOT" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.restlet.jse" name="org.restlet" rev="2.2.1" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9696cc1/helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy b/helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy
index 0b75303..2f4aae4 100644
--- a/helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy
+++ b/helix-agent/helix-agent-0.6.10-SNAPSHOT.ivy
@@ -89,7 +89,7 @@ under the License.
         <artifact name="slf4j-api" ext="jar"/>
     </dependency>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
-        <artifact name="slf4j-log4j12" ext="ivy"/>
+        <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
 		<override org="org.mockito" module="mockito-all" matcher="exact" rev="1.9.5"/>
 		<override org="org.apache.helix" module="helix-core" matcher="exact" rev="0.6.10-SNAPSHOT"/>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9696cc1/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-core/helix-core-0.6.10-SNAPSHOT.ivy b/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
index 1e22533..e86bc7e 100644
--- a/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
+++ b/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
@@ -50,7 +50,7 @@ under the License.
         <artifact name="slf4j-api" ext="jar"/>
     </dependency>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
-        <artifact name="slf4j-log4j12" ext="ivy"/>
+        <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
     <dependency org="org.apache.zookeeper" name="zookeeper" rev="3.4.9" conf="compile->compile(default);runtime->runtime(default);default->default"/>
     <dependency org="org.codehaus.jackson" name="jackson-core-asl" rev="1.8.5" conf="compile->compile(default);runtime->runtime(default);default->default"/>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9696cc1/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
index ec53987..5508152 100644
--- a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
+++ b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
@@ -41,7 +41,7 @@ under the License.
         <artifact name="slf4j-api" ext="jar"/>
     </dependency>
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
-        <artifact name="slf4j-log4j12" ext="ivy"/>
+        <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
 		<dependency org="org.apache.helix" name="helix-core" rev="0.6.10-SNAPSHOT" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.codehaus.jackson" name="jackson-core-asl" rev="1.8.5" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>


[35/50] [abbrv] helix git commit: HELIX-661: implement GET namespace(s)

Posted by jx...@apache.org.
HELIX-661: implement GET namespace(s)


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

Branch: refs/heads/master
Commit: 40710b2713ea0e4f1d4a936396c98ef01f8e2b68
Parents: 4ff98fb
Author: hrzhang <hr...@linkedin.com>
Authored: Wed Dec 13 18:57:40 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:39 2018 -0800

----------------------------------------------------------------------
 .../helix/rest/common/ContextPropertyKeys.java  |   3 +-
 .../helix/rest/common/HelixRestNamespace.java   |  17 +-
 .../helix/rest/common/HelixRestUtils.java       |  15 +-
 .../apache/helix/rest/common/ServletType.java   |  56 ++
 .../helix/rest/server/HelixRestServer.java      |  78 +--
 .../rest/server/resources/AbstractResource.java |  40 --
 .../rest/server/resources/ClusterAccessor.java  | 400 --------------
 .../rest/server/resources/InstanceAccessor.java | 545 -------------------
 .../rest/server/resources/JobAccessor.java      | 200 -------
 .../rest/server/resources/ResourceAccessor.java | 278 ----------
 .../server/resources/UIResourceAccessor.java    |  62 ---
 .../rest/server/resources/WorkflowAccessor.java | 325 -----------
 .../resources/helix/AbstractHelixResource.java  |  79 +++
 .../server/resources/helix/ClusterAccessor.java | 400 ++++++++++++++
 .../resources/helix/InstanceAccessor.java       | 545 +++++++++++++++++++
 .../server/resources/helix/JobAccessor.java     | 200 +++++++
 .../resources/helix/MetadataAccessor.java       |  45 ++
 .../resources/helix/ResourceAccessor.java       | 278 ++++++++++
 .../resources/helix/WorkflowAccessor.java       | 325 +++++++++++
 .../resources/metadata/NamespacesAccessor.java  |  47 ++
 .../helix/rest/server/TestClusterAccessor.java  |   2 +-
 .../helix/rest/server/TestHelixRestServer.java  |   5 +-
 .../helix/rest/server/TestInstanceAccessor.java |   4 +-
 .../helix/rest/server/TestJobAccessor.java      |   4 +-
 .../rest/server/TestNamespacedAPIAccess.java    |  55 ++
 .../helix/rest/server/TestResourceAccessor.java |   2 +-
 .../helix/rest/server/TestWorkflowAccessor.java |   2 +-
 27 files changed, 2099 insertions(+), 1913 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
index ce59abc..ffe1283 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
@@ -21,5 +21,6 @@ package org.apache.helix.rest.common;
 
 public enum ContextPropertyKeys {
   SERVER_CONTEXT,
-  NAMESPACE
+  METADATA,
+  ALL_NAMESPACES
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
index 5d1c8f3..a2fb52c 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
@@ -19,10 +19,15 @@ package org.apache.helix.rest.common;
  * under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
+
+
 public class HelixRestNamespace {
 
   public enum HelixMetadataStoreType {
-    ZOOKEEPER
+    ZOOKEEPER,
+    NO_METADATA_STORE
   }
 
   public enum HelixRestNamespaceProperty {
@@ -78,7 +83,8 @@ public class HelixRestNamespace {
     if (_name == null || _name.length() == 0) {
       throw new IllegalArgumentException("Name of namespace not provided");
     }
-    if (_metadataStoreAddress == null || _metadataStoreAddress.isEmpty()) {
+    if (_metadataStoreType != HelixMetadataStoreType.NO_METADATA_STORE && (_metadataStoreAddress == null
+        || _metadataStoreAddress.isEmpty())) {
       throw new IllegalArgumentException(
           String.format("Metadata store address \"%s\" is not valid for namespace %s", _metadataStoreAddress, _name));
     }
@@ -96,4 +102,11 @@ public class HelixRestNamespace {
     return _metadataStoreAddress;
   }
 
+  public Map<String, String> getRestInfo() {
+    // In REST APIs we currently don't expose metadata store information
+    Map<String, String> ret = new HashMap<>();
+    ret.put(HelixRestNamespaceProperty.NAME.name(), _name);
+    ret.put(HelixRestNamespaceProperty.IS_DEFAULT.name(), String.valueOf(_isDefault));
+    return ret;
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
index 6c4a3df..39491ab 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
@@ -21,17 +21,6 @@ package org.apache.helix.rest.common;
 
 public class HelixRestUtils {
   /**
-   * Generate servlet path spec for a given namespace.
-   * @param namespace Name of the namespace
-   * @param isDefaultServlet mark this as true to get path spec for the special servlet for default namespace
-   * @return servlet path spec
-   */
-  public static String makeServletPathSpec(String namespace, boolean isDefaultServlet) {
-    return isDefaultServlet ? HelixRestNamespace.DEFAULT_NAMESPACE_PATH_SPEC
-        : String.format("/namespaces/%s/*", namespace);
-  }
-
-  /**
    * Extract namespace information from servlet path. There are 3 cases:
    *  1. /namespaces/namespaceName  ->  return namespaceName
    *  2. /namespaces                ->  return ""
@@ -40,7 +29,7 @@ public class HelixRestUtils {
    * @return Namespace name retrieved from servlet spec.
    */
   public static String getNamespaceFromServletPath(String servletPath) {
-    if (isDefaultNamespaceServlet(servletPath)) {
+    if (isDefaultServlet(servletPath)) {
       return HelixRestNamespace.DEFAULT_NAMESPACE_NAME;
     }
 
@@ -52,7 +41,7 @@ public class HelixRestUtils {
     }
   }
 
-  private static boolean isDefaultNamespaceServlet(String servletPath) {
+  public static boolean isDefaultServlet(String servletPath) {
     // Special servlet for default namespace has path spec "/*", so servletPath is empty
     return servletPath == null || servletPath.isEmpty();
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/common/ServletType.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/ServletType.java b/helix-rest/src/main/java/org/apache/helix/rest/common/ServletType.java
new file mode 100644
index 0000000..bbff2d6
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/ServletType.java
@@ -0,0 +1,56 @@
+package org.apache.helix.rest.common;
+
+/*
+ * 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.rest.server.resources.helix.AbstractHelixResource;
+import org.apache.helix.rest.server.resources.metadata.NamespacesAccessor;
+
+public enum ServletType {
+  /**
+   * Servlet serving default API endpoints (/admin/v2/clusters/...)
+   */
+  DEFAULT_SERVLET(HelixRestNamespace.DEFAULT_NAMESPACE_PATH_SPEC,
+      new String[] { AbstractHelixResource.class.getPackage().getName(),
+          NamespacesAccessor.class.getPackage().getName()
+      }),
+
+  /**
+   * Servlet serving namespaced API endpoints (/admin/v2/namespaces/{namespaceName})
+   */
+  COMMON_SERVLET("/namespaces/%s/*",
+      new String[] { AbstractHelixResource.class.getPackage().getName(),
+      });
+
+  private final String _servletPathSpecTemplate;
+  private final String[] _servletPackageArray;
+
+  ServletType(String servletPathSpecTemplate, String[] servletPackageArray) {
+    _servletPathSpecTemplate = servletPathSpecTemplate;
+    _servletPackageArray = servletPackageArray;
+  }
+
+  public String getServletPathSpecTemplate() {
+    return _servletPathSpecTemplate;
+  }
+
+  public String[] getServletPackageArray() {
+    return _servletPackageArray;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
index 3737308..e0c1c4e 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
@@ -27,11 +27,10 @@ import java.util.Map;
 import org.apache.helix.HelixException;
 import org.apache.helix.rest.common.ContextPropertyKeys;
 import org.apache.helix.rest.common.HelixRestNamespace;
-import org.apache.helix.rest.common.HelixRestUtils;
+import org.apache.helix.rest.common.ServletType;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.filters.AuditLogFilter;
 import org.apache.helix.rest.server.filters.CORSFilter;
-import org.apache.helix.rest.server.resources.AbstractResource;
 import org.eclipse.jetty.http.HttpVersion;
 import org.eclipse.jetty.server.HttpConfiguration;
 import org.eclipse.jetty.server.HttpConnectionFactory;
@@ -53,18 +52,13 @@ public class HelixRestServer {
   private int _port;
   private String _urlPrefix;
   private Server _server;
+  private List<HelixRestNamespace> _helixNamespaces;
   private ServletContextHandler _servletContextHandler;
   private List<AuditLogger> _auditLoggers;
 
   // Key is name of namespace, value of the resource config of that namespace
   private Map<String, ResourceConfig> _resourceConfigMap;
 
-  // In additional to regular servlets serving namespaced API endpoints, We have a default servlet
-  // serving un-namespaced API (/admin/v2/clusters/...) for default namespace as well. We use this
-  // literal as a key in _resourceConfigMap to keep records for default servlet.
-  // TODO: try to find a way to serve 2 sets of endpoints of default namespace in 1 servlet
-  private static final String DEFAULT_SERVLET_KEY = "DefaultServlet";
-
   public HelixRestServer(String zkAddr, int port, String urlPrefix) {
     this(zkAddr, port, urlPrefix, Collections.<AuditLogger>emptyList());
   }
@@ -77,7 +71,8 @@ public class HelixRestServer {
     init(namespaces, port, urlPrefix, auditLoggers);
   }
 
-  public HelixRestServer(List<HelixRestNamespace> namespaces, int port, String urlPrefix, List<AuditLogger> auditLoggers) {
+  public HelixRestServer(List<HelixRestNamespace> namespaces, int port, String urlPrefix,
+      List<AuditLogger> auditLoggers) {
     init(namespaces, port, urlPrefix, auditLoggers);
   }
 
@@ -93,32 +88,16 @@ public class HelixRestServer {
     _auditLoggers = auditLoggers;
     _resourceConfigMap = new HashMap<>();
     _servletContextHandler = new ServletContextHandler(_server, _urlPrefix);
+    _helixNamespaces = namespaces;
 
     // Initialize all namespaces
     try {
-      for (HelixRestNamespace namespace : namespaces) {
+      for (HelixRestNamespace namespace : _helixNamespaces) {
         LOG.info("Initializing namespace " + namespace.getName());
-        if (_resourceConfigMap.containsKey(namespace.getName())) {
-          throw new IllegalArgumentException(String.format("Duplicated namespace name \"%s\"", namespace.getName()));
-        }
-
-        // Create resource and context for namespaced servlet
-        _resourceConfigMap.put(namespace.getName(),
-            makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
-        LOG.info("Initializing servlet for namespace " + namespace.getName());
-        initServlet(_resourceConfigMap.get(namespace.getName()),
-            HelixRestUtils.makeServletPathSpec(namespace.getName(), false));
-
-        // Create special resource and context for default namespace servlet
+        prepareServlet(namespace, ServletType.COMMON_SERVLET);
         if (namespace.isDefault()) {
-          if (_resourceConfigMap.containsKey(DEFAULT_SERVLET_KEY)) {
-            throw new IllegalArgumentException("More than 1 default namespaces are provided");
-          }
-          LOG.info("Creating special servlet for default namespace");
-          _resourceConfigMap.put(DEFAULT_SERVLET_KEY,
-              makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
-          initServlet(_resourceConfigMap.get(DEFAULT_SERVLET_KEY),
-              HelixRestUtils.makeServletPathSpec(namespace.getName(), true));
+          LOG.info("Creating default servlet for default namespace");
+          prepareServlet(namespace, ServletType.DEFAULT_SERVLET);
         }
       }
     } catch (Exception e) {
@@ -135,12 +114,39 @@ public class HelixRestServer {
     }));
   }
 
-  private ResourceConfig makeResourceConfig(HelixRestNamespace ns, String... packages) {
+  private void prepareServlet(HelixRestNamespace namespace, ServletType type) {
+    String resourceConfigMapKey = getResourceConfigMapKey(type, namespace);
+    if (_resourceConfigMap.containsKey(resourceConfigMapKey)) {
+      throw new IllegalArgumentException(
+          String.format("Duplicated namespace name \"%s\"", namespace.getName()));
+    }
+
+    // Prepare resource config
+    ResourceConfig config = getResourceConfig(namespace, type);
+    _resourceConfigMap.put(resourceConfigMapKey, config);
+
+    // Initialize servlet
+    initServlet(config, String.format(type.getServletPathSpecTemplate(), namespace.getName()));
+  }
+
+  private String getResourceConfigMapKey(ServletType type, HelixRestNamespace namespace) {
+    return String.format("%s_%s", type.name(), namespace.getName());
+  }
+
+  private ResourceConfig getResourceConfig(HelixRestNamespace namespace, ServletType type) {
     ResourceConfig cfg = new ResourceConfig();
-    cfg.packages(packages)
-        .property(ContextPropertyKeys.SERVER_CONTEXT.name(), new ServerContext(ns.getMetadataStoreAddress()))
-        .register(new CORSFilter())
-        .register(new AuditLogFilter(_auditLoggers));
+    cfg.packages(type.getServletPackageArray());
+
+    cfg.property(ContextPropertyKeys.SERVER_CONTEXT.name(),
+        new ServerContext(namespace.getMetadataStoreAddress()));
+    if (type == ServletType.DEFAULT_SERVLET) {
+      cfg.property(ContextPropertyKeys.ALL_NAMESPACES.name(), _helixNamespaces);
+    } else {
+      cfg.property(ContextPropertyKeys.METADATA.name(), namespace);
+    }
+
+    cfg.register(new CORSFilter());
+    cfg.register(new AuditLogFilter(_auditLoggers));
     return cfg;
   }
 
@@ -186,7 +192,7 @@ public class HelixRestServer {
     for (Map.Entry<String, ResourceConfig> e : _resourceConfigMap.entrySet()) {
       ServerContext ctx = (ServerContext) e.getValue().getProperty(ContextPropertyKeys.SERVER_CONTEXT.name());
       if (ctx == null) {
-        LOG.warn("Server context for servlet " + e.getKey() + " is null.");
+        LOG.info("Server context for servlet " + e.getKey() + " is null.");
       } else {
         LOG.info("Closing context for servlet " + e.getKey());
         ctx.close();

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index a89ae5d..e3c565d 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -83,42 +83,6 @@ public class AbstractResource {
   protected HttpServletRequest _servletRequest;
   protected AuditLog.Builder _auditLogBuilder;
 
-  public ZkClient getZkClient() {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-          .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getZkClient();
-  }
-
-  public HelixAdmin getHelixAdmin() {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-        .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getHelixAdmin();
-  }
-
-  public ClusterSetup getClusterSetup() {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-        .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getClusterSetup();
-  }
-
-  public TaskDriver getTaskDriver(String clusterName) {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-        .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getTaskDriver(clusterName);
-  }
-
-  public ConfigAccessor getConfigAccessor() {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-        .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getConfigAccessor();
-  }
-
-  public HelixDataAccessor getDataAccssor(String clusterName) {
-    ServerContext serverContext = (ServerContext) _application.getProperties()
-        .get(ContextPropertyKeys.SERVER_CONTEXT.name());
-    return serverContext.getDataAccssor(clusterName);
-  }
-
   protected void addExceptionToAuditLog(Exception ex) {
     if (_auditLogBuilder == null) {
       _auditLogBuilder =
@@ -196,10 +160,6 @@ public class AbstractResource {
     return sw.toString();
   }
 
-  protected static ZNRecord toZNRecord(String data) throws IOException {
-    return OBJECT_MAPPER.reader(ZNRecord.class).readValue(data);
-  }
-
   protected Command getCommand(String commandStr) throws HelixException {
     if (commandStr == null) {
       throw new HelixException("Unknown command " + commandStr);

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
deleted file mode 100644
index 1c998b7..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ClusterAccessor.java
+++ /dev/null
@@ -1,400 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Response;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZKUtil;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.HelixConfigScope;
-import org.apache.helix.model.LeaderHistory;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
-import org.apache.helix.tools.ClusterSetup;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Path("/clusters")
-public class ClusterAccessor extends AbstractResource {
-  private static Logger _logger = LoggerFactory.getLogger(ClusterAccessor.class.getName());
-
-  public enum ClusterProperties {
-    controller,
-    instances,
-    liveInstances,
-    resources,
-    paused,
-    maintenance,
-    messages,
-    stateModelDefinitions,
-    clusters
-  }
-
-  @GET
-  public Response getClusters() {
-    HelixAdmin helixAdmin = getHelixAdmin();
-    List<String> clusters = helixAdmin.getClusters();
-
-    Map<String, List<String>> dataMap = new HashMap<>();
-    dataMap.put(ClusterProperties.clusters.name(), clusters);
-
-    return JSONRepresentation(dataMap);
-  }
-
-  @GET
-  @Path("{clusterId}")
-  public Response getClusterInfo(@PathParam("clusterId") String clusterId) {
-    if (!isClusterExist(clusterId)) {
-      return notFound();
-    }
-
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
-
-    Map<String, Object> clusterInfo = new HashMap<>();
-    clusterInfo.put(Properties.id.name(), clusterId);
-
-    LiveInstance controller = dataAccessor.getProperty(keyBuilder.controllerLeader());
-    if (controller != null) {
-      clusterInfo.put(ClusterProperties.controller.name(), controller.getInstanceName());
-    } else {
-      clusterInfo.put(ClusterProperties.controller.name(), "No Lead Controller!");
-    }
-
-    boolean paused = (dataAccessor.getProperty(keyBuilder.pause()) == null ? false : true);
-    clusterInfo.put(ClusterProperties.paused.name(), paused);
-    boolean maintenance =
-        (dataAccessor.getProperty(keyBuilder.maintenance()) == null ? false : true);
-    clusterInfo.put(ClusterProperties.maintenance.name(), maintenance);
-
-    List<String> idealStates = dataAccessor.getChildNames(keyBuilder.idealStates());
-    clusterInfo.put(ClusterProperties.resources.name(), idealStates);
-    List<String> instances = dataAccessor.getChildNames(keyBuilder.instanceConfigs());
-    clusterInfo.put(ClusterProperties.instances.name(), instances);
-    List<String> liveInstances = dataAccessor.getChildNames(keyBuilder.liveInstances());
-    clusterInfo.put(ClusterProperties.liveInstances.name(), liveInstances);
-
-    return JSONRepresentation(clusterInfo);
-  }
-
-
-  @PUT
-  @Path("{clusterId}")
-  public Response createCluster(@PathParam("clusterId") String clusterId,
-      @DefaultValue("false") @QueryParam("recreate") String recreate) {
-    boolean recreateIfExists = Boolean.valueOf(recreate);
-    ClusterSetup clusterSetup = getClusterSetup();
-
-    try {
-      clusterSetup.addCluster(clusterId, recreateIfExists);
-    } catch (Exception ex) {
-      _logger.error("Failed to create cluster " + clusterId + ", exception: " + ex);
-      return serverError(ex);
-    }
-
-    return created();
-  }
-
-  @DELETE
-  @Path("{clusterId}")
-  public Response deleteCluster(@PathParam("clusterId") String clusterId) {
-    ClusterSetup clusterSetup = getClusterSetup();
-
-    try {
-      clusterSetup.deleteCluster(clusterId);
-    } catch (HelixException ex) {
-      _logger.info(
-          "Failed to delete cluster " + clusterId + ", cluster is still in use. Exception: " + ex);
-      return badRequest(ex.getMessage());
-    } catch (Exception ex) {
-      _logger.error("Failed to delete cluster " + clusterId + ", exception: " + ex);
-      return serverError(ex);
-    }
-
-    return OK();
-  }
-
-  @POST
-  @Path("{clusterId}")
-  public Response updateCluster(@PathParam("clusterId") String clusterId,
-      @QueryParam("command") String commandStr, @QueryParam("superCluster") String superCluster,
-      String content) {
-    Command command;
-    try {
-      command = getCommand(commandStr);
-    } catch (HelixException ex) {
-      return badRequest(ex.getMessage());
-    }
-
-    ClusterSetup clusterSetup = getClusterSetup();
-    HelixAdmin helixAdmin = getHelixAdmin();
-
-    switch (command) {
-    case activate:
-      if (superCluster == null) {
-        return badRequest("Super Cluster name is missing!");
-      }
-      try {
-        clusterSetup.activateCluster(clusterId, superCluster, true);
-      } catch (Exception ex) {
-        _logger.error("Failed to add cluster " + clusterId + " to super cluster " + superCluster);
-        return serverError(ex);
-      }
-      break;
-
-    case expand:
-      try {
-        clusterSetup.expandCluster(clusterId);
-      } catch (Exception ex) {
-        _logger.error("Failed to expand cluster " + clusterId);
-        return serverError(ex);
-      }
-      break;
-
-    case enable:
-      try {
-        helixAdmin.enableCluster(clusterId, true);
-      } catch (Exception ex) {
-        _logger.error("Failed to enable cluster " + clusterId);
-        return serverError(ex);
-      }
-      break;
-
-    case disable:
-      try {
-        helixAdmin.enableCluster(clusterId, false);
-      } catch (Exception ex) {
-        _logger.error("Failed to disable cluster " + clusterId);
-        return serverError(ex);
-      }
-      break;
-    case enableMaintenanceMode:
-      try {
-        helixAdmin.enableMaintenanceMode(clusterId, true, content);
-      } catch (Exception ex) {
-        _logger.error("Failed to enable maintenance mode " + clusterId);
-        return serverError(ex);
-      }
-      break;
-    case disableMaintenanceMode:
-      try {
-        helixAdmin.enableMaintenanceMode(clusterId, false);
-      } catch (Exception ex) {
-        _logger.error("Failed to disable maintenance mode " + clusterId);
-        return serverError(ex);
-      }
-      break;
-    default:
-      return badRequest("Unsupported command " + command);
-    }
-
-    return OK();
-  }
-
-
-  @GET
-  @Path("{clusterId}/configs")
-  public Response getClusterConfig(@PathParam("clusterId") String clusterId) {
-    ConfigAccessor accessor = getConfigAccessor();
-    ClusterConfig config = null;
-    try {
-      config = accessor.getClusterConfig(clusterId);
-    } catch (HelixException ex) {
-      // cluster not found.
-      _logger.info("Failed to get cluster config for cluster " + clusterId
-          + ", cluster not found, Exception: " + ex);
-    } catch (Exception ex) {
-      _logger.error("Failed to get cluster config for cluster " + clusterId + " Exception: " + ex);
-      return serverError(ex);
-    }
-    if (config == null) {
-      return notFound();
-    }
-    return JSONRepresentation(config.getRecord());
-  }
-
-  @POST
-  @Path("{clusterId}/configs")
-  public Response updateClusterConfig(
-      @PathParam("clusterId") String clusterId, @QueryParam("command") String commandStr,
-      String content) {
-    Command command;
-    try {
-      command = getCommand(commandStr);
-    } catch (HelixException ex) {
-      return badRequest(ex.getMessage());
-    }
-
-    ZNRecord record;
-    try {
-      record = toZNRecord(content);
-    } catch (IOException e) {
-      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
-      return badRequest("Input is not a valid ZNRecord!");
-    }
-
-    if (!record.getId().equals(clusterId)) {
-      return badRequest("ID does not match the cluster name in input!");
-    }
-
-    ClusterConfig config = new ClusterConfig(record);
-    ConfigAccessor configAccessor = getConfigAccessor();
-    try {
-      switch (command) {
-      case update:
-        configAccessor.updateClusterConfig(clusterId, config);
-        break;
-      case delete: {
-        HelixConfigScope clusterScope =
-            new HelixConfigScopeBuilder(HelixConfigScope.ConfigScopeProperty.CLUSTER)
-                .forCluster(clusterId).build();
-        configAccessor.remove(clusterScope, config.getRecord());
-        }
-        break;
-
-      default:
-        return badRequest("Unsupported command " + commandStr);
-      }
-    } catch (HelixException ex) {
-      return notFound(ex.getMessage());
-    } catch (Exception ex) {
-      _logger.error(
-          "Failed to " + command + " cluster config, cluster " + clusterId + " new config: "
-              + content + ", Exception: " + ex);
-      return serverError(ex);
-    }
-    return OK();
-  }
-
-  @GET
-  @Path("{clusterId}/controller")
-  public Response getClusterController(@PathParam("clusterId") String clusterId) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    Map<String, Object> controllerInfo = new HashMap<>();
-    controllerInfo.put(Properties.id.name(), clusterId);
-
-    LiveInstance leader = dataAccessor.getProperty(dataAccessor.keyBuilder().controllerLeader());
-    if (leader != null) {
-      controllerInfo.put(ClusterProperties.controller.name(), leader.getInstanceName());
-      controllerInfo.putAll(leader.getRecord().getSimpleFields());
-    } else {
-      controllerInfo.put(ClusterProperties.controller.name(), "No Lead Controller!");
-    }
-
-    return JSONRepresentation(controllerInfo);
-  }
-
-  @GET
-  @Path("{clusterId}/controller/history")
-  public Response getClusterControllerHistory(@PathParam("clusterId") String clusterId) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    Map<String, Object> controllerHistory = new HashMap<>();
-    controllerHistory.put(Properties.id.name(), clusterId);
-
-    LeaderHistory history =
-        dataAccessor.getProperty(dataAccessor.keyBuilder().controllerLeaderHistory());
-    if (history != null) {
-      controllerHistory.put(Properties.history.name(), history.getHistoryList());
-    } else {
-      controllerHistory.put(Properties.history.name(), Collections.emptyList());
-    }
-
-    return JSONRepresentation(controllerHistory);
-  }
-
-  @GET
-  @Path("{clusterId}/controller/messages")
-  public Response getClusterControllerMessages(@PathParam("clusterId") String clusterId) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-
-    Map<String, Object> controllerMessages = new HashMap<>();
-    controllerMessages.put(Properties.id.name(), clusterId);
-
-    List<String> messages =
-        dataAccessor.getChildNames(dataAccessor.keyBuilder().controllerMessages());
-    controllerMessages.put(ClusterProperties.messages.name(), messages);
-    controllerMessages.put(Properties.count.name(), messages.size());
-
-    return JSONRepresentation(controllerMessages);
-  }
-
-  @GET
-  @Path("{clusterId}/controller/messages/{messageId}")
-  public Response getClusterControllerMessages(@PathParam("clusterId") String clusterId, @PathParam("messageId") String messageId) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    Message message = dataAccessor.getProperty(
-        dataAccessor.keyBuilder().controllerMessage(messageId));
-    return JSONRepresentation(message.getRecord());
-  }
-
-  @GET
-  @Path("{clusterId}/statemodeldefs")
-  public Response getClusterStateModelDefinitions(@PathParam("clusterId") String clusterId) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    List<String> stateModelDefs =
-        dataAccessor.getChildNames(dataAccessor.keyBuilder().stateModelDefs());
-
-    Map<String, Object> clusterStateModelDefs = new HashMap<>();
-    clusterStateModelDefs.put(Properties.id.name(), clusterId);
-    clusterStateModelDefs.put(ClusterProperties.stateModelDefinitions.name(), stateModelDefs);
-
-    return JSONRepresentation(clusterStateModelDefs);
-  }
-
-  @GET
-  @Path("{clusterId}/statemodeldefs/{statemodel}")
-  public Response getClusterStateModelDefinition(@PathParam("clusterId") String clusterId,
-      @PathParam("statemodel") String statemodel) {
-    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
-    StateModelDefinition stateModelDef =
-        dataAccessor.getProperty(dataAccessor.keyBuilder().stateModelDef(statemodel));
-
-    return JSONRepresentation(stateModelDef.getRecord());
-  }
-
-  private boolean isClusterExist(String cluster) {
-    ZkClient zkClient = getZkClient();
-    if (ZKUtil.isClusterSetup(cluster, zkClient)) {
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
deleted file mode 100644
index eeecba9..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
+++ /dev/null
@@ -1,545 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Response;
-
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.Error;
-import org.apache.helix.model.HealthStat;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.ParticipantHistory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.node.ArrayNode;
-import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
-
-@Path("/clusters/{clusterId}/instances")
-public class InstanceAccessor extends AbstractResource {
-  private final static Logger _logger = LoggerFactory.getLogger(InstanceAccessor.class);
-
-  public enum InstanceProperties {
-    instances,
-    online,
-    disabled,
-    config,
-    liveInstance,
-    resource,
-    resources,
-    partitions,
-    errors,
-    new_messages,
-    read_messages,
-    total_message_count,
-    read_message_count,
-    healthreports,
-    instanceTags
-  }
-
-  @GET
-  public Response getInstances(@PathParam("clusterId") String clusterId) {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), JsonNodeFactory.instance.textNode(clusterId));
-
-    ArrayNode instancesNode = root.putArray(InstanceProperties.instances.name());
-    ArrayNode onlineNode = root.putArray(InstanceProperties.online.name());
-    ArrayNode disabledNode = root.putArray(InstanceProperties.disabled.name());
-
-    List<String> instances = accessor.getChildNames(accessor.keyBuilder().instanceConfigs());
-
-    if (instances != null) {
-      instancesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(instances));
-    } else {
-      return notFound();
-    }
-
-    List<String> liveInstances = accessor.getChildNames(accessor.keyBuilder().liveInstances());
-    ClusterConfig clusterConfig = accessor.getProperty(accessor.keyBuilder().clusterConfig());
-
-    for (String instanceName : instances) {
-      InstanceConfig instanceConfig =
-          accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
-      if (instanceConfig != null) {
-        if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
-            && clusterConfig.getDisabledInstances().containsKey(instanceName))) {
-          disabledNode.add(JsonNodeFactory.instance.textNode(instanceName));
-        }
-
-        if (liveInstances.contains(instanceName)){
-          onlineNode.add(JsonNodeFactory.instance.textNode(instanceName));
-        }
-      }
-    }
-
-    return JSONRepresentation(root);
-  }
-
-  @POST
-  public Response updateInstances(@PathParam("clusterId") String clusterId,
-      @QueryParam("command") String command, String content) {
-    Command cmd;
-    try {
-      cmd = Command.valueOf(command);
-    } catch (Exception e) {
-      return badRequest("Invalid command : " + command);
-    }
-
-    HelixAdmin admin = getHelixAdmin();
-    try {
-      JsonNode node = null;
-      if (content.length() != 0) {
-        node = OBJECT_MAPPER.readTree(content);
-      }
-      if (node == null) {
-        return badRequest("Invalid input for content : " + content);
-      }
-      List<String> enableInstances = OBJECT_MAPPER
-          .readValue(node.get(InstanceProperties.instances.name()).toString(),
-              OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class));
-      switch (cmd) {
-      case enable:
-        admin.enableInstance(clusterId, enableInstances, true);
-
-        break;
-      case disable:
-        admin.enableInstance(clusterId, enableInstances, false);
-        break;
-      default:
-        _logger.error("Unsupported command :" + command);
-        return badRequest("Unsupported command :" + command);
-      }
-    } catch (Exception e) {
-      _logger.error("Failed in updating instances : " + content, e);
-      return badRequest(e.getMessage());
-    }
-    return OK();
-  }
-
-  @GET
-  @Path("{instanceName}")
-  public Response getInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    Map<String, Object> instanceMap = new HashMap<>();
-    instanceMap.put(Properties.id.name(), JsonNodeFactory.instance.textNode(instanceName));
-    instanceMap.put(InstanceProperties.liveInstance.name(), null);
-
-    InstanceConfig instanceConfig =
-        accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
-    LiveInstance liveInstance =
-        accessor.getProperty(accessor.keyBuilder().liveInstance(instanceName));
-
-    if (instanceConfig != null) {
-      instanceMap.put(InstanceProperties.config.name(), instanceConfig.getRecord());
-    } else {
-      return notFound();
-    }
-
-    if (liveInstance != null) {
-      instanceMap.put(InstanceProperties.liveInstance.name(), liveInstance.getRecord());
-    }
-
-    return JSONRepresentation(instanceMap);
-  }
-
-  @PUT
-  @Path("{instanceName}")
-  public Response addInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName, String content) {
-    HelixAdmin admin = getHelixAdmin();
-    ZNRecord record;
-    try {
-      record = toZNRecord(content);
-    } catch (IOException e) {
-      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
-      return badRequest("Input is not a vaild ZNRecord!");
-    }
-
-    try {
-      admin.addInstance(clusterId, new InstanceConfig(record));
-    } catch (Exception ex) {
-      _logger.error("Error in adding an instance: " + instanceName, ex);
-      return serverError(ex);
-    }
-
-    return OK();
-  }
-
-  @POST
-  @Path("{instanceName}")
-  public Response updateInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName, @QueryParam("command") String command,
-      String content) {
-    Command cmd;
-    try {
-      cmd = Command.valueOf(command);
-    } catch (Exception e) {
-      return badRequest("Invalid command : " + command);
-    }
-
-    HelixAdmin admin = getHelixAdmin();
-    try {
-      JsonNode node = null;
-      if (content.length() != 0) {
-        node = OBJECT_MAPPER.readTree(content);
-      }
-
-      switch (cmd) {
-      case enable:
-        admin.enableInstance(clusterId, instanceName, true);
-        break;
-      case disable:
-        admin.enableInstance(clusterId, instanceName, false);
-        break;
-      case reset:
-        if (!validInstance(node, instanceName)) {
-          return badRequest("Instance names are not match!");
-        }
-        admin.resetPartition(clusterId, instanceName,
-            node.get(InstanceProperties.resource.name()).toString(), (List<String>) OBJECT_MAPPER
-                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
-                    OBJECT_MAPPER.getTypeFactory()
-                        .constructCollectionType(List.class, String.class)));
-        break;
-      case addInstanceTag:
-        if (!validInstance(node, instanceName)) {
-          return badRequest("Instance names are not match!");
-        }
-        for (String tag : (List<String>) OBJECT_MAPPER
-            .readValue(node.get(InstanceProperties.instanceTags.name()).toString(),
-                OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class))) {
-          admin.addInstanceTag(clusterId, instanceName, tag);
-        }
-        break;
-      case removeInstanceTag:
-        if (!validInstance(node, instanceName)) {
-          return badRequest("Instance names are not match!");
-        }
-        for (String tag : (List<String>) OBJECT_MAPPER
-            .readValue(node.get(InstanceProperties.instanceTags.name()).toString(),
-                OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class))) {
-          admin.removeInstanceTag(clusterId, instanceName, tag);
-        }
-        break;
-      case enablePartitions:
-        admin.enablePartition(true, clusterId, instanceName,
-            node.get(InstanceProperties.resource.name()).getTextValue(),
-            (List<String>) OBJECT_MAPPER
-                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
-                    OBJECT_MAPPER.getTypeFactory()
-                        .constructCollectionType(List.class, String.class)));
-        break;
-      case disablePartitions:
-        admin.enablePartition(false, clusterId, instanceName,
-            node.get(InstanceProperties.resource.name()).getTextValue(),
-            (List<String>) OBJECT_MAPPER
-                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
-                    OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class)));
-        break;
-      default:
-        _logger.error("Unsupported command :" + command);
-        return badRequest("Unsupported command :" + command);
-      }
-    } catch (Exception e) {
-      _logger.error("Failed in updating instance : " + instanceName, e);
-      return badRequest(e.getMessage());
-    }
-    return OK();
-  }
-
-  @DELETE
-  @Path("{instanceName}")
-  public Response deleteInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) {
-    HelixAdmin admin = getHelixAdmin();
-    try {
-      InstanceConfig instanceConfig = admin.getInstanceConfig(clusterId, instanceName);
-      admin.dropInstance(clusterId, instanceConfig);
-    } catch (HelixException e) {
-      return badRequest(e.getMessage());
-    }
-
-    return OK();
-  }
-
-  @GET
-  @Path("{instanceName}/configs")
-  public Response getInstanceConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    InstanceConfig instanceConfig =
-        accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
-
-    if (instanceConfig != null) {
-      return JSONRepresentation(instanceConfig.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @PUT
-  @Path("{instanceName}/configs")
-  public Response updateInstanceConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName, String content) throws IOException {
-    HelixAdmin admin = getHelixAdmin();
-    ZNRecord record;
-    try {
-      record = toZNRecord(content);
-    } catch (IOException e) {
-      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
-      return badRequest("Input is not a vaild ZNRecord!");
-    }
-
-    try {
-      admin.setInstanceConfig(clusterId, instanceName, new InstanceConfig(record));
-    } catch (Exception ex) {
-      _logger.error("Error in update instance config: " + instanceName, ex);
-      return serverError(ex);
-    }
-
-    return OK();
-  }
-
-  @GET
-  @Path("{instanceName}/resources")
-  public Response getResourcesOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), instanceName);
-    ArrayNode resourcesNode = root.putArray(InstanceProperties.resources.name());
-
-    List<String> sessionIds = accessor.getChildNames(accessor.keyBuilder().sessions(instanceName));
-    if (sessionIds == null || sessionIds.size() == 0) {
-      return null;
-    }
-
-    // Only get resource list from current session id
-    String currentSessionId = sessionIds.get(0);
-
-    List<String> resources =
-        accessor.getChildNames(accessor.keyBuilder().currentStates(instanceName, currentSessionId));
-    if (resources != null && resources.size() > 0) {
-      resourcesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(resources));
-    }
-
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{instanceName}/resources/{resourceName}")
-  public Response getResourceOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName,
-      @PathParam("resourceName") String resourceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    List<String> sessionIds = accessor.getChildNames(accessor.keyBuilder().sessions(instanceName));
-    if (sessionIds == null || sessionIds.size() == 0) {
-      return notFound();
-    }
-
-    // Only get resource list from current session id
-    String currentSessionId = sessionIds.get(0);
-    CurrentState resourceCurrentState = accessor
-        .getProperty(accessor.keyBuilder().currentState(instanceName, currentSessionId, resourceName));
-    if (resourceCurrentState != null) {
-      return JSONRepresentation(resourceCurrentState.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @GET
-  @Path("{instanceName}/errors")
-  public Response getErrorsOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), instanceName);
-    ObjectNode errorsNode = JsonNodeFactory.instance.objectNode();
-
-    List<String> sessionIds =
-        accessor.getChildNames(accessor.keyBuilder().errors(instanceName));
-
-    if (sessionIds == null || sessionIds.size() == 0) {
-      return notFound();
-    }
-
-    for (String sessionId : sessionIds) {
-      List<String> resources =
-          accessor.getChildNames(accessor.keyBuilder().errors(instanceName, sessionId));
-      if (resources != null) {
-        ObjectNode resourcesNode = JsonNodeFactory.instance.objectNode();
-        for (String resourceName : resources) {
-          List<String> partitions = accessor
-              .getChildNames(accessor.keyBuilder().errors(instanceName, sessionId, resourceName));
-          if (partitions != null) {
-            ArrayNode partitionsNode = resourcesNode.putArray(resourceName);
-            partitionsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(partitions));
-          }
-        }
-        errorsNode.put(sessionId, resourcesNode);
-      }
-    }
-    root.put(InstanceProperties.errors.name(), errorsNode);
-
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{instanceName}/errors/{sessionId}/{resourceName}/{partitionName}")
-  public Response getErrorsOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName, @PathParam("sessionId") String sessionId,
-      @PathParam("resourceName") String resourceName,
-      @PathParam("partitionName") String partitionName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    Error error = accessor.getProperty(accessor.keyBuilder()
-        .stateTransitionError(instanceName, sessionId, resourceName, partitionName));
-    if (error != null) {
-      return JSONRepresentation(error.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @GET
-  @Path("{instanceName}/history")
-  public Response getHistoryOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    ParticipantHistory history =
-        accessor.getProperty(accessor.keyBuilder().participantHistory(instanceName));
-    if (history != null) {
-      return JSONRepresentation(history.getRecord());
-    }
-    return notFound();
-  }
-
-  @GET
-  @Path("{instanceName}/messages")
-  public Response getMessagesOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), instanceName);
-    ArrayNode newMessages = root.putArray(InstanceProperties.new_messages.name());
-    ArrayNode readMessages = root.putArray(InstanceProperties.read_messages.name());
-
-
-    List<String> messages =
-        accessor.getChildNames(accessor.keyBuilder().messages(instanceName));
-    if (messages == null || messages.size() == 0) {
-      return notFound();
-    }
-
-    for (String messageName : messages) {
-      Message message = accessor.getProperty(accessor.keyBuilder().message(instanceName, messageName));
-      if (message.getMsgState() == Message.MessageState.NEW) {
-        newMessages.add(messageName);
-      }
-
-      if (message.getMsgState() == Message.MessageState.READ) {
-        readMessages.add(messageName);
-      }
-    }
-
-    root.put(InstanceProperties.total_message_count.name(),
-        newMessages.size() + readMessages.size());
-    root.put(InstanceProperties.read_message_count.name(), readMessages.size());
-
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{instanceName}/messages/{messageId}")
-  public Response getMessageOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName,
-      @PathParam("messageId") String messageId) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    Message message = accessor.getProperty(accessor.keyBuilder().message(instanceName, messageId));
-    if (message != null) {
-      return JSONRepresentation(message.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @GET
-  @Path("{instanceName}/healthreports")
-  public Response getHealthReportsOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), instanceName);
-    ArrayNode healthReportsNode = root.putArray(InstanceProperties.healthreports.name());
-
-    List<String> healthReports =
-        accessor.getChildNames(accessor.keyBuilder().healthReports(instanceName));
-
-    if (healthReports != null && healthReports.size() > 0) {
-      healthReportsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(healthReports));
-    }
-
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{instanceName}/healthreports/{reportName}")
-  public Response getHealthReportsOnInstance(@PathParam("clusterId") String clusterId,
-      @PathParam("instanceName") String instanceName,
-      @PathParam("reportName") String reportName) throws IOException {
-    HelixDataAccessor accessor = getDataAccssor(clusterId);
-    HealthStat healthStat =
-        accessor.getProperty(accessor.keyBuilder().healthReport(instanceName, reportName));
-    if (healthStat != null) {
-      return JSONRepresentation(healthStat);
-    }
-
-    return notFound();
-  }
-
-  private boolean validInstance(JsonNode node, String instanceName) {
-    return instanceName.equals(node.get(Properties.id.name()).getValueAsText());
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/JobAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/JobAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/JobAccessor.java
deleted file mode 100644
index db6e6ad..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/JobAccessor.java
+++ /dev/null
@@ -1,200 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.core.Response;
-
-import org.apache.helix.HelixException;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.task.JobConfig;
-import org.apache.helix.task.JobContext;
-import org.apache.helix.task.TaskConfig;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.WorkflowConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.codehaus.jackson.node.ArrayNode;
-import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
-
-@Path("/clusters/{clusterId}/workflows/{workflowName}/jobs")
-public class JobAccessor extends AbstractResource {
-  private static Logger _logger = LoggerFactory.getLogger(JobAccessor.class.getName());
-
-  public enum JobProperties {
-    Jobs,
-    JobConfig,
-    JobContext,
-    TASK_COMMAND
-  }
-
-  @GET
-  public Response getJobs(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName) {
-    TaskDriver driver = getTaskDriver(clusterId);
-    WorkflowConfig workflowConfig = driver.getWorkflowConfig(workflowName);
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-
-    if (workflowConfig == null) {
-      return badRequest(String.format("Workflow %s is not found!", workflowName));
-    }
-
-    Set<String> jobs = workflowConfig.getJobDag().getAllNodes();
-    root.put(Properties.id.name(), JobProperties.Jobs.name());
-    ArrayNode jobsNode = root.putArray(JobProperties.Jobs.name());
-
-    if (jobs != null) {
-      jobsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(jobs));
-    }
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{jobName}")
-  public Response getJob(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
-    TaskDriver driver = getTaskDriver(clusterId);
-    Map<String, ZNRecord> jobMap = new HashMap<>();
-
-
-    JobConfig jobConfig = driver.getJobConfig(jobName);
-    if (jobConfig != null) {
-      jobMap.put(JobProperties.JobConfig.name(), jobConfig.getRecord());
-    } else {
-      return badRequest(String.format("Job config for %s does not exists", jobName));
-    }
-
-    JobContext jobContext =
-        driver.getJobContext(jobName);
-    jobMap.put(JobProperties.JobContext.name(), null);
-
-    if (jobContext != null) {
-      jobMap.put(JobProperties.JobContext.name(), jobContext.getRecord());
-    }
-
-    return JSONRepresentation(jobMap);
-  }
-
-  @PUT
-  @Path("{jobName}")
-  public Response addJob(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName,
-      String content) {
-    ZNRecord record;
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    try {
-      record = toZNRecord(content);
-      JobConfig.Builder jobConfig = JobAccessor.getJobConfig(record);
-      driver.enqueueJob(workflowName, jobName, jobConfig);
-    } catch (HelixException e) {
-      return badRequest(
-          String.format("Failed to enqueue job %s for reason : %s", jobName, e.getMessage()));
-    } catch (IOException e) {
-      return badRequest(String.format("Invalid input for Job Config of Job : %s", jobName));
-    }
-
-    return OK();
-  }
-
-  @DELETE
-  @Path("{jobName}")
-  public Response deleteJob(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    try {
-      driver.deleteJob(workflowName, jobName);
-    } catch (Exception e) {
-      return badRequest(e.getMessage());
-    }
-
-    return OK();
-  }
-
-  @GET
-  @Path("{jobName}/configs")
-  public Response getJobConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    JobConfig jobConfig = driver.getJobConfig(jobName);
-    if (jobConfig != null) {
-      return JSONRepresentation(jobConfig.getRecord());
-    }
-    return badRequest("Job config for " + jobName + " does not exists");
-  }
-
-  @GET
-  @Path("{jobName}/context")
-  public Response getJobContext(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    JobContext jobContext =
-        driver.getJobContext(jobName);
-    if (jobContext != null) {
-      return JSONRepresentation(jobContext.getRecord());
-    }
-    return badRequest("Job context for " + jobName + " does not exists");
-  }
-
-  protected static JobConfig.Builder getJobConfig(Map<String, String> cfgMap) {
-    return new JobConfig.Builder().fromMap(cfgMap);
-  }
-
-  protected static JobConfig.Builder getJobConfig(ZNRecord record) {
-    JobConfig.Builder jobConfig = new JobConfig.Builder().fromMap(record.getSimpleFields());
-    jobConfig.addTaskConfigMap(getTaskConfigMap(record.getMapFields()));
-
-    return jobConfig;
-  }
-
-  private static Map<String, TaskConfig> getTaskConfigMap(
-      Map<String, Map<String, String>> taskConfigs) {
-    Map<String, TaskConfig> taskConfigsMap = new HashMap<>();
-    if (taskConfigs == null || taskConfigs.isEmpty()) {
-      return Collections.emptyMap();
-    }
-
-    for (Map<String, String> taskConfigMap : taskConfigs.values()) {
-      if (!taskConfigMap.containsKey(JobProperties.TASK_COMMAND.name())) {
-        continue;
-      }
-
-      TaskConfig taskConfig =
-          new TaskConfig(taskConfigMap.get(JobProperties.TASK_COMMAND.name()), taskConfigMap);
-      taskConfigsMap.put(taskConfig.getId(), taskConfig);
-    }
-
-    return taskConfigsMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ResourceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ResourceAccessor.java
deleted file mode 100644
index 7ea571f..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/ResourceAccessor.java
+++ /dev/null
@@ -1,278 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Response;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixException;
-import org.apache.helix.PropertyPathBuilder;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.ResourceConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.codehaus.jackson.node.ArrayNode;
-import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
-
-@Path("/clusters/{clusterId}/resources")
-public class ResourceAccessor extends AbstractResource {
-  private final static Logger _logger = LoggerFactory.getLogger(ResourceAccessor.class);
-  public enum ResourceProperties {
-    idealState,
-    idealStates,
-    externalView,
-    externalViews,
-    resourceConfig,
-  }
-
-  @GET
-  public Response getResources(@PathParam("clusterId") String clusterId) {
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    root.put(Properties.id.name(), JsonNodeFactory.instance.textNode(clusterId));
-
-    ZkClient zkClient = getZkClient();
-
-    ArrayNode idealStatesNode = root.putArray(ResourceProperties.idealStates.name());
-    ArrayNode externalViewsNode = root.putArray(ResourceProperties.externalViews.name());
-
-    List<String> idealStates = zkClient.getChildren(PropertyPathBuilder.idealState(clusterId));
-    List<String> externalViews = zkClient.getChildren(PropertyPathBuilder.externalView(clusterId));
-
-    if (idealStates != null) {
-      idealStatesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(idealStates));
-    } else {
-      return notFound();
-    }
-
-    if (externalViews != null) {
-      externalViewsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(externalViews));
-    }
-
-    return JSONRepresentation(root);
-  }
-
-  @GET
-  @Path("{resourceName}")
-  public Response getResource(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName) throws IOException {
-    ConfigAccessor accessor = getConfigAccessor();
-    HelixAdmin admin = getHelixAdmin();
-
-    ResourceConfig resourceConfig = accessor.getResourceConfig(clusterId, resourceName);
-    IdealState idealState = admin.getResourceIdealState(clusterId, resourceName);
-    ExternalView externalView = admin.getResourceExternalView(clusterId, resourceName);
-
-    Map<String, ZNRecord> resourceMap = new HashMap<>();
-    if (idealState != null) {
-      resourceMap.put(ResourceProperties.idealState.name(), idealState.getRecord());
-    } else {
-      return notFound();
-    }
-
-    resourceMap.put(ResourceProperties.resourceConfig.name(), null);
-    resourceMap.put(ResourceProperties.externalView.name(), null);
-
-    if (resourceConfig != null) {
-      resourceMap.put(ResourceProperties.resourceConfig.name(), resourceConfig.getRecord());
-    }
-
-    if (externalView != null) {
-      resourceMap.put(ResourceProperties.externalView.name(), externalView.getRecord());
-    }
-
-    return JSONRepresentation(resourceMap);
-  }
-
-  @PUT
-  @Path("{resourceName}")
-  public Response addResource(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName,
-      @DefaultValue("-1") @QueryParam("numPartitions") int numPartitions,
-      @DefaultValue("") @QueryParam("stateModelRef") String stateModelRef,
-      @DefaultValue("SEMI_AUTO") @QueryParam("rebalancerMode") String rebalancerMode,
-      @DefaultValue("DEFAULT") @QueryParam("rebalanceStrategy") String rebalanceStrategy,
-      @DefaultValue("0") @QueryParam("bucketSize") int bucketSize,
-      @DefaultValue("-1") @QueryParam("maxPartitionsPerInstance") int maxPartitionsPerInstance,
-      String content) {
-
-    HelixAdmin admin = getHelixAdmin();
-
-    try {
-      if (content.length() != 0) {
-        ZNRecord record;
-        try {
-          record = toZNRecord(content);
-        } catch (IOException e) {
-          _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
-          return badRequest("Input is not a vaild ZNRecord!");
-        }
-
-        if (record.getSimpleFields() != null) {
-          admin.addResource(clusterId, resourceName, new IdealState(record));
-        }
-      } else {
-        admin.addResource(clusterId, resourceName, numPartitions, stateModelRef, rebalancerMode,
-            rebalanceStrategy, bucketSize, maxPartitionsPerInstance);
-      }
-    } catch (Exception e) {
-      _logger.error("Error in adding a resource: " + resourceName, e);
-      return serverError(e);
-    }
-
-    return OK();
-  }
-
-  @POST
-  @Path("{resourceName}")
-  public Response updateResource(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName, @QueryParam("command") String command,
-      @DefaultValue("-1") @QueryParam("replicas") int replicas,
-      @DefaultValue("") @QueryParam("keyPrefix") String keyPrefix,
-      @DefaultValue("") @QueryParam("group") String group){
-    Command cmd;
-    try {
-      cmd = Command.valueOf(command);
-    } catch (Exception e) {
-      return badRequest("Invalid command : " + command);
-    }
-
-    HelixAdmin admin = getHelixAdmin();
-    try {
-      switch (cmd) {
-      case enable:
-        admin.enableResource(clusterId, resourceName, true);
-        break;
-      case disable:
-        admin.enableResource(clusterId, resourceName, false);
-        break;
-      case rebalance:
-        if (replicas == -1) {
-          return badRequest("Number of replicas is needed for rebalancing!");
-        }
-        keyPrefix = keyPrefix.length() == 0 ? resourceName : keyPrefix;
-        admin.rebalance(clusterId, resourceName, replicas, keyPrefix, group);
-        break;
-      default:
-        _logger.error("Unsupported command :" + command);
-        return badRequest("Unsupported command :" + command);
-      }
-    } catch (Exception e) {
-      _logger.error("Failed in updating resource : " + resourceName, e);
-      return badRequest(e.getMessage());
-    }
-    return OK();
-  }
-
-  @DELETE
-  @Path("{resourceName}")
-  public Response deleteResource(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName) {
-    HelixAdmin admin = getHelixAdmin();
-    try {
-      admin.dropResource(clusterId, resourceName);
-    } catch (Exception e) {
-      _logger.error("Error in deleting a resource: " + resourceName, e);
-      return serverError();
-    }
-    return OK();
-  }
-
-  @GET
-  @Path("{resourceName}/configs")
-  public Response getResourceConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName) {
-    ConfigAccessor accessor = getConfigAccessor();
-    ResourceConfig resourceConfig = accessor.getResourceConfig(clusterId, resourceName);
-    if (resourceConfig != null) {
-      return JSONRepresentation(resourceConfig.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @POST
-  @Path("{resourceName}/configs")
-  public Response updateResourceConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName, String content) {
-    ZNRecord record;
-    try {
-      record = toZNRecord(content);
-    } catch (IOException e) {
-      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
-      return badRequest("Input is not a vaild ZNRecord!");
-    }
-    ResourceConfig resourceConfig = new ResourceConfig(record);
-    ConfigAccessor configAccessor = getConfigAccessor();
-    try {
-      configAccessor.updateResourceConfig(clusterId, resourceName, resourceConfig);
-    } catch (HelixException ex) {
-      return notFound(ex.getMessage());
-    } catch (Exception ex) {
-      _logger.error(
-          "Failed to update cluster config, cluster " + clusterId + " new config: " + content
-              + ", Exception: " + ex);
-      return serverError(ex);
-    }
-    return OK();
-  }
-
-  @GET
-  @Path("{resourceName}/idealState")
-  public Response getResourceIdealState(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName) {
-    HelixAdmin admin = getHelixAdmin();
-    IdealState idealState = admin.getResourceIdealState(clusterId, resourceName);
-    if (idealState != null) {
-      return JSONRepresentation(idealState.getRecord());
-    }
-
-    return notFound();
-  }
-
-  @GET
-  @Path("{resourceName}/externalView")
-  public Response getResourceExternalView(@PathParam("clusterId") String clusterId,
-      @PathParam("resourceName") String resourceName) {
-    HelixAdmin admin = getHelixAdmin();
-    ExternalView externalView = admin.getResourceExternalView(clusterId, resourceName);
-    if (externalView != null) {
-      return JSONRepresentation(externalView.getRecord());
-    }
-
-    return notFound();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/UIResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/UIResourceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/UIResourceAccessor.java
deleted file mode 100644
index 7ef22a4..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/UIResourceAccessor.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * 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 javax.ws.rs.GET;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.PathSegment;
-import javax.ws.rs.core.Response;
-import java.io.InputStream;
-import java.util.List;
-
-@Path("/ui")
-public class UIResourceAccessor extends AbstractResource {
-  private static final String INDEX_PAGE = "index.html";
-  private static final String UI_RESOURCE_FOLDER = "ui";
-
-  @GET
-  public Response getIndex() {
-    return getStaticFile(INDEX_PAGE);
-  }
-
-  @GET
-  @Path("{fileName}")
-  public Response getStaticFile(@PathParam("fileName") String fileName) {
-    InputStream is = getClass().getClassLoader().getResourceAsStream(UI_RESOURCE_FOLDER + "/" + fileName);
-
-    if (is == null) {
-      // forward any other requests to index except index is not found
-      return fileName.equalsIgnoreCase(INDEX_PAGE) ? notFound() : getIndex();
-    }
-
-    return Response.ok(is, MediaType.TEXT_HTML).build();
-  }
-
-  @GET
-  @Path("{any: .*}")
-  public Response getStaticFile(@PathParam("any") List<PathSegment> segments) {
-    // get the last segment
-    String fileName = segments.get(segments.size() - 1).getPath();
-
-    return getStaticFile(fileName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/WorkflowAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/WorkflowAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/WorkflowAccessor.java
deleted file mode 100644
index 398a4d2..0000000
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/WorkflowAccessor.java
+++ /dev/null
@@ -1,325 +0,0 @@
-package org.apache.helix.rest.server.resources;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Response;
-
-import org.apache.helix.HelixException;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.task.JobConfig;
-import org.apache.helix.task.JobDag;
-import org.apache.helix.task.JobQueue;
-import org.apache.helix.task.TaskDriver;
-import org.apache.helix.task.Workflow;
-import org.apache.helix.task.WorkflowConfig;
-import org.apache.helix.task.WorkflowContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.type.TypeFactory;
-import org.codehaus.jackson.node.ArrayNode;
-import org.codehaus.jackson.node.JsonNodeFactory;
-import org.codehaus.jackson.node.ObjectNode;
-import org.codehaus.jackson.node.TextNode;
-
-@Path("/clusters/{clusterId}/workflows")
-public class WorkflowAccessor extends AbstractResource {
-  private static Logger _logger = LoggerFactory.getLogger(WorkflowAccessor.class.getName());
-
-  public enum WorkflowProperties {
-    Workflows,
-    WorkflowConfig,
-    WorkflowContext,
-    Jobs,
-    ParentJobs
-  }
-
-  public enum TaskCommand {
-    stop,
-    resume,
-    clean
-  }
-
-  @GET
-  public Response getWorkflows(@PathParam("clusterId") String clusterId) {
-    TaskDriver taskDriver = getTaskDriver(clusterId);
-    Map<String, WorkflowConfig> workflowConfigMap = taskDriver.getWorkflows();
-    Map<String, List<String>> dataMap = new HashMap<>();
-    dataMap.put(WorkflowProperties.Workflows.name(), new ArrayList<>(workflowConfigMap.keySet()));
-
-    return JSONRepresentation(dataMap);
-  }
-
-  @GET
-  @Path("{workflowId}")
-  public Response getWorkflow(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId) {
-    TaskDriver taskDriver = getTaskDriver(clusterId);
-    WorkflowConfig workflowConfig = taskDriver.getWorkflowConfig(workflowId);
-    WorkflowContext workflowContext = taskDriver.getWorkflowContext(workflowId);
-
-    ObjectNode root = JsonNodeFactory.instance.objectNode();
-    TextNode id = JsonNodeFactory.instance.textNode(workflowId);
-    root.put(Properties.id.name(), id);
-
-    ObjectNode workflowConfigNode = JsonNodeFactory.instance.objectNode();
-    ObjectNode workflowContextNode = JsonNodeFactory.instance.objectNode();
-
-    if (workflowConfig != null) {
-      getWorkflowConfigNode(workflowConfigNode, workflowConfig.getRecord());
-    }
-
-    if (workflowContext != null) {
-      getWorkflowContextNode(workflowContextNode, workflowContext.getRecord());
-    }
-
-    root.put(WorkflowProperties.WorkflowConfig.name(), workflowConfigNode);
-    root.put(WorkflowProperties.WorkflowContext.name(), workflowContextNode);
-
-    JobDag jobDag = workflowConfig.getJobDag();
-    ArrayNode jobs = OBJECT_MAPPER.valueToTree(jobDag.getAllNodes());
-    ObjectNode parentJobs = OBJECT_MAPPER.valueToTree(jobDag.getParentsToChildren());
-    root.put(WorkflowProperties.Jobs.name(), jobs);
-    root.put(WorkflowProperties.ParentJobs.name(), parentJobs);
-
-    return JSONRepresentation(root);
-  }
-
-  @PUT
-  @Path("{workflowId}")
-  public Response createWorkflow(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId, String content) {
-    TaskDriver driver = getTaskDriver(clusterId);
-    Map<String, String> cfgMap;
-    try {
-      JsonNode root = OBJECT_MAPPER.readTree(content);
-      cfgMap = OBJECT_MAPPER
-          .readValue(root.get(WorkflowProperties.WorkflowConfig.name()).toString(),
-              TypeFactory.defaultInstance()
-                  .constructMapType(HashMap.class, String.class, String.class));
-
-      WorkflowConfig workflowConfig = WorkflowConfig.Builder.fromMap(cfgMap).build();
-
-      // Since JobQueue can keep adding jobs, Helix create JobQueue will ignore the jobs
-      if (workflowConfig.isJobQueue()) {
-        driver.start(new JobQueue.Builder(workflowId).setWorkflowConfig(workflowConfig).build());
-        return OK();
-      }
-
-      Workflow.Builder workflow = new Workflow.Builder(workflowId);
-
-      if (root.get(WorkflowProperties.Jobs.name()) != null) {
-        Map<String, JobConfig.Builder> jobConfigs =
-            getJobConfigs((ArrayNode) root.get(WorkflowProperties.Jobs.name()));
-        for (Map.Entry<String, JobConfig.Builder> job : jobConfigs.entrySet()) {
-          workflow.addJob(job.getKey(), job.getValue());
-        }
-      }
-
-      if (root.get(WorkflowProperties.ParentJobs.name()) != null) {
-        Map<String, List<String>> parentJobs = OBJECT_MAPPER
-            .readValue(root.get(WorkflowProperties.ParentJobs.name()).toString(),
-                TypeFactory.defaultInstance()
-                    .constructMapType(HashMap.class, String.class, List.class));
-        for (Map.Entry<String, List<String>> entry : parentJobs.entrySet()) {
-          String parentJob = entry.getKey();
-          for (String childJob : entry.getValue()) {
-            workflow.addParentChildDependency(parentJob, childJob);
-          }
-        }
-      }
-
-      driver.start(workflow.build());
-    } catch (IOException e) {
-      return badRequest(String
-          .format("Invalid input of Workflow %s for reason : %s", workflowId, e.getMessage()));
-    } catch (HelixException e) {
-      return badRequest(String
-          .format("Failed to create workflow %s for reason : %s", workflowId, e.getMessage()));
-    }
-    return OK();
-  }
-
-  @DELETE
-  @Path("{workflowId}")
-  public Response deleteWorkflow(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId) {
-    TaskDriver driver = getTaskDriver(clusterId);
-    try {
-      driver.delete(workflowId);
-    } catch (HelixException e) {
-      return badRequest(String
-          .format("Failed to delete workflow %s for reason : %s", workflowId, e.getMessage()));
-    }
-    return OK();
-  }
-
-  @POST
-  @Path("{workflowId}")
-  public Response updateWorkflow(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId, @QueryParam("command") String command) {
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    try {
-      TaskCommand cmd = TaskCommand.valueOf(command);
-      switch (cmd) {
-      case stop:
-        driver.stop(workflowId);
-        break;
-      case resume:
-        driver.resume(workflowId);
-        break;
-      case clean:
-        driver.cleanupQueue(workflowId);
-        break;
-      default:
-        return badRequest(String.format("Invalid command : %s", command));
-      }
-    } catch (HelixException e) {
-      return badRequest(
-          String.format("Failed to execute operation %s for reason : %s", command, e.getMessage()));
-    } catch (Exception e) {
-      return serverError(e);
-    }
-
-    return OK();
-  }
-
-  @GET
-  @Path("{workflowId}/configs")
-  public Response getWorkflowConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId) {
-    TaskDriver taskDriver = getTaskDriver(clusterId);
-    WorkflowConfig workflowConfig = taskDriver.getWorkflowConfig(workflowId);
-    ObjectNode workflowConfigNode = JsonNodeFactory.instance.objectNode();
-    if (workflowConfig != null) {
-      getWorkflowConfigNode(workflowConfigNode, workflowConfig.getRecord());
-    }
-
-    return JSONRepresentation(workflowConfigNode);
-  }
-
-  @POST
-  @Path("{workflowId}/configs")
-  public Response updateWorkflowConfig(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId, String content) {
-    ZNRecord record;
-    TaskDriver driver = getTaskDriver(clusterId);
-
-    try {
-      record = toZNRecord(content);
-
-      WorkflowConfig workflowConfig = driver.getWorkflowConfig(workflowId);
-      if (workflowConfig == null) {
-        return badRequest(
-            String.format("WorkflowConfig for workflow %s does not exists!", workflowId));
-      }
-
-      workflowConfig.getRecord().update(record);
-      driver.updateWorkflow(workflowId, workflowConfig);
-    } catch (HelixException e) {
-      return badRequest(
-          String.format("Failed to update WorkflowConfig for workflow %s", workflowId));
-    } catch (Exception e) {
-      return badRequest(String.format("Invalid WorkflowConfig for workflow %s", workflowId));
-    }
-
-    return OK();
-  }
-
-  @GET
-  @Path("{workflowId}/context")
-  public Response getWorkflowContext(@PathParam("clusterId") String clusterId,
-      @PathParam("workflowId") String workflowId) {
-    TaskDriver taskDriver = getTaskDriver(clusterId);
-    WorkflowContext workflowContext = taskDriver.getWorkflowContext(workflowId);
-    ObjectNode workflowContextNode = JsonNodeFactory.instance.objectNode();
-    if (workflowContext != null) {
-      getWorkflowContextNode(workflowContextNode, workflowContext.getRecord());
-    }
-
-    return JSONRepresentation(workflowContextNode);
-  }
-
-  private void getWorkflowConfigNode(ObjectNode workflowConfigNode, ZNRecord record) {
-    for (Map.Entry<String, String> entry : record.getSimpleFields().entrySet()) {
-      if (!entry.getKey().equals(WorkflowConfig.WorkflowConfigProperty.Dag)) {
-        workflowConfigNode.put(entry.getKey(), JsonNodeFactory.instance.textNode(entry.getValue()));
-      }
-    }
-  }
-
-  private void getWorkflowContextNode(ObjectNode workflowContextNode, ZNRecord record) {
-    if (record.getMapFields() != null) {
-      for (String fieldName : record.getMapFields().keySet()) {
-        JsonNode node = OBJECT_MAPPER.valueToTree(record.getMapField(fieldName));
-        workflowContextNode.put(fieldName, node);
-      }
-    }
-
-    if (record.getSimpleFields() != null) {
-      for (Map.Entry<String, String> entry : record.getSimpleFields().entrySet()) {
-        workflowContextNode
-            .put(entry.getKey(), JsonNodeFactory.instance.textNode(entry.getValue()));
-      }
-    }
-  }
-
-  private Map<String, JobConfig.Builder> getJobConfigs(ArrayNode root)
-      throws HelixException, IOException {
-    Map<String, JobConfig.Builder> jobConfigsMap = new HashMap<>();
-    for (Iterator<JsonNode> it = root.getElements(); it.hasNext(); ) {
-      JsonNode job = it.next();
-      ZNRecord record = null;
-
-      try {
-        record = toZNRecord(job.toString());
-      } catch (IOException e) {
-        // Ignore the parse since it could be just simple fields
-      }
-
-      if (record == null || record.getSimpleFields().isEmpty()) {
-        Map<String, String> cfgMap = OBJECT_MAPPER.readValue(job.toString(),
-            TypeFactory.defaultInstance()
-                .constructMapType(HashMap.class, String.class, String.class));
-        jobConfigsMap
-            .put(job.get(Properties.id.name()).getTextValue(), JobAccessor.getJobConfig(cfgMap));
-      } else {
-        jobConfigsMap
-            .put(job.get(Properties.id.name()).getTextValue(), JobAccessor.getJobConfig(record));
-      }
-    }
-
-    return jobConfigsMap;
-  }
-}


[37/50] [abbrv] helix git commit: Improve Helix maintenance mode

Posted by jx...@apache.org.
Improve Helix maintenance mode

1. Remove the exception in best possible stage to let pipeline pass.
2. Add event generation for maintenance mode change.


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

Branch: refs/heads/master
Commit: 89089b4523e91e356a87f5ad151ee9432b574cf8
Parents: ec7eaaa
Author: Junkai Xue <jx...@linkedin.com>
Authored: Fri Dec 15 11:43:07 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:46 2018 -0800

----------------------------------------------------------------------
 .../controller/GenericHelixController.java      | 47 +++++++++++++-------
 .../stages/BestPossibleStateCalcStage.java      |  8 ++--
 2 files changed, 35 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/89089b45/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index 6d1af7c..2546bd2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -73,6 +73,7 @@ import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.ResourceConfig;
@@ -128,6 +129,7 @@ public class GenericHelixController implements IdealStateChangeListener,
    * will be no-op. Other event handling logic keeps the same when the flag is set.
    */
   private boolean _paused;
+  private boolean _inMaintenanceMode;
 
   /**
    * The timer that can periodically run the rebalancing pipeline. The timer will start if there is
@@ -632,23 +634,10 @@ public class GenericHelixController implements IdealStateChangeListener,
     }
 
     PauseSignal pauseSignal = accessor.getProperty(keyBuilder.pause());
-    if (pauseSignal != null) {
-      if (!_paused) {
-        _paused = true;
-        logger.info("controller is now paused");
-      }
-    } else {
-      if (_paused) {
-        _paused = false;
-        logger.info("controller is now resumed");
-        ClusterEvent event = new ClusterEvent(_clusterName, ClusterEventType.Resume);
-        event.addAttribute(AttributeName.changeContext.name(), changeContext);
-        event.addAttribute(AttributeName.helixmanager.name(), changeContext.getManager());
-        event.addAttribute(AttributeName.eventData.name(), pauseSignal);
-        _eventQueue.put(event);
-        _taskEventQueue.put(event.clone());
-      }
-    }
+    MaintenanceSignal maintenanceSignal = accessor.getProperty(keyBuilder.maintenance());
+    _paused = updateControllerState(changeContext, pauseSignal, _paused);
+    _inMaintenanceMode = updateControllerState(changeContext, maintenanceSignal, _inMaintenanceMode);
+
     synchronized (this) {
       if (_clusterStatusMonitor == null) {
         _clusterStatusMonitor = new ClusterStatusMonitor(changeContext.getManager().getClusterName());
@@ -758,6 +747,30 @@ public class GenericHelixController implements IdealStateChangeListener,
     }
   }
 
+  private boolean updateControllerState(NotificationContext changeContext, PauseSignal signal,
+      boolean statusFlag) {
+    if (signal != null) {
+      // This logic is used for recording first time entering PAUSE/MAINTENCE mode
+      if (!statusFlag) {
+        statusFlag = true;
+        logger.info(String.format("controller is now %s",
+            (signal instanceof MaintenanceSignal) ? "in maintenance mode" : "paused"));
+      }
+    } else {
+      if (statusFlag) {
+        statusFlag = false;
+        logger.info("controller is now resumed from paused state");
+        ClusterEvent event = new ClusterEvent(_clusterName, ClusterEventType.Resume);
+        event.addAttribute(AttributeName.changeContext.name(), changeContext);
+        event.addAttribute(AttributeName.helixmanager.name(), changeContext.getManager());
+        event.addAttribute(AttributeName.eventData.name(), signal);
+        _eventQueue.put(event);
+        _taskEventQueue.put(event.clone());
+      }
+    }
+    return statusFlag;
+  }
+
 
   // TODO: refactor this to use common/ClusterEventProcessor.
   private class ClusterEventProcessor extends Thread {

http://git-wip-us.apache.org/repos/asf/helix/blob/89089b45/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index e96f0f3..9566f2c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -180,15 +180,17 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
             "Offline Instances count %d greater than allowed count %d. Stop rebalance pipeline and pause the cluster %s",
             offlineCount, maxOfflineInstancesAllowed, cache.getClusterName());
         if (manager != null) {
-          manager.getClusterManagmentTool()
-              .enableMaintenanceMode(manager.getClusterName(), true, errMsg);
+          if (manager.getHelixDataAccessor()
+              .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+            manager.getClusterManagmentTool()
+                .enableMaintenanceMode(manager.getClusterName(), true, errMsg);
+          }
         } else {
           logger.error("Failed to pause cluster, HelixManager is not set!");
         }
         if (!cache.isTaskCache()) {
           updateRebalanceStatus(true, manager, cache, clusterStatusMonitor, errMsg);
         }
-        throw new HelixException(errMsg);
       }
     }
   }


[25/50] [abbrv] helix git commit: Remove redundant logs

Posted by jx...@apache.org.
Remove redundant logs


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

Branch: refs/heads/master
Commit: ae58e79eff8b34c16e7ba9713738b85509e47ce7
Parents: d25552b
Author: hrzhang <hr...@linkedin.com>
Authored: Thu Dec 7 15:22:27 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:07 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/helix/controller/HelixControllerMain.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ae58e79e/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
index 8d42c6e..5265c24 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/HelixControllerMain.java
@@ -205,7 +205,6 @@ public class HelixControllerMain {
     controllerName = cmd.getOptionValue(name);
 
     // Espresso_driver.py will consume this
-    System.out.println("HARRY_DEBUG: starting HelixControllerMain");
     logger.info("Cluster manager started, zkServer: " + zkConnectString + ", clusterName:"
         + clusterName + ", controllerName:" + controllerName + ", mode:" + controllerMode);
 


[12/50] [abbrv] helix git commit: Cluster Maintenance Mode feature support

Posted by jx...@apache.org.
Cluster Maintenance Mode feature support

Helix does not have a state that keep original partitions are active status without doing rebalance partition placement.
The only state controller can switch to is paused state. If the controller has been paused, all the replicas in this cluster will not be active anymore.
It is better to have another mode that let current replicas functioning well without new replicas bootstraps when the cluster is full or instance.
There are several scenarios that may need such mode that keep original
assignment of partitions without partition movement. At same time, no partition will be assigned for newly added resources. This mode is
call cluster maintenance mode.

For more detail, please refer: https://iwww.corp.linkedin.com/wiki/cf/display/ENGS/Cluster+Maintenance+Mode+Design


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

Branch: refs/heads/master
Commit: a7477c3bbc85059b2e522f5caa214c33eb4c3e15
Parents: bd9f7a4
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Nov 2 15:00:24 2017 -0700
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:11 2018 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/helix/HelixAdmin.java  | 15 ++++
 .../org/apache/helix/HelixDataAccessor.java     |  2 +
 .../main/java/org/apache/helix/PropertyKey.java |  9 +++
 .../org/apache/helix/PropertyPathBuilder.java   | 20 +++--
 .../java/org/apache/helix/PropertyType.java     |  1 +
 .../rebalancer/AbstractRebalancer.java          | 35 ++++++++-
 .../rebalancer/DelayedAutoRebalancer.java       | 34 --------
 .../rebalancer/MaintenanceRebalancer.java       | 43 +++++++++++
 .../stages/BestPossibleStateCalcStage.java      | 17 ++--
 .../controller/stages/ClusterDataCache.java     |  9 +++
 .../apache/helix/manager/zk/ZKHelixAdmin.java   | 29 ++++++-
 .../helix/manager/zk/ZKHelixDataAccessor.java   |  8 ++
 .../apache/helix/model/MaintenanceSignal.java   | 13 ++++
 .../java/org/apache/helix/MockAccessor.java     |  5 ++
 .../controller/TestClusterMaintenanceMode.java  | 81 ++++++++++++++++++++
 .../org/apache/helix/mock/MockHelixAdmin.java   |  8 ++
 16 files changed, 277 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
index 652ab7a..9562a0b 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixAdmin.java
@@ -259,6 +259,21 @@ public interface HelixAdmin {
   void enableCluster(String clusterName, boolean enabled, String reason);
 
   /**
+   * Enable or disable maintenance mode for a cluster
+   * @param clusterName
+   * @param enabled
+   */
+  void enableMaintenanceMode(String clusterName, boolean enabled);
+
+  /**
+   * Enable or disable maintenance mode for a cluster
+   * @param clusterName
+   * @param enabled
+   * @param reason
+   */
+  void enableMaintenanceMode(String clusterName, boolean enabled, String reason);
+
+  /**
    * Reset a list of partitions in error state for an instance
    * The partitions are assume to be in error state and reset will bring them from error
    * to initial state. An error to initial state transition is required for reset.

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java
index 5c2baec..f8ae131 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixDataAccessor.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.StateModelDefinition;
@@ -38,6 +39,7 @@ public interface HelixDataAccessor {
   boolean createControllerMessage(Message message);
   boolean createControllerLeader(LiveInstance leader);
   boolean createPause(PauseSignal pauseSignal);
+  boolean createMaintenance(MaintenanceSignal maintenanceSignal);
 
   /**
    * Set a property, overwrite if it exists and creates if not exists. This api

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyKey.java b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
index f2d1e7b..4e58a89 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyKey.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyKey.java
@@ -32,6 +32,7 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LeaderHistory;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ParticipantHistory;
 import org.apache.helix.model.PauseSignal;
@@ -674,6 +675,14 @@ public class PropertyKey {
     }
 
     /**
+     * Get a property key associated with {@link MaintenanceSignal}
+     * @return {@link PropertyKey}
+     */
+    public PropertyKey maintenance() {
+      return new PropertyKey(MAINTENANCE, MaintenanceSignal.class, _clusterName);
+    }
+
+    /**
      * Get a property key associated with a {@link HealthStat} for an instance
      * @param instanceName
      * @param id identifies the statistics

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java b/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
index a61519a..cfff5bb 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyPathBuilder.java
@@ -19,17 +19,6 @@ package org.apache.helix;
  * under the License.
  */
 
-import static org.apache.helix.PropertyType.CONFIGS;
-import static org.apache.helix.PropertyType.CURRENTSTATES;
-import static org.apache.helix.PropertyType.EXTERNALVIEW;
-import static org.apache.helix.PropertyType.HISTORY;
-import static org.apache.helix.PropertyType.IDEALSTATES;
-import static org.apache.helix.PropertyType.LIVEINSTANCES;
-import static org.apache.helix.PropertyType.MESSAGES;
-import static org.apache.helix.PropertyType.PAUSE;
-import static org.apache.helix.PropertyType.STATEMODELDEFS;
-import static org.apache.helix.PropertyType.STATUSUPDATES;
-
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -42,6 +31,7 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LeaderHistory;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.StateModelDefinition;
@@ -49,6 +39,8 @@ import org.apache.helix.model.StatusUpdate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.helix.PropertyType.*;
+
 /**
  * Utility mapping properties to their Zookeeper locations
  */
@@ -70,6 +62,7 @@ public class PropertyPathBuilder {
     typeToClassMapping.put(STATUSUPDATES, StatusUpdate.class);
     typeToClassMapping.put(HISTORY, LeaderHistory.class);
     typeToClassMapping.put(PAUSE, PauseSignal.class);
+    typeToClassMapping.put(MAINTENANCE, MaintenanceSignal.class);
 
     // @formatter:off
     addEntry(PropertyType.CONFIGS, 1, "/{clusterName}/CONFIGS");
@@ -132,6 +125,7 @@ public class PropertyPathBuilder {
     addEntry(PropertyType.LEADER, 1, "/{clusterName}/CONTROLLER/LEADER");
     addEntry(PropertyType.HISTORY, 1, "/{clusterName}/CONTROLLER/HISTORY");
     addEntry(PropertyType.PAUSE, 1, "/{clusterName}/CONTROLLER/PAUSE");
+    addEntry(PropertyType.MAINTENANCE, 1, "/{clusterName}/CONTROLLER/MAINTENANCE");
     // @formatter:on
 
   }
@@ -354,4 +348,8 @@ public class PropertyPathBuilder {
   public static String pause(String clusterName) {
     return String.format("/%s/CONTROLLER/PAUSE", clusterName);
   }
+
+  public static String maintenance(String clusterName) {
+    return String.format("/%s/CONTROLLER/MAINTENANCE", clusterName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/PropertyType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/PropertyType.java b/helix-core/src/main/java/org/apache/helix/PropertyType.java
index b522014..73da39e 100644
--- a/helix-core/src/main/java/org/apache/helix/PropertyType.java
+++ b/helix-core/src/main/java/org/apache/helix/PropertyType.java
@@ -58,6 +58,7 @@ public enum PropertyType {
   LEADER(Type.CONTROLLER, false, false, true, true),
   HISTORY(Type.CONTROLLER, true, true, true),
   PAUSE(Type.CONTROLLER, true, false, true),
+  MAINTENANCE(Type.CONTROLLER, true, false, true),
   MESSAGES_CONTROLLER(Type.CONTROLLER, true, false, true),
   STATUSUPDATES_CONTROLLER(Type.CONTROLLER, true, true, true),
   ERRORS_CONTROLLER(Type.CONTROLLER, true, true, true);

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AbstractRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AbstractRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AbstractRebalancer.java
index b1888d1..5d77eb3 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AbstractRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AbstractRebalancer.java
@@ -314,7 +314,7 @@ public abstract class AbstractRebalancer implements Rebalancer, MappingCalculato
    * Sorter for nodes that sorts according to the CurrentState of the partition. There are only two priorities:
    * (1) Top-state and second states have priority 0. (2) Other states(or no state) have priority 1.
    */
-  private static class TopStatePreferenceListComparator implements Comparator<String> {
+  protected static class TopStatePreferenceListComparator implements Comparator<String> {
     protected final Map<String, String> _currentStateMap;
     protected final StateModelDefinition _stateModelDef;
 
@@ -345,4 +345,37 @@ public abstract class AbstractRebalancer implements Rebalancer, MappingCalculato
       return p1 - p2;
     }
   }
+
+  /**
+   * Sorter for nodes that sorts according to the CurrentState of the partition, based on the state priority defined
+   * in the state model definition.
+   * If the CurrentState doesn't exist, treat it as having lowest priority(Integer.MAX_VALUE).
+   */
+  protected static class PreferenceListNodeComparator implements Comparator<String> {
+    protected final Map<String, String> _currentStateMap;
+    protected final StateModelDefinition _stateModelDef;
+
+    public PreferenceListNodeComparator(Map<String, String> currentStateMap, StateModelDefinition stateModelDef) {
+      _currentStateMap = currentStateMap;
+      _stateModelDef = stateModelDef;
+    }
+
+    @Override
+    public int compare(String ins1, String ins2) {
+      Integer p1 = Integer.MAX_VALUE;
+      Integer p2 = Integer.MAX_VALUE;
+
+      Map<String, Integer> statesPriorityMap = _stateModelDef.getStatePriorityMap();
+      String state1 = _currentStateMap.get(ins1);
+      String state2 = _currentStateMap.get(ins2);
+      if (state1 != null && statesPriorityMap.containsKey(state1)) {
+        p1 = statesPriorityMap.get(state1);
+      }
+      if (state2 != null && statesPriorityMap.containsKey(state2)) {
+        p2 = statesPriorityMap.get(state2);
+      }
+
+      return p1.compareTo(p2);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index adac235..5ebb57d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -529,38 +529,4 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
 
     return true;
   }
-
-
-  /**
-   * Sorter for nodes that sorts according to the CurrentState of the partition, based on the state priority defined
-   * in the state model definition.
-   * If the CurrentState doesn't exist, treat it as having lowest priority(Integer.MAX_VALUE).
-   */
-  private static class PreferenceListNodeComparator implements Comparator<String> {
-    protected final Map<String, String> _currentStateMap;
-    protected final StateModelDefinition _stateModelDef;
-
-    public PreferenceListNodeComparator(Map<String, String> currentStateMap, StateModelDefinition stateModelDef) {
-      _currentStateMap = currentStateMap;
-      _stateModelDef = stateModelDef;
-    }
-
-    @Override
-    public int compare(String ins1, String ins2) {
-      Integer p1 = Integer.MAX_VALUE;
-      Integer p2 = Integer.MAX_VALUE;
-
-      Map<String, Integer> statesPriorityMap = _stateModelDef.getStatePriorityMap();
-      String state1 = _currentStateMap.get(ins1);
-      String state2 = _currentStateMap.get(ins2);
-      if (state1 != null && statesPriorityMap.containsKey(state1)) {
-        p1 = statesPriorityMap.get(state1);
-      }
-      if (state2 != null && statesPriorityMap.containsKey(state2)) {
-        p2 = statesPriorityMap.get(state2);
-      }
-
-      return p1.compareTo(p2);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/MaintenanceRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/MaintenanceRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/MaintenanceRebalancer.java
new file mode 100644
index 0000000..d324659
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/MaintenanceRebalancer.java
@@ -0,0 +1,43 @@
+package org.apache.helix.controller.rebalancer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Partition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MaintenanceRebalancer extends SemiAutoRebalancer {
+  private static final Logger LOG = LoggerFactory.getLogger(MaintenanceRebalancer.class);
+
+  @Override
+  public IdealState computeNewIdealState(String resourceName, IdealState currentIdealState,
+      CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) {
+    LOG.info(String
+        .format("Start computing ideal state for resource %s in maintenance mode.", resourceName));
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    if (currentStateMap == null || currentStateMap.size() == 0) {
+      LOG.warn(String
+          .format("No new partition will be assigned for %s in maintenance mode", resourceName));
+      currentIdealState.setPreferenceLists(Collections.EMPTY_MAP);
+      return currentIdealState;
+    }
+
+    // One principal is to prohibit DROP -> OFFLINE and OFFLINE -> DROP state transitions.
+    // Derived preference list from current state with state priority
+    for (Partition partition : currentStateMap.keySet()) {
+      Map<String, String> stateMap = currentStateMap.get(partition);
+      List<String> preferenceList = new ArrayList<>(stateMap.keySet());
+      Collections.sort(preferenceList, new PreferenceListNodeComparator(stateMap,
+          clusterData.getStateModelDef(currentIdealState.getStateModelDefRef())));
+      currentIdealState.setPreferenceList(partition.getPartitionName(), preferenceList);
+    }
+    LOG.info("End computing ideal state for resource %s in maintenance mode.");
+    return currentIdealState;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index c43b96c..4fb8cd7 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -26,6 +26,7 @@ import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.AutoRebalancer;
 import org.apache.helix.controller.rebalancer.CustomRebalancer;
+import org.apache.helix.controller.rebalancer.MaintenanceRebalancer;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
@@ -211,7 +212,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       idealState.setStateModelDefRef(resource.getStateModelDefRef());
     }
 
-    Rebalancer rebalancer = getRebalancer(idealState, resourceName);
+    Rebalancer rebalancer =
+        getRebalancer(idealState, resourceName, cache.isMaintenanceModeEnabled());
     MappingCalculator mappingCalculator = getMappingCalculator(rebalancer, resourceName);
 
     if (rebalancer == null || mappingCalculator == null) {
@@ -289,7 +291,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     }
   }
 
-  private Rebalancer getRebalancer(IdealState idealState, String resourceName) {
+  private Rebalancer getRebalancer(IdealState idealState, String resourceName,
+      boolean isMaintenanceModeEnabled) {
     Rebalancer customizedRebalancer = null;
     String rebalancerClassName = idealState.getRebalancerClassName();
     if (rebalancerClassName != null) {
@@ -305,10 +308,14 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     Rebalancer rebalancer = null;
     switch (idealState.getRebalanceMode()) {
     case FULL_AUTO:
-      if (customizedRebalancer != null) {
-        rebalancer = customizedRebalancer;
+      if (isMaintenanceModeEnabled) {
+        rebalancer = new MaintenanceRebalancer();
       } else {
-        rebalancer = new AutoRebalancer();
+        if (customizedRebalancer != null) {
+          rebalancer = customizedRebalancer;
+        } else {
+          rebalancer = new AutoRebalancer();
+        }
       }
       break;
     case SEMI_AUTO:

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index 1dd862d..5b4aa83 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -45,6 +45,7 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ParticipantHistory;
 import org.apache.helix.model.ResourceAssignment;
@@ -111,6 +112,7 @@ public class ClusterDataCache {
 
   boolean _updateInstanceOfflineTime = true;
   boolean _isTaskCache;
+  boolean _isMaintenanceModeEnabled;
 
   private String _clusterName;
 
@@ -204,6 +206,9 @@ public class ClusterDataCache {
       LOG.warn("Cluster config is null!");
     }
 
+    MaintenanceSignal maintenanceSignal = accessor.getProperty(keyBuilder.maintenance());
+    _isMaintenanceModeEnabled = (maintenanceSignal != null) ? true : false;
+
     long endTime = System.currentTimeMillis();
     LOG.info(
         "END: ClusterDataCache.refresh() for cluster " + getClusterName() + ", took " + (endTime
@@ -1008,6 +1013,10 @@ public class ClusterDataCache {
     return _isTaskCache;
   }
 
+  public boolean isMaintenanceModeEnabled() {
+    return _isMaintenanceModeEnabled;
+  }
+
   /**
    * toString method to print the entire cluster state
    */

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index c3fa9e9..1af881e 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -67,6 +67,7 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
@@ -326,7 +327,33 @@ public class ZKHelixAdmin implements HelixAdmin {
       if (reason != null) {
         pauseSignal.setReason(reason);
       }
-      accessor.createPause(pauseSignal);
+      if (!accessor.createPause(pauseSignal)) {
+        throw new HelixException("Failed to create pause signal");
+      }
+    }
+  }
+
+  @Override
+  public void enableMaintenanceMode(String clusterName, boolean enabled) {
+    enableMaintenanceMode(clusterName, enabled, null);
+  }
+
+  @Override
+  public void enableMaintenanceMode(String clusterName, boolean enabled, String reason) {
+    HelixDataAccessor accessor =
+        new ZKHelixDataAccessor(clusterName, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
+    Builder keyBuilder = accessor.keyBuilder();
+
+    if (!enabled) {
+      accessor.removeProperty(keyBuilder.maintenance());
+    } else {
+      MaintenanceSignal maintenanceSignal = new MaintenanceSignal("maintenance");
+      if (reason != null) {
+        maintenanceSignal.setReason(reason);
+      }
+      if (!accessor.createMaintenance(maintenanceSignal)) {
+        throw new HelixException("Failed to create maintenance signal");
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
index c5ce121..eff76f8 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
@@ -43,6 +43,7 @@ import org.apache.helix.ZNRecordAssembler;
 import org.apache.helix.ZNRecordBucketizer;
 import org.apache.helix.ZNRecordUpdater;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.StateModelDefinition;
@@ -113,6 +114,13 @@ public class ZKHelixDataAccessor implements HelixDataAccessor {
   }
 
   @Override
+  public boolean createMaintenance(MaintenanceSignal maintenanceSignal) {
+    return _baseDataAccessor
+        .create(PropertyPathBuilder.maintenance(_clusterName), maintenanceSignal.getRecord(),
+            AccessOption.PERSISTENT);
+  }
+
+  @Override
   public <T extends HelixProperty> boolean setProperty(PropertyKey key, T value) {
     PropertyType type = key.getType();
     if (!value.isValid()) {

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/main/java/org/apache/helix/model/MaintenanceSignal.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/MaintenanceSignal.java b/helix-core/src/main/java/org/apache/helix/model/MaintenanceSignal.java
new file mode 100644
index 0000000..b678738
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/model/MaintenanceSignal.java
@@ -0,0 +1,13 @@
+package org.apache.helix.model;
+
+import org.apache.helix.ZNRecord;
+
+public class MaintenanceSignal extends PauseSignal {
+  public MaintenanceSignal(String id) {
+    super(id);
+  }
+
+  public MaintenanceSignal(ZNRecord record) {
+    super(record);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/test/java/org/apache/helix/MockAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/MockAccessor.java b/helix-core/src/test/java/org/apache/helix/MockAccessor.java
index d41c2d3..583dbd8 100644
--- a/helix-core/src/test/java/org/apache/helix/MockAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/MockAccessor.java
@@ -27,6 +27,7 @@ import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.apache.helix.mock.MockBaseDataAccessor;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.StateModelDefinition;
@@ -67,6 +68,10 @@ public class MockAccessor implements HelixDataAccessor {
     return false;
   }
 
+  @Override public boolean createMaintenance(MaintenanceSignal maintenanceSignal) {
+    return false;
+  }
+
   @Override public boolean setProperty(PropertyKey key, HelixProperty value) {
     String path = key.getPath();
     _baseDataAccessor.set(path, value.getRecord(), AccessOption.PERSISTENT);

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
new file mode 100644
index 0000000..3949183
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/controller/TestClusterMaintenanceMode.java
@@ -0,0 +1,81 @@
+package org.apache.helix.integration.controller;
+
+import java.util.Map;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.integration.task.TaskTestBase;
+import org.apache.helix.integration.task.WorkflowGenerator;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestClusterMaintenanceMode extends TaskTestBase {
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _numDbs = 1;
+    _numNodes = 3;
+    _numReplicas = 3;
+    _numParitions = 5;
+    super.beforeClass();
+  }
+
+  @Test
+  public void testMaintenanceModeAddNewInstance() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool().enableMaintenanceMode(CLUSTER_NAME, true, "Test");
+    Thread.sleep(2000);
+    ExternalView prevExternalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    String instanceName = PARTICIPANT_PREFIX + "_" + (_startPort + 10);
+    _setupTool.addInstanceToCluster(CLUSTER_NAME, instanceName);
+    MockParticipantManager newInstance =
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+    newInstance.syncStart();
+    _gSetupTool.getClusterManagementTool()
+        .rebalance(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB, 3);
+    Thread.sleep(3000);
+    ExternalView newExternalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    Assert.assertEquals(prevExternalView.getRecord().getMapFields(),
+        newExternalView.getRecord().getMapFields());
+  }
+
+  @Test (dependsOnMethods = "testMaintenanceModeAddNewInstance")
+  public void testMaintenanceModeAddNewResource() throws InterruptedException {
+    _gSetupTool.getClusterManagementTool()
+        .addResource(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + 1, 7, "MasterSlave",
+            IdealState.RebalanceMode.FULL_AUTO.name());
+    _gSetupTool.getClusterManagementTool()
+        .rebalance(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + 1, 3);
+    Thread.sleep(2000);
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB + 1);
+    Assert.assertNull(externalView);
+  }
+
+  @Test (dependsOnMethods = "testMaintenanceModeAddNewResource")
+  public void testMaintenanceModeInstanceDown() throws InterruptedException {
+    _participants[0].syncStop();
+    Thread.sleep(2000);
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      Assert.assertTrue(stateMap.values().contains("MASTER"));
+    }
+  }
+
+  @Test (dependsOnMethods = "testMaintenanceModeInstanceDown")
+  public void testMaintenanceModeInstanceBack() throws InterruptedException {
+    _participants[0] =
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, _participants[0].getInstanceName());
+    _participants[0].syncStart();
+    Thread.sleep(2000);
+    ExternalView externalView = _gSetupTool.getClusterManagementTool()
+        .getResourceExternalView(CLUSTER_NAME, WorkflowGenerator.DEFAULT_TGT_DB);
+    for (Map<String, String> stateMap : externalView.getRecord().getMapFields().values()) {
+      if (stateMap.containsKey(_participants[0].getInstanceName())) {
+        Assert.assertTrue(stateMap.get(_participants[0].getInstanceName()).equals("SLAVE"));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/a7477c3b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
index 037d92b..b1d5da7 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
@@ -253,6 +253,14 @@ public class MockHelixAdmin implements HelixAdmin {
 
   }
 
+  @Override public void enableMaintenanceMode(String clusterName, boolean enabled) {
+
+  }
+
+  @Override public void enableMaintenanceMode(String clusterName, boolean enabled, String reason) {
+
+  }
+
   @Override public void resetPartition(String clusterName, String instanceName, String resourceName,
       List<String> partitionNames) {
 


[29/50] [abbrv] helix git commit: HELIX-661: initial impl of multiple zk support in helix rest

Posted by jx...@apache.org.
HELIX-661: initial impl of multiple zk support in helix rest


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

Branch: refs/heads/master
Commit: d9052a303b59a3f1f3f347eb5dff28c1bf1c8113
Parents: 019d6f4
Author: hrzhang <hr...@linkedin.com>
Authored: Thu Dec 7 17:10:17 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:20 2018 -0800

----------------------------------------------------------------------
 helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy       |   3 +
 helix-rest/pom.xml                              |   5 +
 .../helix/rest/common/ContextPropertyKeys.java  |   3 +-
 .../helix/rest/common/HelixRestNamespace.java   |  99 +++++++++++++++
 .../helix/rest/common/HelixRestUtils.java       |  60 +++++++++
 .../apache/helix/rest/server/HelixRestMain.java |  43 ++++++-
 .../helix/rest/server/HelixRestServer.java      | 124 ++++++++++++++++---
 .../helix/rest/server/auditlog/AuditLog.java    |  21 +++-
 .../rest/server/filters/AuditLogFilter.java     |  18 ++-
 .../rest/server/resources/AbstractResource.java |   4 +-
 .../helix/rest/server/AbstractTestClass.java    |  45 ++++++-
 .../helix/rest/server/TestClusterAccessor.java  |   2 +
 .../helix/rest/server/TestHelixRestServer.java  |  85 +++++++++++++
 .../rest/server/TestNamespacedAPIAccess.java    |  84 +++++++++++++
 14 files changed, 563 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
index 5508152..df5262f 100644
--- a/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
+++ b/helix-rest/helix-rest-0.6.10-SNAPSHOT.ivy
@@ -43,6 +43,9 @@ under the License.
     <dependency org="org.slf4j" name="slf4j-log4j12" rev="1.7.14" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)">
         <artifact name="slf4j-log4j12" ext="jar"/>
     </dependency>
+    <dependency org="org.yaml" name="snakeyaml" rev="1.17">
+        <artifact name="snakeyaml" m:classifier="sources" ext="jar"/>
+    </dependency>
 		<dependency org="org.apache.helix" name="helix-core" rev="0.6.10-SNAPSHOT" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.codehaus.jackson" name="jackson-core-asl" rev="1.8.5" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
 		<dependency org="org.codehaus.jackson" name="jackson-mapper-asl" rev="1.8.5" force="true" conf="compile->compile(*),master(*);runtime->runtime(*)"/>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/pom.xml
----------------------------------------------------------------------
diff --git a/helix-rest/pom.xml b/helix-rest/pom.xml
index b27f0b9..32b0731 100644
--- a/helix-rest/pom.xml
+++ b/helix-rest/pom.xml
@@ -43,6 +43,11 @@ under the License.
 
   <dependencies>
     <dependency>
+      <groupId>org.yaml</groupId>
+      <artifactId>snakeyaml</artifactId>
+      <version>1.17</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <version>1.7.25</version>

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
index f2aee51..ce59abc 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/ContextPropertyKeys.java
@@ -20,5 +20,6 @@ package org.apache.helix.rest.common;
  */
 
 public enum ContextPropertyKeys {
-  SERVER_CONTEXT
+  SERVER_CONTEXT,
+  NAMESPACE
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
new file mode 100644
index 0000000..5d1c8f3
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestNamespace.java
@@ -0,0 +1,99 @@
+package org.apache.helix.rest.common;
+
+/*
+ * 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.
+ */
+
+public class HelixRestNamespace {
+
+  public enum HelixMetadataStoreType {
+    ZOOKEEPER
+  }
+
+  public enum HelixRestNamespaceProperty {
+    NAME,
+    METADATA_STORE_TYPE,
+    METADATA_STORE_ADDRESS,
+    IS_DEFAULT
+  }
+
+  /**
+   * Namespaced object will have path such as /url_prefix/namespaces/{namespace_name}/clusters/...
+   * We are going to have path /url_prefix/clusters/... point to default namespace if there is one
+   */
+  public static final String DEFAULT_NAMESPACE_PATH_SPEC = "/*";
+  public static final String DEFAULT_NAMESPACE_NAME = "default";
+
+  /**
+   * Name of Helix namespace
+   */
+  private String _name;
+
+  /**
+   * Type of a metadata store that belongs to Helix namespace
+   */
+  private HelixMetadataStoreType _metadataStoreType;
+
+  /**
+   * Address of metadata store. Should be informat of
+   * "[ip-address]:[port]" or "[dns-name]:[port]"
+   */
+  private String _metadataStoreAddress;
+
+  /**
+   * Flag indicating whether this namespace is default or not
+   */
+  private boolean _isDefault;
+
+  public HelixRestNamespace(String metadataStoreAddress) throws IllegalArgumentException {
+    this(DEFAULT_NAMESPACE_NAME, HelixMetadataStoreType.ZOOKEEPER, metadataStoreAddress, true);
+  }
+
+  public HelixRestNamespace(String name, HelixMetadataStoreType metadataStoreType, String metadataStoreAddress, boolean isDefault)
+      throws IllegalArgumentException {
+    _name = name;
+    _metadataStoreAddress = metadataStoreAddress;
+    _metadataStoreType = metadataStoreType;
+    _isDefault = isDefault;
+    validate();
+  }
+
+  private void validate() throws IllegalArgumentException {
+    // TODO: add more strict validation for NAME as this will be part of URL
+    if (_name == null || _name.length() == 0) {
+      throw new IllegalArgumentException("Name of namespace not provided");
+    }
+    if (_metadataStoreAddress == null || _metadataStoreAddress.isEmpty()) {
+      throw new IllegalArgumentException(
+          String.format("Metadata store address \"%s\" is not valid for namespace %s", _metadataStoreAddress, _name));
+    }
+  }
+
+  public boolean isDefault() {
+    return _isDefault;
+  }
+
+  public String getName() {
+    return _name;
+  }
+
+  public String getMetadataStoreAddress() {
+    return _metadataStoreAddress;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
new file mode 100644
index 0000000..6c4a3df
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/common/HelixRestUtils.java
@@ -0,0 +1,60 @@
+package org.apache.helix.rest.common;
+
+/*
+ * 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.
+ */
+
+public class HelixRestUtils {
+  /**
+   * Generate servlet path spec for a given namespace.
+   * @param namespace Name of the namespace
+   * @param isDefaultServlet mark this as true to get path spec for the special servlet for default namespace
+   * @return servlet path spec
+   */
+  public static String makeServletPathSpec(String namespace, boolean isDefaultServlet) {
+    return isDefaultServlet ? HelixRestNamespace.DEFAULT_NAMESPACE_PATH_SPEC
+        : String.format("/namespaces/%s/*", namespace);
+  }
+
+  /**
+   * Extract namespace information from servlet path. There are 3 cases:
+   *  1. /namespaces/namespaceName  ->  return namespaceName
+   *  2. /namespaces                ->  return ""
+   *  3. this is special servlet for default namespace  ->  return the reserved name for default namespace
+   * @param servletPath servletPath
+   * @return Namespace name retrieved from servlet spec.
+   */
+  public static String getNamespaceFromServletPath(String servletPath) {
+    if (isDefaultNamespaceServlet(servletPath)) {
+      return HelixRestNamespace.DEFAULT_NAMESPACE_NAME;
+    }
+
+    String namespaceName = servletPath.replace("/namespaces", "");
+    if (namespaceName.isEmpty() || namespaceName.equals("/")) {
+      return "";
+    } else {
+      return namespaceName.replace("/", "");
+    }
+  }
+
+  private static boolean isDefaultNamespaceServlet(String servletPath) {
+    // Special servlet for default namespace has path spec "/*", so servletPath is empty
+    return servletPath == null || servletPath.isEmpty();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
index 7368bbd..ac870d0 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestMain.java
@@ -19,7 +19,13 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -29,15 +35,18 @@ import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.helix.HelixException;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.auditlog.auditloggers.FileBasedAuditLogger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.Yaml;
 
 public class HelixRestMain {
   private static Logger LOG = LoggerFactory.getLogger(HelixRestServer.class);
   private static final String HELP = "help";
   private static final String ZKSERVERADDRESS = "zkSvr";
+  private static final String NAMESPACE_MANIFEST_FILE = "namespace-manifest-file";
   private static final String PORT = "port";
   private static final int DEFAULT_PORT = 8100;
   private static final String URI_PREFIX = "/admin/v2";
@@ -62,6 +71,13 @@ public class HelixRestMain {
     zkServerOption.setRequired(true);
     zkServerOption.setArgName("ZookeeperServerAddress(Required)");
 
+    Option helixRestNamespaceOption = OptionBuilder.withLongOpt(NAMESPACE_MANIFEST_FILE)
+        .withDescription("A yaml file describing helix namespace")
+        .create();
+    helixRestNamespaceOption.setArgs(1);
+    helixRestNamespaceOption.setRequired(false);
+    helixRestNamespaceOption.setArgName("NamespaceManifestFile(Optional)");
+
     Option portOption =
         OptionBuilder.withLongOpt(PORT).withDescription("Provide web service port").create();
     portOption.setArgs(1);
@@ -72,11 +88,12 @@ public class HelixRestMain {
     options.addOption(helpOption);
     options.addOption(zkServerOption);
     options.addOption(portOption);
+    options.addOption(helixRestNamespaceOption);
 
     return options;
   }
 
-  public static void processCommandLineArgs(String[] cliArgs) throws Exception {
+  private static void processCommandLineArgs(String[] cliArgs) throws Exception {
     CommandLineParser cliParser = new GnuParser();
     Options cliOptions = constructCommandLineOptions();
     CommandLine cmd = null;
@@ -90,6 +107,7 @@ public class HelixRestMain {
     }
     int port = DEFAULT_PORT;
     String zkAddr;
+    List<HelixRestNamespace> namespaces = new ArrayList<>();
     if (cmd.hasOption(HELP)) {
       printUsage(cliOptions);
       return;
@@ -98,9 +116,13 @@ public class HelixRestMain {
         port = Integer.parseInt(cmd.getOptionValue(PORT));
       }
       zkAddr = String.valueOf(cmd.getOptionValue(ZKSERVERADDRESS));
+      namespaces.add(new HelixRestNamespace(zkAddr));
+      if (cmd.hasOption(NAMESPACE_MANIFEST_FILE)) {
+        constructNamespaceFromConfigFile(String.valueOf(cmd.getOptionValue(NAMESPACE_MANIFEST_FILE)), namespaces);
+      }
     }
 
-    final HelixRestServer restServer = new HelixRestServer(zkAddr, port, URI_PREFIX,
+    final HelixRestServer restServer = new HelixRestServer(namespaces, port, URI_PREFIX,
         Arrays.<AuditLogger>asList(new FileBasedAuditLogger()));
 
     try {
@@ -113,6 +135,23 @@ public class HelixRestMain {
     }
   }
 
+  private static void constructNamespaceFromConfigFile(String filePath, List<HelixRestNamespace> namespaces)
+      throws IOException {
+    Yaml yaml = new Yaml();
+    @SuppressWarnings("unchecked")
+    ArrayList<Map<String, String>> configs =
+        (ArrayList<Map<String, String>>) yaml.load(new FileInputStream(new File(filePath)));
+    for (Map<String, String> config : configs) {
+      // Currently we don't support adding default namespace through yaml manifest so all
+      // namespaces created here will not be default
+      // TODO: support specifying default namespace from config file
+      namespaces.add(new HelixRestNamespace(config.get(HelixRestNamespace.HelixRestNamespaceProperty.NAME.name()),
+          HelixRestNamespace.HelixMetadataStoreType.valueOf(
+              config.get(HelixRestNamespace.HelixRestNamespaceProperty.METADATA_STORE_TYPE.name())),
+          config.get(HelixRestNamespace.HelixRestNamespaceProperty.METADATA_STORE_ADDRESS.name()), false));
+    }
+  }
+
   /**
    * @param args
    * @throws Exception

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
index 20f5b1b..3737308 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/HelixRestServer.java
@@ -19,16 +19,26 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import org.apache.helix.HelixException;
 import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.common.HelixRestUtils;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.filters.AuditLogFilter;
 import org.apache.helix.rest.server.filters.CORSFilter;
 import org.apache.helix.rest.server.resources.AbstractResource;
 import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.*;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
@@ -37,27 +47,87 @@ import org.glassfish.jersey.servlet.ServletContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HelixRestServer extends ResourceConfig {
+public class HelixRestServer {
   private static Logger LOG = LoggerFactory.getLogger(HelixRestServer.class);
 
   private int _port;
   private String _urlPrefix;
   private Server _server;
-  private ServerContext _serverContext;
+  private ServletContextHandler _servletContextHandler;
+  private List<AuditLogger> _auditLoggers;
+
+  // Key is name of namespace, value of the resource config of that namespace
+  private Map<String, ResourceConfig> _resourceConfigMap;
+
+  // In additional to regular servlets serving namespaced API endpoints, We have a default servlet
+  // serving un-namespaced API (/admin/v2/clusters/...) for default namespace as well. We use this
+  // literal as a key in _resourceConfigMap to keep records for default servlet.
+  // TODO: try to find a way to serve 2 sets of endpoints of default namespace in 1 servlet
+  private static final String DEFAULT_SERVLET_KEY = "DefaultServlet";
+
+  public HelixRestServer(String zkAddr, int port, String urlPrefix) {
+    this(zkAddr, port, urlPrefix, Collections.<AuditLogger>emptyList());
+  }
 
   public HelixRestServer(String zkAddr, int port, String urlPrefix, List<AuditLogger> auditLoggers) {
+    // Create default namespace using zkAddr
+    ArrayList<HelixRestNamespace> namespaces = new ArrayList<>();
+    namespaces.add(new HelixRestNamespace(HelixRestNamespace.DEFAULT_NAMESPACE_NAME,
+        HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, zkAddr, true));
+    init(namespaces, port, urlPrefix, auditLoggers);
+  }
+
+  public HelixRestServer(List<HelixRestNamespace> namespaces, int port, String urlPrefix, List<AuditLogger> auditLoggers) {
+    init(namespaces, port, urlPrefix, auditLoggers);
+  }
+
+  private void init(List<HelixRestNamespace> namespaces, int port, String urlPrefix,
+      List<AuditLogger> auditLoggers) {
+    if (namespaces.size() == 0) {
+      throw new IllegalArgumentException(
+          "No namespace specified! Please provide ZOOKEEPER address or namespace manifest.");
+    }
     _port = port;
     _urlPrefix = urlPrefix;
     _server = new Server(_port);
+    _auditLoggers = auditLoggers;
+    _resourceConfigMap = new HashMap<>();
+    _servletContextHandler = new ServletContextHandler(_server, _urlPrefix);
 
-    packages(AbstractResource.class.getPackage().getName());
-
-    _serverContext = new ServerContext(zkAddr);
-    property(ContextPropertyKeys.SERVER_CONTEXT.name(), _serverContext);
-
-    register(new CORSFilter());
-    register(new AuditLogFilter(auditLoggers));
+    // Initialize all namespaces
+    try {
+      for (HelixRestNamespace namespace : namespaces) {
+        LOG.info("Initializing namespace " + namespace.getName());
+        if (_resourceConfigMap.containsKey(namespace.getName())) {
+          throw new IllegalArgumentException(String.format("Duplicated namespace name \"%s\"", namespace.getName()));
+        }
+
+        // Create resource and context for namespaced servlet
+        _resourceConfigMap.put(namespace.getName(),
+            makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
+        LOG.info("Initializing servlet for namespace " + namespace.getName());
+        initServlet(_resourceConfigMap.get(namespace.getName()),
+            HelixRestUtils.makeServletPathSpec(namespace.getName(), false));
+
+        // Create special resource and context for default namespace servlet
+        if (namespace.isDefault()) {
+          if (_resourceConfigMap.containsKey(DEFAULT_SERVLET_KEY)) {
+            throw new IllegalArgumentException("More than 1 default namespaces are provided");
+          }
+          LOG.info("Creating special servlet for default namespace");
+          _resourceConfigMap.put(DEFAULT_SERVLET_KEY,
+              makeResourceConfig(namespace, AbstractResource.class.getPackage().getName()));
+          initServlet(_resourceConfigMap.get(DEFAULT_SERVLET_KEY),
+              HelixRestUtils.makeServletPathSpec(namespace.getName(), true));
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("Failed to initialize helix rest server. Tearing down.");
+      cleanupResourceConfigs();
+      throw e;
+    }
 
+    // Start special servlet for serving namespaces
     Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
       @Override public void run() {
         shutdown();
@@ -65,15 +135,21 @@ public class HelixRestServer extends ResourceConfig {
     }));
   }
 
-  public HelixRestServer(String zkAddr, int port, String urlPrefix) {
-    this(zkAddr, port, urlPrefix, Collections.<AuditLogger>emptyList());
+  private ResourceConfig makeResourceConfig(HelixRestNamespace ns, String... packages) {
+    ResourceConfig cfg = new ResourceConfig();
+    cfg.packages(packages)
+        .property(ContextPropertyKeys.SERVER_CONTEXT.name(), new ServerContext(ns.getMetadataStoreAddress()))
+        .register(new CORSFilter())
+        .register(new AuditLogFilter(_auditLoggers));
+    return cfg;
   }
 
-  public void start() throws HelixException, InterruptedException {
-    ServletHolder servlet = new ServletHolder(new ServletContainer(this));
-    ServletContextHandler contextHandler = new ServletContextHandler(_server, _urlPrefix);
-    contextHandler.addServlet(servlet, "/*");
+  private void initServlet(ResourceConfig cfg, String servletPathSpec) {
+    ServletHolder servlet = new ServletHolder(new ServletContainer(cfg));
+    _servletContextHandler.addServlet(servlet, servletPathSpec);
+  }
 
+  public void start() throws HelixException, InterruptedException {
     try {
       _server.start();
     } catch (Exception ex) {
@@ -103,9 +179,19 @@ public class HelixRestServer extends ResourceConfig {
         LOG.error("Failed to stop Helix rest server, " + ex);
       }
     }
-    ServerContext serverContext =
-        (ServerContext) getProperty(ContextPropertyKeys.SERVER_CONTEXT.name());
-    serverContext.close();
+    cleanupResourceConfigs();
+  }
+
+  private void cleanupResourceConfigs() {
+    for (Map.Entry<String, ResourceConfig> e : _resourceConfigMap.entrySet()) {
+      ServerContext ctx = (ServerContext) e.getValue().getProperty(ContextPropertyKeys.SERVER_CONTEXT.name());
+      if (ctx == null) {
+        LOG.warn("Server context for servlet " + e.getKey() + " is null.");
+      } else {
+        LOG.info("Closing context for servlet " + e.getKey());
+        ctx.close();
+      }
+    }
   }
 
   public void setupSslServer(int port, SslContextFactory sslContextFactory) {

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java b/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
index c857e6d..2afa4ba 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/auditlog/AuditLog.java
@@ -35,6 +35,7 @@ public class AuditLog {
   private Principal _principal;
   private String _clientIP;
   private String _clientHostPort;
+  private String _namespace;
   private String _requestPath;
   private String _httpMethod;
   private List<String> _requestHeaders;
@@ -46,7 +47,7 @@ public class AuditLog {
   private String _additionalInfo;
 
   public AuditLog(Date startTime, Date completeTime, Principal principal, String clientIP,
-      String clientHostPort, String requestPath, String httpMethod, List<String> requestHeaders,
+      String clientHostPort, String namespace, String requestPath, String httpMethod, List<String> requestHeaders,
       String requestEntity, int responseCode, String responseEntity,
       String additionalInfo, List<Exception> exceptions) {
     _startTime = startTime;
@@ -54,6 +55,7 @@ public class AuditLog {
     _principal = principal;
     _clientIP = clientIP;
     _clientHostPort = clientHostPort;
+    _namespace = namespace;
     _requestPath = requestPath;
     _httpMethod = httpMethod;
     _requestHeaders = requestHeaders;
@@ -72,6 +74,7 @@ public class AuditLog {
         ", _principal=" + _principal +
         ", _clientIP='" + _clientIP + '\'' +
         ", _clientHostPort='" + _clientHostPort + '\'' +
+        ", _namespace='" + _namespace + '\'' +
         ", _requestPath='" + _requestPath + '\'' +
         ", _httpMethod='" + _httpMethod + '\'' +
         ", _requestHeaders=" + _requestHeaders +
@@ -103,6 +106,10 @@ public class AuditLog {
     return _clientHostPort;
   }
 
+  public String getNamespace() {
+    return _namespace;
+  }
+
   public String getRequestPath() {
     return _requestPath;
   }
@@ -141,6 +148,7 @@ public class AuditLog {
     private Principal _principal;
     private String _clientIP;
     private String _clientHostPort;
+    private String _namespace;
     private String _requestPath;
     private String _httpMethod;
     private List<String> _requestHeaders;
@@ -196,6 +204,15 @@ public class AuditLog {
       return this;
     }
 
+    public String getNamespace() {
+      return _namespace;
+    }
+
+    public Builder namespace(String namespace) {
+      _namespace = namespace;
+      return this;
+    }
+
     public String getRequestPath() {
       return _requestPath;
     }
@@ -278,7 +295,7 @@ public class AuditLog {
 
     public AuditLog build() {
       return new AuditLog(_startTime, _completeTime, _principal, _clientIP, _clientHostPort,
-          _requestPath, _httpMethod, _requestHeaders, _requestEntity, _responseCode,
+          _namespace, _requestPath, _httpMethod, _requestHeaders, _requestEntity, _responseCode,
           _responseEntity, _additionalInfo, _exceptions);
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java b/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
index 235b0bb..646e3d3 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/filters/AuditLogFilter.java
@@ -37,6 +37,7 @@ import javax.ws.rs.container.PreMatching;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MultivaluedMap;
 import javax.ws.rs.ext.Provider;
+import org.apache.helix.rest.common.HelixRestUtils;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.slf4j.Logger;
@@ -59,9 +60,14 @@ public class AuditLogFilter implements ContainerRequestFilter, ContainerResponse
   @Override
   public void filter(ContainerRequestContext request) throws IOException {
     AuditLog.Builder auditLogBuilder = new AuditLog.Builder();
-    auditLogBuilder.requestPath(request.getUriInfo().getPath()).httpMethod(request.getMethod())
-        .startTime(new Date()).requestHeaders(getHeaders(request.getHeaders()))
-        .principal(_servletRequest.getUserPrincipal()).clientIP(_servletRequest.getRemoteAddr())
+
+    auditLogBuilder.namespace(getNamespace())
+        .requestPath(request.getUriInfo().getPath())
+        .httpMethod(request.getMethod())
+        .startTime(new Date())
+        .requestHeaders(getHeaders(request.getHeaders()))
+        .principal(_servletRequest.getUserPrincipal())
+        .clientIP(_servletRequest.getRemoteAddr())
         .clientHostPort(_servletRequest.getRemoteHost() + ":" + _servletRequest.getRemotePort());
 
     String entity = getEntity(request.getEntityStream());
@@ -115,4 +121,10 @@ public class AuditLogFilter implements ContainerRequestFilter, ContainerResponse
     }
     return null;
   }
+
+  private String getNamespace() {
+    String servletPath = _servletRequest.getServletPath();
+    return HelixRestUtils.getNamespaceFromServletPath(servletPath);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index 50f6f08..70f713a 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -41,10 +41,10 @@ import org.apache.helix.rest.server.ServerContext;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @Produces({MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN})
 @Consumes({MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN})

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
index 29b198e..8b86c17 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
@@ -33,7 +33,6 @@ import javax.ws.rs.client.Entity;
 import javax.ws.rs.client.WebTarget;
 import javax.ws.rs.core.Application;
 import javax.ws.rs.core.Response;
-
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
@@ -48,6 +47,7 @@ import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.auditlog.AuditLogger;
 import org.apache.helix.rest.server.filters.AuditLogFilter;
@@ -88,6 +88,13 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
   protected static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   protected static boolean _init = false;
 
+  // For testing namespaced access
+  protected static ZkServer _zkServerTestNS;
+  protected static final String _zkAddrTestNS = "localhost:2124";
+  protected static final String TEST_NAMESPACE = "test-namespace";
+  protected static ZkClient _gZkClientTestNS;
+  protected static BaseDataAccessor<ZNRecord> _baseAccessorTestNS;
+
   protected static Set<String> _clusters;
   protected static String _superCluster = "superCluster";
   protected static Map<String, Set<String>> _instancesMap = new HashMap<>();
@@ -123,6 +130,12 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
         Assert.assertTrue(_zkServer != null);
         ZKClientPool.reset();
       }
+
+      if (_zkServerTestNS == null) {
+        _zkServerTestNS = TestHelper.startZkServer(_zkAddrTestNS);
+        Assert.assertTrue(_zkServerTestNS != null);
+        ZKClientPool.reset();
+      }
     } catch (Exception e) {
       Assert.assertTrue(false, String.format("Failed to start ZK server: %s", e.toString()));
     }
@@ -157,15 +170,21 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
 
           @Override
           public void start() {
+            // Create namespace manifest map
+            List<HelixRestNamespace> namespaces = new ArrayList<>();
+            // Add test namespace
+            namespaces.add(new HelixRestNamespace(TEST_NAMESPACE, HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER,
+                _zkAddrTestNS, false));
+            // Add default namesapce
+            namespaces.add(new HelixRestNamespace(ZK_ADDR));
             try {
-              _helixRestServer = new HelixRestServer(ZK_ADDR, baseUri.getPort(), baseUri.getPath(),
+              _helixRestServer = new HelixRestServer(namespaces, baseUri.getPort(), baseUri.getPath(),
                   Arrays.<AuditLogger>asList(_auditLogger));
               _helixRestServer.start();
             } catch (Exception ex) {
               throw new TestContainerException(ex);
             }
           }
-
           @Override
           public void stop() {
             _helixRestServer.shutdown();
@@ -184,9 +203,12 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
 
       _gZkClient = new ZkClient(ZK_ADDR, ZkClient.DEFAULT_CONNECTION_TIMEOUT,
           ZkClient.DEFAULT_SESSION_TIMEOUT, new ZNRecordSerializer());
+      _gZkClientTestNS = new ZkClient(_zkAddrTestNS, ZkClient.DEFAULT_CONNECTION_TIMEOUT, ZkClient.DEFAULT_SESSION_TIMEOUT,
+          new ZNRecordSerializer());
       _gSetupTool = new ClusterSetup(_gZkClient);
       _configAccessor = new ConfigAccessor(_gZkClient);
       _baseAccessor = new ZkBaseDataAccessor<>(_gZkClient);
+      _baseAccessorTestNS = new ZkBaseDataAccessor<>(_gZkClientTestNS);
 
       // wait for the web service to start
       Thread.sleep(100);
@@ -208,6 +230,15 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
       TestHelper.stopZkServer(_zkServer);
       _zkServer = null;
     }
+
+    if (_gZkClientTestNS != null) {
+      _gZkClientTestNS.close();
+      _gZkClientTestNS = null;
+    }
+    if (_zkServerTestNS != null) {
+      TestHelper.stopZkServer(_zkServerTestNS);
+      _zkServerTestNS = null;
+    }
   }
 
   protected void setup() throws Exception {
@@ -342,7 +373,13 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
   protected String get(String uri, int expectedReturnStatus, boolean expectBodyReturned) {
     final Response response = target(uri).request().get();
     Assert.assertEquals(response.getStatus(), expectedReturnStatus);
-    Assert.assertEquals(response.getMediaType().getType(), "application");
+
+    // NOT_FOUND will throw text based html
+    if (expectedReturnStatus != Response.Status.NOT_FOUND.getStatusCode()) {
+      Assert.assertEquals(response.getMediaType().getType(), "application");
+    } else {
+      Assert.assertEquals(response.getMediaType().getType(), "text");
+    }
 
     String body = response.readEntity(String.class);
     if (expectBodyReturned) {

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
index b48c01b..94c5f63 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
@@ -39,6 +39,7 @@ import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.resources.AbstractResource.Command;
 import org.apache.helix.rest.server.resources.ClusterAccessor;
@@ -315,6 +316,7 @@ public class TestClusterAccessor extends AbstractTestClass {
     Assert.assertNotNull(auditLog.getClientHostPort());
     Assert.assertNotNull(auditLog.getCompleteTime());
     Assert.assertNotNull(auditLog.getStartTime());
+    Assert.assertEquals(auditLog.getNamespace(), HelixRestNamespace.DEFAULT_NAMESPACE_NAME);
     Assert.assertEquals(auditLog.getRequestPath(), requestPath);
     Assert.assertEquals(auditLog.getResponseCode(), statusCode);
     Assert.assertEquals(auditLog.getResponseEntity(), responseEntity);

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
new file mode 100644
index 0000000..e213bd3
--- /dev/null
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
@@ -0,0 +1,85 @@
+package org.apache.helix.rest.server;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.server.auditlog.AuditLogger;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestHelixRestServer extends AbstractTestClass {
+  @Test
+  public void testInvalidHelixRestServerInitialization() {
+    // Namespace manifests has invalid metadata store type should generate failure
+    try {
+      List<HelixRestNamespace> invalidManifest1 = new ArrayList<>();
+      invalidManifest1.add(
+          new HelixRestNamespace("test1", HelixRestNamespace.HelixMetadataStoreType.valueOf("InvalidMetadataStore"),
+              ZK_ADDR, false));
+      HelixRestServer svr = new HelixRestServer(invalidManifest1, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest1 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // Namespace manifests has invalid namespace name shall generate failure
+    try {
+      List<HelixRestNamespace> invalidManifest2 = new ArrayList<>();
+      invalidManifest2.add(
+          new HelixRestNamespace("", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      HelixRestServer svr = new HelixRestServer(invalidManifest2, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest2 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // Duplicated namespace shall cause exception
+    try {
+      List<HelixRestNamespace> invalidManifest3 = new ArrayList<>();
+      invalidManifest3.add(
+          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
+              true));
+      invalidManifest3.add(
+          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
+              false));
+      HelixRestServer svr = new HelixRestServer(invalidManifest3, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest3 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+
+    // More than 1 default namespace shall cause failure
+    try {
+      List<HelixRestNamespace> invalidManifest4 = new ArrayList<>();
+      invalidManifest4.add(
+          new HelixRestNamespace("test4-1", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      invalidManifest4.add(
+          new HelixRestNamespace("test4-2", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
+      HelixRestServer svr = new HelixRestServer(invalidManifest4, 10250, "/", Collections.<AuditLogger>emptyList());
+      Assert.assertFalse(true, "InvalidManifest4 test failed");
+    } catch (IllegalArgumentException e) {
+      // OK
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d9052a30/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
new file mode 100644
index 0000000..e6f036d
--- /dev/null
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
@@ -0,0 +1,84 @@
+package org.apache.helix.rest.server;
+
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestNamespacedAPIAccess extends AbstractTestClass {
+  @Test
+  public void testDefaultNamespaceCompatibility() {
+    String testClusterName1 = "testClusterForDefaultNamespaceCompatibility1";
+    String testClusterName2 = "testClusterForDefaultNamespaceCompatibility2";
+
+    // Create from namespaced API and ensure we can access it from old apis, and vice-versa
+    // Assume other api end points will behave the same way
+    put(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName1), null,
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
+    get(String.format("/clusters/%s", testClusterName1), Response.Status.OK.getStatusCode(), false);
+
+    put(String.format("/clusters/%s", testClusterName2), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME, testClusterName2),
+        Response.Status.OK.getStatusCode(), false);
+  }
+
+
+  @Test
+  public void testNamespacedCRUD() throws IOException {
+    String testClusterName = "testClusterForNamespacedCRUD";
+
+    // Create cluster in test namespace and verify it's only appears in test namespace
+    put(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName), null,
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.CREATED.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.OK.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    // Create cluster with same name in different namespacces
+    put(String.format("/clusters/%s", testClusterName), null, Entity.entity("", MediaType.APPLICATION_JSON_TYPE),
+        Response.Status.CREATED.getStatusCode());
+    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+
+    // Modify cluster in default namespace
+    post(String.format("/clusters/%s", testClusterName), ImmutableMap.of("command", "disable"),
+        Entity.entity("", MediaType.APPLICATION_JSON_TYPE), Response.Status.OK.getStatusCode());
+
+    // Verify the cluster in default namespace is modified, while the one in test namespace is not.
+    PropertyKey.Builder keyBuilder = new PropertyKey.Builder(testClusterName);
+    Assert.assertTrue(_baseAccessor.exists(keyBuilder.pause().getPath(), 0));
+    Assert.assertFalse(_baseAccessorTestNS.exists(keyBuilder.pause().getPath(), 0));
+
+    // Verify that deleting cluster in one namespace will not affect the other
+    delete(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.OK.getStatusCode());
+    get(String.format("/namespaces/%s/clusters/%s", TEST_NAMESPACE, testClusterName),
+        Response.Status.NOT_FOUND.getStatusCode(), false);
+    get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
+  }
+
+}


[19/50] [abbrv] helix git commit: Fix two tests for maintenance mode

Posted by jx...@apache.org.
Fix two tests for maintenance mode


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

Branch: refs/heads/master
Commit: de2e3e8be366984461c0b43c8dc6a28048c66157
Parents: 3cfe785
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Nov 16 13:26:06 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:40 2018 -0800

----------------------------------------------------------------------
 ...MaintenanceModeWhenReachingMaxPartition.java | 140 ++++++++++++
 ...ceModeWhenReachingOfflineInstancesLimit.java | 211 +++++++++++++++++++
 ...estPauseClusterWhenReachingMaxPartition.java | 139 ------------
 ...lusterWhenReachingOfflineInstancesLimit.java | 208 ------------------
 4 files changed, 351 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/de2e3e8b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
new file mode 100644
index 0000000..58036f0
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingMaxPartition.java
@@ -0,0 +1,140 @@
+package org.apache.helix.integration.rebalancer;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.PauseSignal;
+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.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestClusterInMaintenanceModeWhenReachingMaxPartition extends ZkIntegrationTestBase {
+  final int NUM_NODE = 5;
+  protected static final int START_PORT = 12918;
+  protected static final int _PARTITIONS = 5;
+
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  protected ClusterControllerManager _controller;
+
+  List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  int _replica = 3;
+  HelixClusterVerifier _clusterVerifier;
+  List<String> _testDBs = new ArrayList<String>();
+  HelixDataAccessor _dataAccessor;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+
+      // start dummy participants
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    _clusterVerifier =
+        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+  }
+
+  protected String[] TestStateModels = {
+      BuiltInStateModelDefinitions.MasterSlave.name(),
+      BuiltInStateModelDefinitions.OnlineOffline.name(),
+      BuiltInStateModelDefinitions.LeaderStandby.name()
+  };
+
+  @Test
+  public void testDisableCluster() throws Exception {
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+    clusterConfig.setMaxPartitionsPerInstance(10);
+    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+          _replica, -1);
+      _testDBs.add(db);
+    }
+    Thread.sleep(100);
+    Assert.assertTrue(_clusterVerifier.verify());
+
+    MaintenanceSignal maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNull(maintenanceSignal);
+
+    for (i = 2; i < NUM_NODE; i++) {
+      _participants.get(i).syncStop();
+    }
+
+    Thread.sleep(500);
+    maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNotNull(maintenanceSignal);
+    Assert.assertNotNull(maintenanceSignal.getReason());
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    /**
+     * shutdown order: 1) disconnect the controller 2) disconnect participants
+     */
+    _controller.syncStop();
+    for (MockParticipantManager participant : _participants) {
+      if (participant.isConnected()) {
+        participant.syncStop();
+      }
+    }
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/de2e3e8b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
new file mode 100644
index 0000000..e03ee14
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit.java
@@ -0,0 +1,211 @@
+package org.apache.helix.integration.rebalancer;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.integration.common.ZkIntegrationTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZKHelixAdmin;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
+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.BeforeClass;
+import org.testng.annotations.Test;
+
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import static org.apache.helix.monitoring.mbeans.ClusterStatusMonitor.CLUSTER_DN_KEY;
+import static org.apache.helix.util.StatusUpdateUtil.ErrorType.RebalanceResourceFailure;
+
+public class TestClusterInMaintenanceModeWhenReachingOfflineInstancesLimit
+    extends ZkIntegrationTestBase {
+  static final int NUM_NODE = 10;
+  static final int START_PORT = 12918;
+  static final int _PARTITIONS = 5;
+  private static final MBeanServerConnection _server = ManagementFactory.getPlatformMBeanServer();
+
+  final String CLASS_NAME = getShortClassName();
+  final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  private ClusterControllerManager _controller;
+
+  private List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
+  private HelixClusterVerifier _clusterVerifier;
+  private HelixDataAccessor _dataAccessor;
+  private int _maxOfflineInstancesAllowed = 4;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+
+    String namespace = "/" + CLUSTER_NAME;
+    if (_gZkClient.exists(namespace)) {
+      _gZkClient.deleteRecursive(namespace);
+    }
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < NUM_NODE; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instanceName);
+
+      // start dummy participants
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    _clusterVerifier =
+        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+    clusterConfig.setMaxOfflineInstancesAllowed(_maxOfflineInstancesAllowed);
+    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+    for (int i = 0; i < 3; i++) {
+      String db = "Test-DB-" + i++;
+      createResourceWithDelayedRebalance(CLUSTER_NAME, db,
+          BuiltInStateModelDefinitions.MasterSlave.name(), _PARTITIONS, 3, 3, -1);
+    }
+    Thread.sleep(100);
+    Assert.assertTrue(_clusterVerifier.verify());
+  }
+
+  @Test
+  public void testWithDisabledInstancesLimit() throws Exception {
+    MaintenanceSignal maintenanceSignal =
+        _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNull(maintenanceSignal);
+
+    HelixAdmin admin = new ZKHelixAdmin(_gZkClient);
+
+    // disable instance
+    int i;
+    for (i = 2; i < 2 + _maxOfflineInstancesAllowed; i++) {
+      String instance = _participants.get(i).getInstanceName();
+      admin.enableInstance(CLUSTER_NAME, instance, false);
+    }
+
+    Thread.sleep(500);
+
+    maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNull(maintenanceSignal);
+
+    String instance = _participants.get(i).getInstanceName();
+    admin.enableInstance(CLUSTER_NAME, instance, false);
+
+    Thread.sleep(500);
+    maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNotNull(maintenanceSignal);
+    Assert.assertNotNull(maintenanceSignal.getReason());
+
+    for (i = 2; i < 2 + _maxOfflineInstancesAllowed + 1; i++) {
+      instance = _participants.get(i).getInstanceName();
+      admin.enableInstance(CLUSTER_NAME, instance, true);
+    }
+    admin.enableMaintenanceMode(CLUSTER_NAME, false);
+  }
+
+
+  @Test (dependsOnMethods = "testWithDisabledInstancesLimit")
+  public void testWithOfflineInstancesLimit() throws Exception {
+    MaintenanceSignal maintenanceSignal =
+        _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNull(maintenanceSignal);
+    int i;
+    for (i = 2; i < 2 + _maxOfflineInstancesAllowed; i++) {
+      _participants.get(i).syncStop();
+    }
+
+    Thread.sleep(500);
+
+    maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNull(maintenanceSignal);
+
+    _participants.get(i).syncStop();
+
+    Thread.sleep(500);
+    maintenanceSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().maintenance());
+    Assert.assertNotNull(maintenanceSignal);
+    Assert.assertNotNull(maintenanceSignal.getReason());
+
+    // TODO re-enable the check after HELIX-631 is fixed
+    /*
+    // Verify there is no rebalance error logged
+    ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+    PropertyKey errorNodeKey =
+        accessor.keyBuilder().controllerTaskError(RebalanceResourceFailure.name());
+    Assert.assertNotNull(accessor.getProperty(errorNodeKey));
+
+    Long value =
+        (Long) _server.getAttribute(getMbeanName(CLUSTER_NAME), "RebalanceFailureGauge");
+    Assert.assertNotNull(value);
+    Assert.assertTrue(value.longValue() > 0);
+    */
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    /**
+     * shutdown order: 1) disconnect the controller 2) disconnect participants
+     */
+    _controller.syncStop();
+    for (MockParticipantManager participant : _participants) {
+      if (participant.isConnected()) {
+        participant.syncStop();
+      }
+    }
+    _gSetupTool.deleteCluster(CLUSTER_NAME);
+    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+  }
+
+  private ObjectName getMbeanName(String clusterName)
+      throws MalformedObjectNameException {
+    String clusterBeanName =
+        String.format("%s=%s", CLUSTER_DN_KEY, clusterName);
+    return new ObjectName(
+        String.format("%s:%s", MonitorDomainNames.ClusterStatus.name(), clusterBeanName));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/de2e3e8b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingMaxPartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingMaxPartition.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingMaxPartition.java
deleted file mode 100644
index 454edcf..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingMaxPartition.java
+++ /dev/null
@@ -1,139 +0,0 @@
-package org.apache.helix.integration.rebalancer;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.model.BuiltInStateModelDefinitions;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.PauseSignal;
-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.BeforeClass;
-import org.testng.annotations.Test;
-
-public class TestPauseClusterWhenReachingMaxPartition extends ZkIntegrationTestBase {
-  final int NUM_NODE = 5;
-  protected static final int START_PORT = 12918;
-  protected static final int _PARTITIONS = 5;
-
-  protected final String CLASS_NAME = getShortClassName();
-  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
-  protected ClusterControllerManager _controller;
-
-  List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
-  int _replica = 3;
-  HelixClusterVerifier _clusterVerifier;
-  List<String> _testDBs = new ArrayList<String>();
-  HelixDataAccessor _dataAccessor;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
-
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-    _gSetupTool.addCluster(CLUSTER_NAME, true);
-
-    for (int i = 0; i < NUM_NODE; i++) {
-      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
-
-      // start dummy participants
-      MockParticipantManager participant =
-          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, storageNodeName);
-      participant.syncStart();
-      _participants.add(participant);
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
-
-    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
-    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
-  }
-
-  protected String[] TestStateModels = {
-      BuiltInStateModelDefinitions.MasterSlave.name(),
-      BuiltInStateModelDefinitions.OnlineOffline.name(),
-      BuiltInStateModelDefinitions.LeaderStandby.name()
-  };
-
-  @Test
-  public void testDisableCluster() throws Exception {
-    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
-    clusterConfig.setMaxPartitionsPerInstance(10);
-    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
-
-    int i = 0;
-    for (String stateModel : TestStateModels) {
-      String db = "Test-DB-" + i++;
-      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
-          _replica, -1);
-      _testDBs.add(db);
-    }
-    Thread.sleep(100);
-    Assert.assertTrue(_clusterVerifier.verify());
-
-    PauseSignal pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNull(pauseSignal);
-
-    for (i = 2; i < NUM_NODE; i++) {
-      _participants.get(i).syncStop();
-    }
-
-    Thread.sleep(500);
-    pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNotNull(pauseSignal);
-    Assert.assertNotNull(pauseSignal.getReason());
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    /**
-     * shutdown order: 1) disconnect the controller 2) disconnect participants
-     */
-    _controller.syncStop();
-    for (MockParticipantManager participant : _participants) {
-      if (participant.isConnected()) {
-        participant.syncStop();
-      }
-    }
-    _gSetupTool.deleteCluster(CLUSTER_NAME);
-    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/helix/blob/de2e3e8b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
deleted file mode 100644
index 9c2c434..0000000
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestPauseClusterWhenReachingOfflineInstancesLimit.java
+++ /dev/null
@@ -1,208 +0,0 @@
-package org.apache.helix.integration.rebalancer;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.integration.common.ZkIntegrationTestBase;
-import org.apache.helix.integration.manager.ClusterControllerManager;
-import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixAdmin;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
-import org.apache.helix.model.BuiltInStateModelDefinitions;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.PauseSignal;
-import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
-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.BeforeClass;
-import org.testng.annotations.Test;
-
-import javax.management.MBeanServerConnection;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
-import static org.apache.helix.monitoring.mbeans.ClusterStatusMonitor.CLUSTER_DN_KEY;
-import static org.apache.helix.util.StatusUpdateUtil.ErrorType.RebalanceResourceFailure;
-
-public class TestPauseClusterWhenReachingOfflineInstancesLimit extends ZkIntegrationTestBase {
-  static final int NUM_NODE = 10;
-  static final int START_PORT = 12918;
-  static final int _PARTITIONS = 5;
-  private static final MBeanServerConnection _server = ManagementFactory.getPlatformMBeanServer();
-
-  final String CLASS_NAME = getShortClassName();
-  final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
-  private ClusterControllerManager _controller;
-
-  private List<MockParticipantManager> _participants = new ArrayList<MockParticipantManager>();
-  private HelixClusterVerifier _clusterVerifier;
-  private HelixDataAccessor _dataAccessor;
-  private int _maxOfflineInstancesAllowed = 4;
-
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
-
-    String namespace = "/" + CLUSTER_NAME;
-    if (_gZkClient.exists(namespace)) {
-      _gZkClient.deleteRecursive(namespace);
-    }
-    _gSetupTool.addCluster(CLUSTER_NAME, true);
-
-    for (int i = 0; i < NUM_NODE; i++) {
-      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
-      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instanceName);
-
-      // start dummy participants
-      MockParticipantManager participant =
-          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
-      participant.syncStart();
-      _participants.add(participant);
-    }
-
-    // start controller
-    String controllerName = CONTROLLER_PREFIX + "_0";
-    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
-    _controller.syncStart();
-
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
-
-    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
-    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
-
-    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
-    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
-    clusterConfig.setMaxOfflineInstancesAllowed(_maxOfflineInstancesAllowed);
-    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
-
-    for (int i = 0; i < 3; i++) {
-      String db = "Test-DB-" + i++;
-      createResourceWithDelayedRebalance(CLUSTER_NAME, db,
-          BuiltInStateModelDefinitions.MasterSlave.name(), _PARTITIONS, 3, 3, -1);
-    }
-    Thread.sleep(100);
-    Assert.assertTrue(_clusterVerifier.verify());
-  }
-
-  @Test
-  public void testWithDisabledInstancesLimit() throws Exception {
-    PauseSignal pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNull(pauseSignal);
-
-    HelixAdmin admin = new ZKHelixAdmin(_gZkClient);
-
-    // disable instance
-    int i;
-    for (i = 2; i < 2 + _maxOfflineInstancesAllowed; i++) {
-      String instance = _participants.get(i).getInstanceName();
-      admin.enableInstance(CLUSTER_NAME, instance, false);
-    }
-
-    Thread.sleep(500);
-
-    pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNull(pauseSignal);
-
-    String instance = _participants.get(i).getInstanceName();
-    admin.enableInstance(CLUSTER_NAME, instance, false);
-
-    Thread.sleep(500);
-    pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNotNull(pauseSignal);
-    Assert.assertNotNull(pauseSignal.getReason());
-
-    for (i = 2; i < 2 + _maxOfflineInstancesAllowed + 1; i++) {
-      instance = _participants.get(i).getInstanceName();
-      admin.enableInstance(CLUSTER_NAME, instance, true);
-    }
-    admin.enableCluster(CLUSTER_NAME, true);
-  }
-
-
-  @Test (dependsOnMethods = "testWithDisabledInstancesLimit")
-  public void testWithOfflineInstancesLimit() throws Exception {
-    PauseSignal pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNull(pauseSignal);
-    int i;
-    for (i = 2; i < 2 + _maxOfflineInstancesAllowed; i++) {
-      _participants.get(i).syncStop();
-    }
-
-    Thread.sleep(500);
-
-    pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNull(pauseSignal);
-
-    _participants.get(i).syncStop();
-
-    Thread.sleep(500);
-    pauseSignal = _dataAccessor.getProperty(_dataAccessor.keyBuilder().pause());
-    Assert.assertNotNull(pauseSignal);
-    Assert.assertNotNull(pauseSignal.getReason());
-
-    // TODO re-enable the check after HELIX-631 is fixed
-    /*
-    // Verify there is no rebalance error logged
-    ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
-    PropertyKey errorNodeKey =
-        accessor.keyBuilder().controllerTaskError(RebalanceResourceFailure.name());
-    Assert.assertNotNull(accessor.getProperty(errorNodeKey));
-
-    Long value =
-        (Long) _server.getAttribute(getMbeanName(CLUSTER_NAME), "RebalanceFailureGauge");
-    Assert.assertNotNull(value);
-    Assert.assertTrue(value.longValue() > 0);
-    */
-  }
-
-  @AfterClass
-  public void afterClass() throws Exception {
-    /**
-     * shutdown order: 1) disconnect the controller 2) disconnect participants
-     */
-    _controller.syncStop();
-    for (MockParticipantManager participant : _participants) {
-      if (participant.isConnected()) {
-        participant.syncStop();
-      }
-    }
-    _gSetupTool.deleteCluster(CLUSTER_NAME);
-    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
-  }
-
-  private ObjectName getMbeanName(String clusterName)
-      throws MalformedObjectNameException {
-    String clusterBeanName =
-        String.format("%s=%s", CLUSTER_DN_KEY, clusterName);
-    return new ObjectName(
-        String.format("%s:%s", MonitorDomainNames.ClusterStatus.name(), clusterBeanName));
-  }
-
-}


[08/50] [abbrv] helix git commit: Cleanup ZkHelixManager and ZkClient config items to remove the ambiguous ones.

Posted by jx...@apache.org.
Cleanup ZkHelixManager and ZkClient config items to remove the ambiguous ones.

We notice that several items are using default values from different classes. And some default items are not set properly. Fix the problem to prevent any future config issues.
Also add comments to clear the usage of each settings.


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

Branch: refs/heads/master
Commit: 2f791a688ae5aafa27e40d7e984a4fd7ce480499
Parents: 401ada6
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Thu Nov 9 11:22:18 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:30:55 2018 -0800

----------------------------------------------------------------------
 .../apache/helix/manager/zk/ZKHelixManager.java | 68 ++++++++------------
 .../org/apache/helix/manager/zk/ZkClient.java   | 34 ++++------
 2 files changed, 39 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/2f791a68/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 430c56b..60b9a3f 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -19,43 +19,21 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
-import javax.management.JMException;
-
 import org.I0Itec.zkclient.IZkStateListener;
 import org.I0Itec.zkclient.ZkConnection;
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.ClusterMessagingService;
-import org.apache.helix.ConfigAccessor;
-import org.apache.helix.api.listeners.ClusterConfigChangeListener;
+import org.apache.helix.*;
+import org.apache.helix.HelixConstants.ChangeType;
+import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.api.listeners.*;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.ControllerChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
-import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixConstants.ChangeType;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixManager;
-import org.apache.helix.HelixManagerProperties;
-import org.apache.helix.HelixTimerTask;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
-import org.apache.helix.InstanceType;
 import org.apache.helix.api.listeners.LiveInstanceChangeListener;
-import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.api.listeners.MessageListener;
-import org.apache.helix.PreConnectCallback;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.PropertyPathBuilder;
-import org.apache.helix.PropertyType;
-import org.apache.helix.api.listeners.ResourceConfigChangeListener;
 import org.apache.helix.api.listeners.ScopedConfigChangeListener;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.GenericHelixController;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
@@ -70,11 +48,17 @@ import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.store.zk.AutoFallbackPropertyStore;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooKeeper.States;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.JMException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 
 public class ZKHelixManager implements HelixManager, IZkStateListener {
@@ -90,10 +74,10 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   private final String _clusterName;
   private final String _instanceName;
   private final InstanceType _instanceType;
-  private final int _sessionTimeout;
-  private final int _clientConnectionTimeout;
-  private final int _connectionRetryTimeout;
-  private final int _waitForConnectedTimeout;
+  private final int _waitForConnectedTimeout; // wait time for testing connect
+  private final int _sessionTimeout; // client side session timeout, will be overridden by server timeout. Disconnect after timeout
+  private final int _connectionInitTimeout; // client timeout to init connect
+  private final int _connectionRetryTimeout; // retry when connect being re-established
   private final List<PreConnectCallback> _preConnectCallbacks;
   protected final List<CallbackHandler> _handlers;
   private final HelixManagerProperties _properties;
@@ -241,7 +225,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
 
     _sessionTimeout = getSystemPropertyAsInt("zk.session.timeout", ZkClient.DEFAULT_SESSION_TIMEOUT);
 
-    _clientConnectionTimeout = getSystemPropertyAsInt("zk.connection.timeout", ZkClient.DEFAULT_CONNECTION_TIMEOUT);
+    _connectionInitTimeout = getSystemPropertyAsInt("zk.connection.timeout", ZkClient.DEFAULT_CONNECTION_TIMEOUT);
 
     _connectionRetryTimeout = getSystemPropertyAsInt("zk.connectionReEstablishment.timeout",
         DEFAULT_CONNECTION_ESTABLISHMENT_RETRY_TIMEOUT);
@@ -309,7 +293,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
         + _clusterName + " by instance: " + _instanceName);
 
     synchronized (this) {
-      List<CallbackHandler> toRemove = new ArrayList<CallbackHandler>();
+      List<CallbackHandler> toRemove = new ArrayList<>();
       for (CallbackHandler handler : _handlers) {
         // compare property-key path and listener reference
         if (handler.getPath().equals(key.getPath()) && handler.getListener().equals(listener)) {
@@ -346,13 +330,15 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
 
     boolean isConnected = isConnected();
     if (!isConnected && timeout > 0) {
-      LOG.warn("zkClient to " + _zkAddress + " is not connected, wait for " + timeout + "ms.");
-      isConnected = _zkclient.waitUntilConnected(timeout, TimeUnit.MILLISECONDS);
+      LOG.warn(
+          "zkClient to " + _zkAddress + " is not connected, wait for " + _waitForConnectedTimeout
+              + "ms.");
+      isConnected = _zkclient.waitUntilConnected(_waitForConnectedTimeout, TimeUnit.MILLISECONDS);
     }
 
     if (!isConnected) {
-      LOG.error("zkClient is not connected after waiting " +
-          timeout + "ms." + ", clusterName: " + _clusterName + ", zkAddress: " + _zkAddress);
+      LOG.error("zkClient is not connected after waiting " + timeout + "ms."
+          + ", clusterName: " + _clusterName + ", zkAddress: " + _zkAddress);
       throw new HelixException(
           "HelixManager is not connected within retry timeout for cluster " + _clusterName);
     }
@@ -589,7 +575,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   }
 
   BaseDataAccessor<ZNRecord> createBaseDataAccessor() {
-    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<ZNRecord>(_zkclient);
+    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<>(_zkclient);
 
     return baseDataAccessor;
   }
@@ -610,7 +596,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
 
     ZkClient.Builder zkClientBuilder = new ZkClient.Builder();
     zkClientBuilder.setZkServer(_zkAddress).setSessionTimeout(_sessionTimeout)
-        .setConnectionTimeout(_clientConnectionTimeout).setZkSerializer(zkSerializer)
+        .setConnectionTimeout(_connectionInitTimeout).setZkSerializer(zkSerializer)
         .setMonitorType(_instanceType.name())
         .setMonitorKey(_clusterName)
         .setMonitorInstanceName(_instanceName)
@@ -633,7 +619,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
     _zkclient.subscribeStateChanges(this);
     while (retryCount < 3) {
       try {
-        _zkclient.waitUntilConnected(_sessionTimeout, TimeUnit.MILLISECONDS);
+        _zkclient.waitUntilConnected(_connectionInitTimeout, TimeUnit.MILLISECONDS);
         handleStateChanged(KeeperState.SyncConnected);
         handleNewSession();
         break;

http://git-wip-us.apache.org/repos/asf/helix/blob/2f791a68/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
index 7229914..c9f7ccf 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
@@ -19,10 +19,6 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.Callable;
-
 import org.I0Itec.zkclient.IZkConnection;
 import org.I0Itec.zkclient.ZkConnection;
 import org.I0Itec.zkclient.exception.ZkException;
@@ -32,20 +28,19 @@ import org.I0Itec.zkclient.serialize.SerializableSerializer;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.DeleteCallbackHandler;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.ExistsCallbackHandler;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.GetDataCallbackHandler;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.SetDataCallbackHandler;
+import org.apache.helix.manager.zk.ZkAsyncCallbacks.*;
 import org.apache.helix.monitoring.mbeans.ZkClientMonitor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.management.JMException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
 
 /**
  * ZKClient does not provide some functionalities, this will be used for quick fixes if
@@ -57,8 +52,6 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient {
   private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
   public static final int DEFAULT_CONNECTION_TIMEOUT = 60 * 1000;
   public static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000;
-  // public static String sessionId;
-  // public static String sessionPassword;
 
   private PathBasedZkSerializer _zkSerializer;
   private ZkClientMonitor _monitor;
@@ -83,7 +76,7 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient {
   }
 
   public ZkClient(String zkServers, String monitorType, String monitorKey) {
-    this(new ZkConnection(zkServers), Integer.MAX_VALUE,
+    this(new ZkConnection(zkServers, DEFAULT_SESSION_TIMEOUT), Integer.MAX_VALUE,
         new BasicZkSerializer(new SerializableSerializer()), monitorType, monitorKey);
   }
 
@@ -126,7 +119,8 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient {
   }
 
   public ZkClient(String zkServers, int connectionTimeout) {
-    this(new ZkConnection(zkServers), connectionTimeout, new SerializableSerializer());
+    this(new ZkConnection(zkServers, DEFAULT_SESSION_TIMEOUT), connectionTimeout,
+        new SerializableSerializer());
   }
 
   public ZkClient(String zkServers) {
@@ -598,12 +592,12 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient {
   public static class Builder {
     IZkConnection _connection;
     String _zkServer;
-    Integer _sessionTimeout;
 
     PathBasedZkSerializer _zkSerializer;
 
     long _operationRetryTimeout = -1L;
-    int _connectionTimeout = Integer.MAX_VALUE;
+    int _connectionTimeout = DEFAULT_CONNECTION_TIMEOUT;
+    int _sessionTimeout = DEFAULT_SESSION_TIMEOUT;
 
     String _monitorType;
     String _monitorKey;
@@ -684,11 +678,7 @@ public class ZkClient extends org.I0Itec.zkclient.ZkClient {
           throw new HelixException(
               "Failed to build ZkClient since no connection or ZK server address is specified.");
         } else {
-          if (_sessionTimeout == null) {
-            _connection = new ZkConnection(_zkServer);
-          } else {
-            _connection = new ZkConnection(_zkServer, _sessionTimeout);
-          }
+          _connection = new ZkConnection(_zkServer, _sessionTimeout);
         }
       }
 


[17/50] [abbrv] helix git commit: Support update complete instance config in REST

Posted by jx...@apache.org.
Support update complete instance config in REST


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

Branch: refs/heads/master
Commit: ae537054b61e4801c55bee246622f67c748c53a4
Parents: d03dca1
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Nov 15 17:54:28 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:33 2018 -0800

----------------------------------------------------------------------
 .../rest/server/resources/InstanceAccessor.java | 23 +++++++++++++++++++
 .../helix/rest/server/TestInstanceAccessor.java | 24 ++++++++++++++++++++
 2 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ae537054/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
index 0099097..72ffc19 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
@@ -300,6 +300,29 @@ public class InstanceAccessor extends AbstractResource {
     return notFound();
   }
 
+  @PUT
+  @Path("{instanceName}/configs")
+  public Response updateInstanceConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName, String content) throws IOException {
+    HelixAdmin admin = getHelixAdmin();
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a vaild ZNRecord!");
+    }
+
+    try {
+      admin.setInstanceConfig(clusterId, instanceName, new InstanceConfig(record));
+    } catch (Exception ex) {
+      _logger.error("Error in update instance config: " + instanceName, ex);
+      return serverError(ex);
+    }
+
+    return OK();
+  }
+
   @GET
   @Path("{instanceName}/resources")
   public Response getResourcesOnInstance(@PathParam("clusterId") String clusterId,

http://git-wip-us.apache.org/repos/asf/helix/blob/ae537054/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 7e1b4cb..8db348e 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -32,6 +32,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.apache.helix.HelixException;
 import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.rest.server.resources.AbstractResource;
@@ -163,4 +164,27 @@ public class TestInstanceAccessor extends AbstractTestClass {
     Assert.assertEquals(clusterConfig.getDisabledInstances().keySet(),
         new HashSet<>(Arrays.asList(CLUSTER_NAME + "localhost_12919")));
   }
+
+  @Test(dependsOnMethods = "updateInstance")
+  public void updateInstanceConfig() throws IOException {
+    System.out.println("Start test :" + TestHelper.getTestMethodName());
+    String instanceName = CLUSTER_NAME + "localhost_12918";
+    InstanceConfig instanceConfig = _configAccessor.getInstanceConfig(CLUSTER_NAME, instanceName);
+    ZNRecord record = instanceConfig.getRecord();
+    record.getSimpleFields().put("TestSimple", "value");
+    record.getMapFields().put("TestMap", ImmutableMap.of("key", "value"));
+    record.getListFields().put("TestList", Arrays.asList("e1", "e2", "e3"));
+
+    Entity entity =
+        Entity.entity(OBJECT_MAPPER.writeValueAsString(record), MediaType.APPLICATION_JSON_TYPE);
+    put("clusters/" + CLUSTER_NAME + "/instances/" + instanceName + "/configs", null, entity,
+        Response.Status.OK.getStatusCode());
+    Assert.assertEquals(record.getSimpleFields(),
+        _configAccessor.getInstanceConfig(CLUSTER_NAME, instanceName).getRecord()
+            .getSimpleFields());
+    Assert.assertEquals(record.getListFields(),
+        _configAccessor.getInstanceConfig(CLUSTER_NAME, instanceName).getRecord().getListFields());
+    Assert.assertEquals(record.getMapFields(),
+        _configAccessor.getInstanceConfig(CLUSTER_NAME, instanceName).getRecord().getMapFields());
+  }
 }


[24/50] [abbrv] helix git commit: HELIX-620: reduce amount of unnessary newing of helix management object in service context

Posted by jx...@apache.org.
HELIX-620: reduce amount of unnessary newing of helix management object in service context


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

Branch: refs/heads/master
Commit: d25552b30ab745dbcc94bcd870c3ecb2e1c6d35c
Parents: ae13411
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Nov 28 15:02:32 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:03 2018 -0800

----------------------------------------------------------------------
 .../org/apache/helix/tools/ClusterSetup.java    |  6 +++
 .../apache/helix/rest/server/ServerContext.java | 54 +++++++++++++++-----
 .../helix/rest/server/AbstractTestClass.java    | 17 ++++--
 .../helix/rest/server/TestInstanceAccessor.java |  3 ++
 4 files changed, 62 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d25552b3/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
index 5736169..030cd3d 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterSetup.java
@@ -150,6 +150,12 @@ public class ClusterSetup {
     _admin = new ZKHelixAdmin(_zkClient);
   }
 
+  public ClusterSetup(ZkClient zkClient, HelixAdmin zkHelixAdmin) {
+    _zkServerAddress = zkClient.getServers();
+    _zkClient = zkClient;
+    _admin = zkHelixAdmin;
+  }
+
   public void addCluster(String clusterName, boolean overwritePrevious) {
     _admin.addCluster(clusterName, overwritePrevious);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/d25552b3/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
index c258104..2a79e6b 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
@@ -20,6 +20,8 @@ package org.apache.helix.rest.server;
  * under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
@@ -34,41 +36,67 @@ import org.apache.helix.task.TaskDriver;
 import org.apache.helix.tools.ClusterSetup;
 
 public class ServerContext {
-  private String _zkAddr;
-  private ZkClient _zkClient;
+  private final String _zkAddr;
+  private final ZkClient _zkClient;
+  private final ZKHelixAdmin _zkHelixAdmin;
+  private final ClusterSetup _clusterSetup;
+  private final ConfigAccessor _configAccessor;
+
+  // 1 Cluster name will correspond to 1 helix data accessor
+  private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
+
+  // 1 Cluster name will correspond to 1 task driver
+  private final Map<String, TaskDriver> _taskDriverPool;
 
   public ServerContext(String zkAddr) {
     _zkAddr = zkAddr;
+    _zkClient = new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+        ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
+
+    // Accessors
+    _configAccessor = new ConfigAccessor(getZkClient());
+    _helixDataAccessorPool = new HashMap<>();
+    _taskDriverPool = new HashMap<>();
+
+    // High level interfaces
+    _zkHelixAdmin = new ZKHelixAdmin(getZkClient());
+    _clusterSetup = new ClusterSetup(getZkClient(), getHelixAdmin());
   }
 
   public ZkClient getZkClient() {
-    if (_zkClient == null) {
-      _zkClient = new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
-          ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
-    }
-
     return _zkClient;
   }
 
   public HelixAdmin getHelixAdmin() {
-    return new ZKHelixAdmin(getZkClient());
+    return _zkHelixAdmin;
   }
 
   public ClusterSetup getClusterSetup() {
-    return new ClusterSetup(getZkClient());
+    return _clusterSetup;
   }
 
   public TaskDriver getTaskDriver(String clusterName) {
-    return new TaskDriver(getZkClient(), clusterName);
+    synchronized (_taskDriverPool) {
+      if (!_taskDriverPool.containsKey(clusterName)) {
+        _taskDriverPool.put(clusterName, new TaskDriver(getZkClient(), clusterName));
+      }
+      return _taskDriverPool.get(clusterName);
+    }
   }
 
   public ConfigAccessor getConfigAccessor() {
-    return new ConfigAccessor(getZkClient());
+    return _configAccessor;
   }
 
   public HelixDataAccessor getDataAccssor(String clusterName) {
-    ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<>(getZkClient());
-    return new ZKHelixDataAccessor(clusterName, InstanceType.ADMINISTRATOR, baseDataAccessor);
+    synchronized (_helixDataAccessorPool) {
+      if (!_helixDataAccessorPool.containsKey(clusterName)) {
+        ZkBaseDataAccessor<ZNRecord> baseDataAccessor = new ZkBaseDataAccessor<>(getZkClient());
+        _helixDataAccessorPool.put(clusterName,
+            new ZKHelixDataAccessor(clusterName, InstanceType.ADMINISTRATOR, baseDataAccessor));
+      }
+      return _helixDataAccessorPool.get(clusterName);
+    }
   }
 
   public void close() {

http://git-wip-us.apache.org/repos/asf/helix/blob/d25552b3/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
index 1eb494c..29b198e 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/AbstractTestClass.java
@@ -116,6 +116,18 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
 
   @Override
   protected Application configure() {
+    // start zk
+    try {
+      if (_zkServer == null) {
+        _zkServer = TestHelper.startZkServer(ZK_ADDR);
+        Assert.assertTrue(_zkServer != null);
+        ZKClientPool.reset();
+      }
+    } catch (Exception e) {
+      Assert.assertTrue(false, String.format("Failed to start ZK server: %s", e.toString()));
+    }
+
+    // Configure server context
     ResourceConfig resourceConfig = new ResourceConfig();
     resourceConfig.packages(AbstractResource.class.getPackage().getName());
     ServerContext serverContext = new ServerContext(ZK_ADDR);
@@ -170,11 +182,6 @@ public class AbstractTestClass extends JerseyTestNg.ContainerPerClassTest {
       java.util.logging.Logger topJavaLogger = java.util.logging.Logger.getLogger("");
       topJavaLogger.setLevel(Level.WARNING);
 
-      // start zk
-      _zkServer = TestHelper.startZkServer(ZK_ADDR);
-      Assert.assertTrue(_zkServer != null);
-      ZKClientPool.reset();
-
       _gZkClient = new ZkClient(ZK_ADDR, ZkClient.DEFAULT_CONNECTION_TIMEOUT,
           ZkClient.DEFAULT_SESSION_TIMEOUT, new ZNRecordSerializer());
       _gSetupTool = new ClusterSetup(_gZkClient);

http://git-wip-us.apache.org/repos/asf/helix/blob/d25552b3/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 8db348e..6c83bef 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -138,6 +138,8 @@ public class TestInstanceAccessor extends AbstractTestClass {
     Assert.assertEquals(_configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME).getTags(),
         ImmutableList.of("tag2"));
 
+    // TODO (JK): Reenable this after storage node bug fixed.
+    /*
     // Batch disable instances
     List<String> instancesToDisable = Arrays.asList(
         new String[] { CLUSTER_NAME + "localhost_12918", CLUSTER_NAME + "localhost_12919",
@@ -163,6 +165,7 @@ public class TestInstanceAccessor extends AbstractTestClass {
     clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
     Assert.assertEquals(clusterConfig.getDisabledInstances().keySet(),
         new HashSet<>(Arrays.asList(CLUSTER_NAME + "localhost_12919")));
+    */
   }
 
   @Test(dependsOnMethods = "updateInstance")


[11/50] [abbrv] helix git commit: Update ivy file in helix-core to use metrics-core 3.2.3

Posted by jx...@apache.org.
Update ivy file in helix-core to use metrics-core 3.2.3


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

Branch: refs/heads/master
Commit: bd9f7a4a8ef8dcf26e1886a993eea67131e47fe3
Parents: 143420c
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Nov 14 14:56:54 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:08 2018 -0800

----------------------------------------------------------------------
 helix-core/helix-core-0.6.10-SNAPSHOT.ivy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/bd9f7a4a/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
----------------------------------------------------------------------
diff --git a/helix-core/helix-core-0.6.10-SNAPSHOT.ivy b/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
index d6a597b..1e22533 100644
--- a/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
+++ b/helix-core/helix-core-0.6.10-SNAPSHOT.ivy
@@ -62,7 +62,7 @@ under the License.
     <dependency org="com.google.guava" name="guava" rev="15.0" conf="compile->compile(default);runtime->runtime(default);default->default"/>
     <dependency org="org.yaml" name="snakeyaml" rev="1.12" conf="compile->compile(default);runtime->runtime(default);default->default"/>
     <dependency org="commons-logging" name="commons-logging-api" rev="1.1" conf="compile->compile(*),master(*);runtime->runtime(*)"/>
-    <dependency org="io.dropwizard.metrics" name="metrics-core" rev="3.1.2" conf="compile->compile(default);runtime->runtime(default);default->default"/>
+    <dependency org="io.dropwizard.metrics" name="metrics-core" rev="3.2.3" conf="compile->compile(default);runtime->runtime(default);default->default"/>
 	</dependencies>
 </ivy-module>
 


[40/50] [abbrv] helix git commit: HELIX-690: don't new zkClient and its dependencies in ServerContext constructor

Posted by jx...@apache.org.
HELIX-690: don't new zkClient and its dependencies in ServerContext constructor


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

Branch: refs/heads/master
Commit: 4d2734e723e79e2f7659be9fcbe238ed49fcb574
Parents: 310d476
Author: hrzhang <hr...@linkedin.com>
Authored: Thu Jan 4 17:43:36 2018 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:02 2018 -0800

----------------------------------------------------------------------
 .../apache/helix/rest/server/ServerContext.java | 33 +++++++++++++-------
 1 file changed, 21 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4d2734e7/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
index 2a79e6b..02e74c5 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/ServerContext.java
@@ -37,10 +37,10 @@ import org.apache.helix.tools.ClusterSetup;
 
 public class ServerContext {
   private final String _zkAddr;
-  private final ZkClient _zkClient;
-  private final ZKHelixAdmin _zkHelixAdmin;
-  private final ClusterSetup _clusterSetup;
-  private final ConfigAccessor _configAccessor;
+  private ZkClient _zkClient;
+  private ZKHelixAdmin _zkHelixAdmin;
+  private ClusterSetup _clusterSetup;
+  private ConfigAccessor _configAccessor;
 
   // 1 Cluster name will correspond to 1 helix data accessor
   private final Map<String, HelixDataAccessor> _helixDataAccessorPool;
@@ -50,28 +50,34 @@ public class ServerContext {
 
   public ServerContext(String zkAddr) {
     _zkAddr = zkAddr;
-    _zkClient = new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
-        ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
 
-    // Accessors
-    _configAccessor = new ConfigAccessor(getZkClient());
+    // We should NOT initiate _zkClient and anything that depends on _zkClient in
+    // constructor, as it is reasonable to start up HelixRestServer first and then
+    // ZooKeeper. In this case, initializing _zkClient will fail and HelixRestServer
+    // cannot be started correctly.
     _helixDataAccessorPool = new HashMap<>();
     _taskDriverPool = new HashMap<>();
-
-    // High level interfaces
-    _zkHelixAdmin = new ZKHelixAdmin(getZkClient());
-    _clusterSetup = new ClusterSetup(getZkClient(), getHelixAdmin());
   }
 
   public ZkClient getZkClient() {
+    if (_zkClient == null) {
+      _zkClient = new ZkClient(_zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+          ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
+    }
     return _zkClient;
   }
 
   public HelixAdmin getHelixAdmin() {
+    if (_zkHelixAdmin == null) {
+      _zkHelixAdmin = new ZKHelixAdmin(getZkClient());
+    }
     return _zkHelixAdmin;
   }
 
   public ClusterSetup getClusterSetup() {
+    if (_clusterSetup == null) {
+      _clusterSetup = new ClusterSetup(getZkClient(), getHelixAdmin());
+    }
     return _clusterSetup;
   }
 
@@ -85,6 +91,9 @@ public class ServerContext {
   }
 
   public ConfigAccessor getConfigAccessor() {
+    if (_configAccessor == null) {
+      _configAccessor = new ConfigAccessor(getZkClient());
+    }
     return _configAccessor;
   }
 


[33/50] [abbrv] helix git commit: Support Helix REST enable/disable partitions

Posted by jx...@apache.org.
Support Helix REST enable/disable partitions


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

Branch: refs/heads/master
Commit: 4ff98fb19ffc1b21893089a9d4d4e61864248647
Parents: 7fc03f4
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Nov 16 17:39:12 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:35 2018 -0800

----------------------------------------------------------------------
 .../rest/server/resources/AbstractResource.java |  2 +
 .../rest/server/resources/InstanceAccessor.java | 15 ++++++++
 .../helix/rest/server/TestInstanceAccessor.java | 40 ++++++++++++++++++--
 3 files changed, 54 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/4ff98fb1/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
index 70f713a..a89ae5d 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/AbstractResource.java
@@ -67,6 +67,8 @@ public class AbstractResource {
     disable,
     enableMaintenanceMode,
     disableMaintenanceMode,
+    enablePartitions,
+    disablePartitions,
     update,
     delete,
     rebalance,

http://git-wip-us.apache.org/repos/asf/helix/blob/4ff98fb1/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
index 72ffc19..eeecba9 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
@@ -259,6 +259,21 @@ public class InstanceAccessor extends AbstractResource {
           admin.removeInstanceTag(clusterId, instanceName, tag);
         }
         break;
+      case enablePartitions:
+        admin.enablePartition(true, clusterId, instanceName,
+            node.get(InstanceProperties.resource.name()).getTextValue(),
+            (List<String>) OBJECT_MAPPER
+                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
+                    OBJECT_MAPPER.getTypeFactory()
+                        .constructCollectionType(List.class, String.class)));
+        break;
+      case disablePartitions:
+        admin.enablePartition(false, clusterId, instanceName,
+            node.get(InstanceProperties.resource.name()).getTextValue(),
+            (List<String>) OBJECT_MAPPER
+                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
+                    OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class)));
+        break;
       default:
         _logger.error("Unsupported command :" + command);
         return badRequest("Unsupported command :" + command);

http://git-wip-us.apache.org/repos/asf/helix/blob/4ff98fb1/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 6c83bef..947ba49 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -137,10 +137,10 @@ public class TestInstanceAccessor extends AbstractTestClass {
         Response.Status.OK.getStatusCode());
     Assert.assertEquals(_configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME).getTags(),
         ImmutableList.of("tag2"));
-
-    // TODO (JK): Reenable this after storage node bug fixed.
-    /*
+    
+    // TODO: Reenable the test after storage node fix the problem
     // Batch disable instances
+    /*
     List<String> instancesToDisable = Arrays.asList(
         new String[] { CLUSTER_NAME + "localhost_12918", CLUSTER_NAME + "localhost_12919",
             CLUSTER_NAME + "localhost_12920"
@@ -166,6 +166,40 @@ public class TestInstanceAccessor extends AbstractTestClass {
     Assert.assertEquals(clusterConfig.getDisabledInstances().keySet(),
         new HashSet<>(Arrays.asList(CLUSTER_NAME + "localhost_12919")));
     */
+
+    // Test enable disable partitions
+    String dbName = "_db_0_";
+    List<String> partitionsToDisable = Arrays.asList(
+        new String[] { CLUSTER_NAME + dbName + "0", CLUSTER_NAME + dbName + "1",
+            CLUSTER_NAME + dbName + "3"
+        });
+
+    entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(ImmutableMap
+            .of(AbstractResource.Properties.id.name(), INSTANCE_NAME,
+                InstanceAccessor.InstanceProperties.resource.name(),
+                CLUSTER_NAME + dbName.substring(0, dbName.length() - 1),
+                InstanceAccessor.InstanceProperties.partitions.name(), partitionsToDisable)),
+        MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME,
+        ImmutableMap.of("command", "disablePartitions"), entity,
+        Response.Status.OK.getStatusCode());
+    InstanceConfig instanceConfig = _configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME);
+    Assert.assertEquals(new HashSet<>(instanceConfig.getDisabledPartitionsMap()
+            .get(CLUSTER_NAME + dbName.substring(0, dbName.length() - 1))),
+        new HashSet<>(partitionsToDisable));
+    entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(ImmutableMap
+        .of(AbstractResource.Properties.id.name(), INSTANCE_NAME,
+            InstanceAccessor.InstanceProperties.resource.name(),
+            CLUSTER_NAME + dbName.substring(0, dbName.length() - 1),
+            InstanceAccessor.InstanceProperties.partitions.name(),
+            ImmutableList.of(CLUSTER_NAME + dbName + "1"))), MediaType.APPLICATION_JSON_TYPE);
+
+    post("clusters/" + CLUSTER_NAME + "/instances/" + INSTANCE_NAME,
+        ImmutableMap.of("command", "enablePartitions"), entity, Response.Status.OK.getStatusCode());
+    instanceConfig = _configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME);
+    Assert.assertEquals(new HashSet<>(instanceConfig.getDisabledPartitionsMap()
+            .get(CLUSTER_NAME + dbName.substring(0, dbName.length() - 1))),
+        new HashSet<>(Arrays.asList(CLUSTER_NAME + dbName + "0", CLUSTER_NAME + dbName + "3")));
   }
 
   @Test(dependsOnMethods = "updateInstance")


[48/50] [abbrv] helix git commit: Remove duplicate topology calculation and Consistent Hashing ring constructions for better algorithm performance.

Posted by jx...@apache.org.
Remove duplicate topology calculation and Consistent Hashing ring constructions for better algorithm performance.

This change reduces running time greatly, especially when resource count is larger than 100.


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

Branch: refs/heads/master
Commit: 77b09c33d85723efcabd6a6e5c6175b1f3747d1d
Parents: 9dfb098
Author: Jiajun Wang <jj...@linkedin.com>
Authored: Fri Dec 22 17:31:32 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:29 2018 -0800

----------------------------------------------------------------------
 ...stractEvenDistributionRebalanceStrategy.java | 112 +--
 .../ConsistentHashingAdjustmentAlgorithm.java   |  85 ++-
 .../java/org/apache/helix/DistributionTest.java | 753 -------------------
 3 files changed, 110 insertions(+), 840 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/77b09c33/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
index 9012f73..c3093b1 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/AbstractEvenDistributionRebalanceStrategy.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.*;
+import java.util.concurrent.ExecutionException;
 
 /**
  * Abstract class of Forced Even Assignment Patched Algorithm.
@@ -65,80 +66,81 @@ public abstract class AbstractEvenDistributionRebalanceStrategy implements Rebal
   public ZNRecord computePartitionAssignment(final List<String> allNodes,
       final List<String> liveNodes, final Map<String, Map<String, String>> currentMapping,
       ClusterDataCache clusterData) throws HelixException {
-    boolean continueNextStep = true;
     // Round 1: Calculate mapping using the base strategy.
     // Note to use all nodes for minimizing the influence of live node changes to mapping.
     ZNRecord origAssignment = getBaseRebalanceStrategy()
         .computePartitionAssignment(allNodes, allNodes, currentMapping, clusterData);
     Map<String, List<String>> origPartitionMap = origAssignment.getListFields();
-    // If the original calculation contains no assignment, skip patching
-    if (origPartitionMap.isEmpty()) {
-      continueNextStep = false;
-    }
 
-    // Transform current assignment to instance->partitions map, and get total partitions
-    Map<String, List<String>> nodeToPartitionMap = convertMap(origPartitionMap);
+    // Try to re-assign if the original map is not empty
+    if (!origPartitionMap.isEmpty()) {
+      // Transform current assignment to instance->partitions map, and get total partitions
+      Map<String, List<String>> nodeToPartitionMap = convertMap(origPartitionMap);
+
+      Map<String, List<String>> finalPartitionMap = null;
 
-    if (continueNextStep) {
       // Round 2: Rebalance mapping using card dealing algorithm. For ensuring evenness distribution.
       Topology allNodeTopo = new Topology(allNodes, allNodes, clusterData.getInstanceConfigMap(),
           clusterData.getClusterConfig());
       CardDealingAdjustmentAlgorithm cardDealer =
           new CardDealingAdjustmentAlgorithm(allNodeTopo, _replica);
-      continueNextStep = cardDealer.computeMapping(nodeToPartitionMap, _resourceName.hashCode());
-    }
-
-    // Round 3: Reorder preference Lists to ensure participants' orders (so as the states) are uniform.
-    Map<String, List<String>> partitionMap = shufflePreferenceList(nodeToPartitionMap);
-
-    // Round 4: Re-mapping the partitions on non-live nodes using consistent hashing for reducing movement.
-    if (continueNextStep && !liveNodes.containsAll(allNodes)) {
-      Topology liveNodeTopo = new Topology(allNodes, liveNodes, clusterData.getInstanceConfigMap(),
-          clusterData.getClusterConfig());
-      ConsistentHashingAdjustmentAlgorithm hashPlacement =
-          new ConsistentHashingAdjustmentAlgorithm(liveNodeTopo);
-      if (hashPlacement.computeMapping(nodeToPartitionMap, _resourceName.hashCode())) {
-        // Since mapping is changed by hashPlacement, need to adjust nodes order.
-        Map<String, List<String>> adjustedPartitionMap = convertMap(nodeToPartitionMap);
-        for (String partition : adjustedPartitionMap.keySet()) {
-          List<String> preSelectedList = partitionMap.get(partition);
-          Set<String> adjustedNodeList = new HashSet<>(adjustedPartitionMap.get(partition));
-          List<String> finalNodeList = adjustedPartitionMap.get(partition);
-          int index = 0;
-          // 1. Add the ones in pre-selected node list first, in order
-          for (String node : preSelectedList) {
-            if (adjustedNodeList.remove(node)) {
-              finalNodeList.set(index++, node);
+      if (cardDealer.computeMapping(nodeToPartitionMap, _resourceName.hashCode())) {
+        // Round 3: Reorder preference Lists to ensure participants' orders (so as the states) are uniform.
+        finalPartitionMap = shufflePreferenceList(nodeToPartitionMap);
+        if (!liveNodes.containsAll(allNodes)) {
+          try {
+            // Round 4: Re-mapping the partitions on non-live nodes using consistent hashing for reducing movement.
+            ConsistentHashingAdjustmentAlgorithm hashPlacement =
+                new ConsistentHashingAdjustmentAlgorithm(allNodeTopo, liveNodes);
+            if (hashPlacement.computeMapping(nodeToPartitionMap, _resourceName.hashCode())) {
+              // Since mapping is changed by hashPlacement, need to adjust nodes order.
+              Map<String, List<String>> adjustedPartitionMap = convertMap(nodeToPartitionMap);
+              for (String partition : adjustedPartitionMap.keySet()) {
+                List<String> preSelectedList = finalPartitionMap.get(partition);
+                Set<String> adjustedNodeList = new HashSet<>(adjustedPartitionMap.get(partition));
+                List<String> finalNodeList = adjustedPartitionMap.get(partition);
+                int index = 0;
+                // 1. Add the ones in pre-selected node list first, in order
+                for (String node : preSelectedList) {
+                  if (adjustedNodeList.remove(node)) {
+                    finalNodeList.set(index++, node);
+                  }
+                }
+                // 2. Add the rest of nodes to the map
+                for (String node : adjustedNodeList) {
+                  finalNodeList.set(index++, node);
+                }
+              }
+              finalPartitionMap = adjustedPartitionMap;
+            } else {
+              // Adjustment failed, the final partition map is not valid
+              finalPartitionMap = null;
             }
-          }
-          // 2. Add the rest of nodes to the map
-          for (String node : adjustedNodeList) {
-            finalNodeList.set(index++, node);
+          } catch (ExecutionException e) {
+            _logger.error("Failed to perform consistent hashing partition assigner.", e);
+            finalPartitionMap = null;
           }
         }
-        partitionMap = adjustedPartitionMap;
-      } else {
-        continueNextStep = false;
+      }
+
+      if (null != finalPartitionMap) {
+        ZNRecord result = new ZNRecord(_resourceName);
+        result.setListFields(finalPartitionMap);
+        return result;
       }
     }
 
-    if (continueNextStep) {
-      ZNRecord result = new ZNRecord(_resourceName);
-      result.setListFields(partitionMap);
-      return result;
+    // Force even is not possible, fallback to use default strategy
+    if (_logger.isDebugEnabled()) {
+      _logger.debug("Force even distribution is not possible, using the default strategy: "
+          + getBaseRebalanceStrategy().getClass().getSimpleName());
+    }
+    if (liveNodes.equals(allNodes)) {
+      return origAssignment;
     } else {
-      if (_logger.isDebugEnabled()) {
-        _logger.debug("Force even distribution is not possible, using the default strategy: "
-            + getBaseRebalanceStrategy().getClass().getSimpleName());
-      }
-      // Force even is not possible, fallback to use default strategy
-      if (liveNodes.equals(allNodes)) {
-        return origAssignment;
-      } else {
-        // need to re-calculate since node list is different.
-        return getBaseRebalanceStrategy()
-            .computePartitionAssignment(allNodes, liveNodes, currentMapping, clusterData);
-      }
+      // need to re-calculate since node list is different.
+      return getBaseRebalanceStrategy()
+          .computePartitionAssignment(allNodes, liveNodes, currentMapping, clusterData);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/77b09c33/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
index e594cd1..c7ff844 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/strategy/crushMapping/ConsistentHashingAdjustmentAlgorithm.java
@@ -1,29 +1,46 @@
 package org.apache.helix.controller.rebalancer.strategy.crushMapping;
 
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import org.apache.helix.controller.rebalancer.topology.Node;
 import org.apache.helix.controller.rebalancer.topology.Topology;
 import org.apache.helix.util.JenkinsHash;
 
 import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 public class ConsistentHashingAdjustmentAlgorithm {
+  private static final int MAX_SELETOR_CACHE_SIZE = 1000;
+  private static final int SELETOR_CACHE_EXPIRE = 3;
+
   private JenkinsHash _hashFunction;
   private ConsistentHashSelector _selector;
-  Set<String> _liveInstances = new HashSet<>();
+  Set<String> _activeInstances = new HashSet<>();
+
   // Instance -> FaultZone Tag
   private Map<String, String> _faultZoneMap = new HashMap<>();
   // Record existing partitions that are assigned to a fault zone
   private Map<String, Set<String>> _faultZonePartitionMap = new HashMap<>();
 
-  public ConsistentHashingAdjustmentAlgorithm(Topology topology) {
+  // Cache records all known topology.
+  private final static LoadingCache<Set<String>, ConsistentHashSelector> _selectorCache =
+      CacheBuilder.newBuilder().maximumSize(MAX_SELETOR_CACHE_SIZE)
+          .expireAfterAccess(SELETOR_CACHE_EXPIRE, TimeUnit.MINUTES)
+          .build(new CacheLoader<Set<String>, ConsistentHashSelector>() {
+            public ConsistentHashSelector load(Set<String> allInstances) {
+              return new ConsistentHashSelector(allInstances);
+            }
+          });
+
+  public ConsistentHashingAdjustmentAlgorithm(Topology topology, Collection<String> activeInstances)
+      throws ExecutionException {
     _hashFunction = new JenkinsHash();
-    List<String> allInstances = new ArrayList<>();
+    Set<String> allInstances = new HashSet<>();
     // Get all instance related information.
     for (Node zone : topology.getFaultZones()) {
       for (Node instance : Topology.getAllLeafNodes(zone)) {
-        if (!instance.isFailed()) {
-          _liveInstances.add(instance.getName());
-        }
         allInstances.add(instance.getName());
         _faultZoneMap.put(instance.getName(), zone.getName());
         if (!_faultZonePartitionMap.containsKey(zone.getName())) {
@@ -31,11 +48,12 @@ public class ConsistentHashingAdjustmentAlgorithm {
         }
       }
     }
-    _selector = new ConsistentHashSelector(allInstances);
+    _selector = _selectorCache.get(allInstances);
+    _activeInstances.addAll(activeInstances);
   }
 
   public boolean computeMapping(Map<String, List<String>> nodeToPartitionMap, int randomSeed) {
-    if (_liveInstances.isEmpty()) {
+    if (_activeInstances.isEmpty()) {
       return false;
     }
 
@@ -46,7 +64,7 @@ public class ConsistentHashingAdjustmentAlgorithm {
     while (nodeIter.hasNext()) {
       String instance = nodeIter.next();
       List<String> partitions = nodeToPartitionMap.get(instance);
-      if (!_liveInstances.contains(instance)) {
+      if (!_activeInstances.contains(instance)) {
         inactiveInstances.add(instance);
         addToReAssignPartition(toBeReassigned, partitions);
         partitions.clear();
@@ -60,11 +78,13 @@ public class ConsistentHashingAdjustmentAlgorithm {
       int remainReplicas = toBeReassigned.get(partition);
       Set<String> conflictInstance = new HashSet<>();
       for (int index = 0; index < toBeReassigned.get(partition); index++) {
-        Iterable<String> sortedInstances = _selector.getCircle(_hashFunction.hash(randomSeed, partition.hashCode(), index));
+        Iterable<String> sortedInstances =
+            _selector.getCircle(_hashFunction.hash(randomSeed, partition.hashCode(), index));
         Iterator<String> instanceItr = sortedInstances.iterator();
-        while (instanceItr.hasNext() && conflictInstance.size() + inactiveInstances.size() != _selector.instanceSize) {
+        while (instanceItr.hasNext()
+            && conflictInstance.size() + inactiveInstances.size() != _selector.instanceSize) {
           String instance = instanceItr.next();
-          if (!_liveInstances.contains(instance)) {
+          if (!_activeInstances.contains(instance)) {
             inactiveInstances.add(instance);
           }
           if (inactiveInstances.contains(instance) || conflictInstance.contains(instance)) {
@@ -105,32 +125,33 @@ public class ConsistentHashingAdjustmentAlgorithm {
       }
     }
   }
+}
 
-  private class ConsistentHashSelector {
-    private final static int DEFAULT_TOKENS_PER_INSTANCE = 1000;
-    private final SortedMap<Long, String> circle = new TreeMap<Long, String>();
-    protected int instanceSize = 0;
+class ConsistentHashSelector {
+  private final static int DEFAULT_TOKENS_PER_INSTANCE = 1000;
+  private final static JenkinsHash _hashFunction = new JenkinsHash();
+  private final SortedMap<Long, String> circle = new TreeMap<>();
+  protected int instanceSize = 0;
 
-    public ConsistentHashSelector(List<String> instances) {
-      for (String instance : instances) {
-        long tokenCount = DEFAULT_TOKENS_PER_INSTANCE;
-        add(instance, tokenCount);
-        instanceSize++;
-      }
+  public ConsistentHashSelector(Set<String> instances) {
+    for (String instance : instances) {
+      add(instance, DEFAULT_TOKENS_PER_INSTANCE);
+      instanceSize++;
     }
+  }
 
-    private void add(String instance, long numberOfReplicas) {
-      for (int i = 0; i < numberOfReplicas; i++) {
-        circle.put(_hashFunction.hash(instance.hashCode(), i), instance);
-      }
+  private void add(String instance, long numberOfReplicas) {
+    int instanceHashCode = instance.hashCode();
+    for (int i = 0; i < numberOfReplicas; i++) {
+      circle.put(_hashFunction.hash(instanceHashCode, i), instance);
     }
+  }
 
-    public Iterable<String> getCircle(long data) {
-      if (circle.isEmpty()) {
-        return null;
-      }
-      long hash = _hashFunction.hash(data);
-      return circle.tailMap(hash).values();
+  public Iterable<String> getCircle(long data) {
+    if (circle.isEmpty()) {
+      return null;
     }
+    long hash = _hashFunction.hash(data);
+    return circle.tailMap(hash).values();
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/77b09c33/helix-core/src/test/java/org/apache/helix/DistributionTest.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/DistributionTest.java b/helix-core/src/test/java/org/apache/helix/DistributionTest.java
deleted file mode 100644
index bf580d1..0000000
--- a/helix-core/src/test/java/org/apache/helix/DistributionTest.java
+++ /dev/null
@@ -1,753 +0,0 @@
-package org.apache.helix;
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
-import org.apache.helix.controller.rebalancer.topology.Node;
-import org.apache.helix.controller.rebalancer.topology.Topology;
-import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.apache.helix.model.BuiltInStateModelDefinitions;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.util.HelixUtil;
-import org.testng.annotations.DataProvider;
-import org.testng.annotations.Test;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.*;
-
-public class DistributionTest {
-  private static String instanceFolderPath;
-  private static String instanceList;
-  private static String idealStateFolderPath;
-  private static String idealStateList;
-
-  String Path = "/home/jjwang/Desktop/FEAP-test";
-  //String Path = "/Users/jjwang/Desktop/FEAP-test";
-
-  @DataProvider(name = "rebalanceStrategies")
-  public static String[][] rebalanceStrategies() {
-    return new String[][] {
-        //{AutoRebalanceStrategy.class.getName()},
-        { CrushRebalanceStrategy.class.getName() },
-        //{ MultiRoundCrushRebalanceStrategy.class.getName() },
-        //{ CrushEdRebalanceStrategy.class.getName() }
-    };
-  }
-
-  String[] fabrics = { "lor1", "lva1", "ltx1", "lsg1",
-  };
-  String[] clusters = { "ESPRESSO_IDENTITY", "ESPRESSO_MT-MD-1", "ESPRESSO_TSCP", "ESPRESSO_MT_PHASE1",
-          "ESPRESSO_MT-MD-3", "ESPRESSO_USCP", "ESPRESSO_MT-LEGACY", /* "venice-0" */
-  };
-  String topState = "master";
-  float[] nodeAdjustSimulator =
-      { /*-0.5f, -0.2f, -0.1f, -0.01f, */ 0.01f, 0.1f, 0.2f, 0.5f, 1f};
-
-  @Test(dataProvider = "rebalanceStrategies")
-  public void testNodeChange(String rebalanceStrategyClass) throws Exception {
-    for (String cluster : clusters) {
-      System.out.println(cluster
-          + "\tChangeType\tNumOfNodeChange\tDiffRate\tTotalMv\tTotalMvRate\tExtraMvRate\tExtraMvRateComparedWithAvgDist\tTopStateChange\tTopStateChangeRate\tTopStateChangeWithNewDeployRate\tExtraTopStateChangeRate");
-      for (String fabric : fabrics) {
-        String path = Path + "/" + cluster + "/" + fabric;
-        if (new File(path).exists()) {
-          System.out.print(fabric);
-          for (float adjustRate : nodeAdjustSimulator) {
-            Set<String> deltaNode = new HashSet<>();
-            List<String> liveInstances = new ArrayList<>();
-            Map<String, Map<String, String>> resultA =
-                calculate(path, rebalanceStrategyClass, adjustRate, deltaNode, liveInstances);
-            double[] distEval = checkEvenness(liveInstances, resultA, false);
-            if (adjustRate != 0) {
-              Map<String, Map<String, String>> result =
-                  calculate(path, rebalanceStrategyClass, 0, deltaNode, new ArrayList<String>());
-              double[] diff = checkMovement(result, resultA, deltaNode, false);
-              System.out.println(
-                  "\t" + (adjustRate > 0 ? "Adding\t" : "Disabling\t") + diff[0] + "\t"
-                      + distEval[3] + "\t" + diff[1] + "\t" + diff[2] + "\t" + diff[3] + "\t"
-                      + diff[8] + "\t" + diff[4] + "\t" + diff[5] + "\t" + diff[10] + "\t"
-                      + diff[6]);
-            }
-          }
-        }
-      }
-      System.out.println();
-    }
-  }
-
-  @Test(dataProvider = "rebalanceStrategies")
-  public void testDist(String rebalanceStrategyClass) throws Exception {
-    for (String cluster : clusters) {
-      System.out.println(cluster
-          + "\tTotalReplica\tMinReplica\tMaxReplica\tDiffRate\tSTDEV\tMinTopState\tMaxTopState\ttopStateDiffRate\ttopStateSTDEV");
-      for (String fabric : fabrics) {
-        String path = Path + "/" + cluster + "/" + fabric;
-        if (new File(path).exists()) {
-          Set<String> deltaNode = new HashSet<>();
-          List<String> liveInstances = new ArrayList<>();
-          Map<String, Map<String, String>> result =
-              calculate(path, rebalanceStrategyClass, 0, deltaNode, liveInstances);
-          double[] distEval = checkEvenness(liveInstances, result, false);
-          System.out.println(
-              fabric + "\t" + distEval[0] + "\t" + distEval[1] + "\t" + distEval[2] + "\t"
-                  + distEval[3] + "\t" + distEval[4] + "\t" + distEval[5] + "\t" + distEval[6]
-                  + "\t" + distEval[7] + "\t" + distEval[8]);
-        }
-      }
-      System.out.println();
-    }
-  }
-
-  int _replica = 1;
-  int partitionCount = 101;
-  int faultZone = 10;
-  int[] resourceCounts = new int[] { 100 };
-  int[] nodeCounts = new int[] { 100, /*100, 200, 500, 1000*/ };
-
-  @Test(dataProvider = "rebalanceStrategies")
-  public void testDistUsingRandomTopo(String rebalanceStrategyClass) throws Exception {
-    for (int nodeCount : nodeCounts) {
-      for (int resourceCount : resourceCounts) {
-        System.out.println(
-            "NodeCount\tResourceCount\tTotalReplica\tMinReplica\tMaxReplica\tDiffRate\tSTDEV\tMinTopState\tMaxTopState\tTopStateDiffRate\tTopStateSTDEV");
-        List<String> liveInstances = new ArrayList<>();
-        Map<String, Map<String, String>> result =
-            calculateUsingRandomTopo(rebalanceStrategyClass, _replica, partitionCount,
-                resourceCount, nodeCount, faultZone, liveInstances);
-        double[] distEval = checkEvenness(liveInstances, result, false);
-        System.out.println(
-            nodeCount + "\t" + resourceCount + "\t" + distEval[0] + "\t" + distEval[1] + "\t"
-                + distEval[2] + "\t" + distEval[3] + "\t" + distEval[4] + "\t" + distEval[5] + "\t"
-                + distEval[6] + "\t" + distEval[7] + "\t" + distEval[8]);
-      }
-    }
-
-    System.out.println();
-  }
-
-  @Test(dataProvider = "rebalanceStrategies")
-  public void testRollingUpgrade(String rebalanceStrategyClass) throws Exception {
-    for (String cluster : clusters) {
-      System.out.println(cluster
-          + "\tTotalMv\tTotalMvRate\tExtraMvRate\tExtraMvRateComparedWithAvgDist\tTopStateChange\tTopStateChangeRate\tTopStateChangeWithNewDeployRate\tExtraTopStateChange");
-      for (String fabric : fabrics) {
-        String path = Path + "/" + cluster + "/" + fabric;
-        if (new File(path).exists()) {
-
-          List<List<String>> deltaNodesHistory = new ArrayList<>();
-          List<List<String>> liveInstancesHistory = new ArrayList<>();
-
-          List<Map<String, Map<String, String>>> mappingHistory =
-              calculateRollingUpgrade(path, rebalanceStrategyClass, deltaNodesHistory,
-                  liveInstancesHistory, true);
-
-          Map<String, Map<String, String>> basicMapping =
-              calculate(path, rebalanceStrategyClass, 0, new HashSet<String>(),
-                  new ArrayList<String>());
-
-          double[] maxDiff = new double[8];
-          for (int i = 0; i < mappingHistory.size(); i++) {
-            List<String> deltaNode = deltaNodesHistory.get(i);
-            Map<String, Map<String, String>> mapA = mappingHistory.get(i);
-
-            Map<String, Map<String, String>> mapB = basicMapping;
-            if (i != 0) {
-              deltaNode.addAll(deltaNodesHistory.get(i - 1));
-              mapB = mappingHistory.get(i - 1);
-            }
-            double[] diff = checkMovement(mapB, mapA, deltaNode, false);
-
-            maxDiff[0] = Math.max(diff[1], maxDiff[0]);
-            maxDiff[1] = Math.max(diff[2], maxDiff[1]);
-            maxDiff[2] = Math.max(diff[3], maxDiff[2]);
-            maxDiff[3] = Math.max(diff[4], maxDiff[3]);
-            maxDiff[4] = Math.max(diff[5], maxDiff[4]);
-            maxDiff[5] = Math.max(diff[6], maxDiff[5]);
-            maxDiff[6] = Math.max(diff[8], maxDiff[6]);
-            maxDiff[7] = Math.max(diff[10], maxDiff[7]);
-          }
-          System.out.println(
-              fabric + "\t" + maxDiff[0] + "\t" + maxDiff[1] + "\t" + maxDiff[2] + "\t" + maxDiff[6]
-                  + "\t" + maxDiff[3] + "\t" + maxDiff[4] + "\t" + maxDiff[7] + "\t" + maxDiff[5]);
-        }
-      }
-      System.out.println();
-    }
-  }
-
-  public List<Map<String, Map<String, String>>> calculateRollingUpgrade(String Path,
-      String rebalanceStrategyClass, List<List<String>> deltaNodesHistory,
-      List<List<String>> liveInstancesHistory, boolean recoverNode) throws Exception {
-    instanceFolderPath = Path + "/instanceConfigs/";
-    instanceList = Path + "/instance";
-    idealStateFolderPath = Path + "/idealStates/";
-    idealStateList = Path + "/idealstate";
-    Path path = Paths.get(Path + "/clusterConfig");
-    ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
-    ClusterConfig clusterConfig = new ClusterConfig(record);
-    List<String> allNodes = new ArrayList<>();
-    List<InstanceConfig> instanceConfigs =
-        getInstanceConfigs(instanceFolderPath, instanceList, allNodes);
-    List<IdealState> idealStates = getIdealStates(idealStateFolderPath, idealStateList);
-
-    List<String> deltaNodes = new ArrayList<>();
-
-    List<Map<String, Map<String, String>>> totalMapHistory = new ArrayList<>();
-    for (String downNode : allNodes) {
-      deltaNodes.add(downNode);
-
-      List<String> liveInstances = new ArrayList<>(allNodes);
-      liveInstances.removeAll(deltaNodes);
-      Map<String, Map<String, String>> totalMaps = new HashMap<>();
-
-      totalMapHistory.add(totalMaps);
-      liveInstancesHistory.add(liveInstances);
-      deltaNodesHistory.add(new ArrayList<>(deltaNodes));
-
-      Map<String, Integer> partitions = new HashMap<>();
-      for (int i = 0; i < idealStates.size(); i++) {
-        Map<String, Map<String, String>> maps = HelixUtil
-            .getIdealAssignmentForFullAuto(clusterConfig, instanceConfigs, liveInstances,
-                idealStates.get(i), new ArrayList<>(idealStates.get(i).getPartitionSet()),
-                rebalanceStrategyClass);
-        for (String partitionName : idealStates.get(i).getPartitionSet()) {
-          partitions.put(partitionName, idealStates.get(i).getReplicaCount(liveInstances.size()));
-        }
-        totalMaps.putAll(maps);
-      }
-      Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
-      for (InstanceConfig config : instanceConfigs) {
-        instanceConfigMap.put(config.getInstanceName(), config);
-      }
-      verifyDistribution(totalMaps, liveInstances, partitions,
-          new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances,
-              instanceConfigMap, clusterConfig));
-      if (recoverNode) {
-        deltaNodes.remove(downNode);
-      }
-    }
-    return totalMapHistory;
-  }
-
-  public Map<String, Map<String, String>> calculateUsingRandomTopo(String rebalanceStrategyClass,
-      int replica, int partitionCount, int nodeCount, int resourceCount, int faultZone,
-      List<String> liveInstances) throws Exception {
-    String[] className = rebalanceStrategyClass.split("\\.");
-    String PARTICIPANT_PREFIX =
-        className[className.length - 1] + "_node_" + nodeCount + resourceCount;
-    String RESOURCE_PREFIX =
-        className[className.length - 1] + "_resource_" + nodeCount + resourceCount;
-    String CLUSTER_NAME =
-        className[className.length - 1] + nodeCount + resourceCount + "TestingCluster";
-
-    ClusterConfig clusterConfig = new ClusterConfig(CLUSTER_NAME);
-    clusterConfig.setTopologyAwareEnabled(true);
-    clusterConfig.setFaultZoneType("zone");
-    clusterConfig.setTopology("/zone/rack/instance");
-
-    List<InstanceConfig> newInstanceConfigs = new ArrayList<>();
-    Random rand = new Random();
-    for (int i = 0; i < nodeCount; i++) {
-      String nodeName = PARTICIPANT_PREFIX + Math.abs(rand.nextInt()) + "_" + i;
-      String zone = "zone-" + i % faultZone;
-      InstanceConfig newConfig = new InstanceConfig(nodeName);
-      liveInstances.add(nodeName);
-      newConfig.setInstanceEnabled(true);
-      newConfig.setHostName(nodeName);
-      newConfig.setPort(new Integer(i).toString());
-      newConfig.setDomain(String
-          .format("cluster=%s,zone=%s,rack=myRack,instance=%s", CLUSTER_NAME, zone, nodeName));
-      newConfig.setWeight(1000);
-      newConfig.setDelayRebalanceEnabled(false);
-      newConfig.setMaxConcurrentTask(1000);
-      newInstanceConfigs.add(newConfig);
-    }
-
-    Map<String, Map<String, String>> totalMaps = new HashMap<>();
-    Map<String, Integer> partitions = new HashMap<>();
-    List<IdealState> idealStates = new ArrayList<>();
-
-    for (int i = 0; i < resourceCount; i++) {
-      String resourceName = RESOURCE_PREFIX + "_" + i;
-      IdealState idealState = new IdealState(resourceName);
-      idealState.setStateModelDefRef(BuiltInStateModelDefinitions.MasterSlave.name());
-      idealState.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO);
-      idealState.setReplicas(new Integer(replica).toString());
-      idealState.setNumPartitions(partitionCount);
-      idealState.setRebalancerClassName(rebalanceStrategyClass);
-      for (int p = 0; p < partitionCount; p++) {
-        String partitionName = resourceName + "_" + p;
-        idealState.setPreferenceList(partitionName, new ArrayList<String>());
-      }
-      idealStates.add(idealState);
-    }
-
-    long duration = 0;
-    for (IdealState idealState : idealStates) {
-      long startTime = System.currentTimeMillis();
-      Map<String, Map<String, String>> maps = HelixUtil
-          .getIdealAssignmentForFullAuto(clusterConfig, newInstanceConfigs, liveInstances,
-              idealState, new ArrayList<>(idealState.getPartitionSet()), rebalanceStrategyClass);
-      duration += System.currentTimeMillis() - startTime;
-
-      for (String partitionName : idealState.getPartitionSet()) {
-        partitions.put(partitionName, idealState.getReplicaCount(liveInstances.size()));
-      }
-      totalMaps.putAll(maps);
-    }
-
-    //System.out.println("Total running time:\t" + duration);
-
-    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
-    for (InstanceConfig config : newInstanceConfigs) {
-      instanceConfigMap.put(config.getInstanceName(), config);
-    }
-    verifyDistribution(totalMaps, liveInstances, partitions,
-        new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances, instanceConfigMap,
-            clusterConfig));
-    return totalMaps;
-  }
-
-  public Map<String, Map<String, String>> calculate(String Path, String rebalanceStrategyClass,
-      float instanceAdjustRate, Set<String> deltaNode, List<String> liveInstances)
-      throws Exception {
-    instanceFolderPath = Path + "/instanceConfigs/";
-    instanceList = Path + "/instance";
-    idealStateFolderPath = Path + "/idealStates/";
-    idealStateList = Path + "/idealstate";
-    Path path = Paths.get(Path + "/clusterConfig");
-    ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
-    ClusterConfig clusterConfig = new ClusterConfig(record);
-    List<InstanceConfig> instanceConfigs =
-        getInstanceConfigs(instanceFolderPath, instanceList, liveInstances);
-
-    int adjustNodeCount = (int) (instanceAdjustRate > 0 ?
-        Math.ceil(instanceAdjustRate * liveInstances.size()) :
-        Math.floor(instanceAdjustRate * liveInstances.size()));
-
-    if (adjustNodeCount > 0) {
-      for (int i = 0; i < adjustNodeCount; i++) {
-        int cloneIndex = i % (liveInstances.size() - 1);
-        String nodeName = instanceConfigs.get(cloneIndex).getInstanceName() + "_random" + i;
-        liveInstances.add(nodeName);
-        InstanceConfig cloneConfig = new InstanceConfig(nodeName);
-        cloneConfig.setHostName(nodeName);
-        cloneConfig.setInstanceEnabled(true);
-        cloneConfig.setPort(instanceConfigs.get(cloneIndex).getPort());
-        cloneConfig.setDomain(instanceConfigs.get(cloneIndex).getDomain() + "_random" + i);
-        if (instanceConfigs.get(cloneIndex).getWeight() > 0) {
-          cloneConfig.setWeight(instanceConfigs.get(cloneIndex).getWeight());
-        }
-        cloneConfig
-            .setDelayRebalanceEnabled(instanceConfigs.get(cloneIndex).isDelayRebalanceEnabled());
-        if (instanceConfigs.get(cloneIndex).getMaxConcurrentTask() > 0) {
-          cloneConfig.setMaxConcurrentTask(instanceConfigs.get(cloneIndex).getMaxConcurrentTask());
-        }
-        instanceConfigs.add(cloneConfig);
-        deltaNode.add(nodeName);
-      }
-    } else {
-      if (adjustNodeCount > liveInstances.size()) {
-        throw new Exception("All nodes are removed, no assignment possible.");
-      }
-      for (int i = 0; i < Math.abs(adjustNodeCount); i++) {
-        String nodeName = liveInstances.remove(i);
-        deltaNode.add(nodeName);
-      }
-    }
-
-    List<IdealState> idealStates = getIdealStates(idealStateFolderPath, idealStateList);
-    Map<String, Map<String, String>> totalMaps = new HashMap<>();
-    Map<String, Integer> partitions = new HashMap<>();
-
-    long duration = 0;
-    for (int i = 0; i < idealStates.size(); i++) {
-      long startTime = System.currentTimeMillis();
-      int partitionCount = idealStates.get(i).getNumPartitions();
-      List<String> partitionList =
-          new ArrayList<>(idealStates.get(i).getPartitionSet()).subList(0, partitionCount);
-      Map<String, Map<String, String>> maps = HelixUtil
-          .getIdealAssignmentForFullAuto(clusterConfig, instanceConfigs, liveInstances,
-              idealStates.get(i), partitionList, rebalanceStrategyClass);
-      for (String partitionName : partitionList) {
-        partitions.put(partitionName, idealStates.get(i).getReplicaCount(liveInstances.size()));
-      }
-      duration += System.currentTimeMillis() - startTime;
-
-      // print resource details
-/*      Map<String, Set<String>> nodeMapping = convertMapping(maps);
-      String partitionCountsStr = idealStates.get(i).getResourceName();
-      List<String> sortedInstances = new ArrayList<>(liveInstances);
-      Collections.sort(sortedInstances);
-      for (String node : sortedInstances) {
-        partitionCountsStr += "\t" + (nodeMapping.containsKey(node) ? nodeMapping.get(node).size() : 0);
-      }
-      System.out.println(partitionCountsStr);*/
-
-      totalMaps.putAll(maps);
-    }
-    //System.out.println("Takes " + duration + "ms");
-    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
-    for (InstanceConfig config : instanceConfigs) {
-      instanceConfigMap.put(config.getInstanceName(), config);
-    }
-    verifyDistribution(totalMaps, liveInstances, partitions,
-        new Topology(new ArrayList<>(instanceConfigMap.keySet()), liveInstances, instanceConfigMap,
-            clusterConfig));
-    return totalMaps;
-  }
-
-  private void verifyDistribution(Map<String, Map<String, String>> map, List<String> liveInstances,
-      Map<String, Integer> partitionExp, Topology topology) throws Exception {
-    Map<String, Set<String>> faultZonePartition = new HashMap<>();
-    Map<String, String> instanceFaultZone = new HashMap<>();
-    for (Node node : topology.getFaultZones()) {
-      faultZonePartition.put(node.getName(), new HashSet<String>());
-      for (Node instance : Topology.getAllLeafNodes(node)) {
-        instanceFaultZone.put(instance.getName(), node.getName());
-      }
-    }
-    for (String partition : map.keySet()) {
-      // no partition missing, no partition duplicate
-      if (!partitionExp.containsKey(partition) || map.get(partition).size() != partitionExp
-          .get(partition)) {
-        throw new Exception("partition replica in mapping is not as expected");
-      }
-      partitionExp.remove(partition);
-      // no partition on non-live node
-      for (String instance : map.get(partition).keySet()) {
-        if (!liveInstances.contains(instance)) {
-          throw new Exception("assignment is not on a live node!");
-        }
-        // no fault zone conflict
-        String faultZone = instanceFaultZone.get(instance);
-        if (faultZonePartition.get(faultZone).contains(partition)) {
-          throw new Exception("faultzone conflict!");
-        }
-        faultZonePartition.get(faultZone).add(partition);
-      }
-    }
-    if (!partitionExp.isEmpty()) {
-      throw new Exception("partition is not assigned");
-    }
-  }
-
-  private double[] checkEvenness(List<String> liveInstances,
-      Map<String, Map<String, String>> totalMaps, boolean verbose) {
-    StringBuilder output = new StringBuilder();
-    Map<String, List<String>> detailMap = new HashMap<>();
-    Map<String, Integer> distributionMap = new TreeMap<>();
-    Map<String, Integer> topStateDistributionMap = new HashMap<>();
-    for (String instance : liveInstances) {
-      distributionMap.put(instance, 0);
-      topStateDistributionMap.put(instance, 0);
-      detailMap.put(instance, new ArrayList<String>());
-    }
-
-    for (String partition : totalMaps.keySet()) {
-      Map<String, String> instanceMap = totalMaps.get(partition);
-      for (String instance : instanceMap.keySet()) {
-        detailMap.get(instance).add(partition + "-" + totalMaps.get(partition).get(instance));
-        distributionMap.put(instance, distributionMap.get(instance) + 1);
-        if (instanceMap.get(instance).equalsIgnoreCase(topState)) {
-          topStateDistributionMap.put(instance, topStateDistributionMap.get(instance) + 1);
-        }
-      }
-    }
-
-    int totalReplicas = 0;
-    int minR = Integer.MAX_VALUE;
-    int maxR = 0;
-    int mminR = Integer.MAX_VALUE;
-    int mmaxR = 0;
-    for (String instance : distributionMap.keySet()) {
-      output.append(instance + "\t" + distributionMap.get(instance) + "\tpartitions\t"
-          + topStateDistributionMap.get(instance) + "\ttopStates\n");
-      //output.append(instance + "\t:\t" + distributionMap.get(instance) + "\tpartitions\t" + topStateDistributionMap.get(instance) + "\ttopStates\t" + detailMap.get(instance) + "\n");
-      totalReplicas += distributionMap.get(instance);
-      minR = Math.min(minR, distributionMap.get(instance));
-      maxR = Math.max(maxR, distributionMap.get(instance));
-    }
-    for (String instance : topStateDistributionMap.keySet()) {
-      mminR = Math.min(mminR, topStateDistributionMap.get(instance));
-      mmaxR = Math.max(mmaxR, topStateDistributionMap.get(instance));
-    }
-
-    output.append("Maximum holds " + maxR + " replicas and minimum holds " + minR
-        + " replicas, differentiation is " + (double) (maxR - minR) / maxR * 100 + "%\n");
-    output.append("Maximum holds " + mmaxR + " topStates and minimum holds " + mminR
-        + " topStates, differentiation is " + (double) (mmaxR - mminR) / mmaxR * 100 + "%\n ");
-
-    if (verbose) {
-      System.out.println(output.toString());
-    }
-    return new double[] { totalReplicas, minR, maxR, (double) (maxR - minR) / maxR * 100,
-        STDEV(new ArrayList<>(distributionMap.values())), mminR, mmaxR,
-        (double) (mmaxR - mminR) / mmaxR * 100,
-        STDEV(new ArrayList<>(topStateDistributionMap.values()))
-    };
-  }
-
-  private double STDEV(List<Integer> data) {
-    if (data.isEmpty() || data.size() == 1) {
-      return 0;
-    }
-    double totalDiff = 0;
-    double average = 0;
-    for (int num : data) {
-      average += num;
-    }
-    average /= data.size();
-    for (int i = 0; i < data.size(); i++) {
-      totalDiff += Math.pow(data.get(i) - average, 2);
-    }
-    return Math.sqrt(totalDiff) / (data.size() - 1);
-  }
-
-  private static List<InstanceConfig> getInstanceConfigs(String instanceFolderPath,
-      String instanceList, List<String> liveInstances) throws IOException {
-    List<InstanceConfig> instanceConfigs = new ArrayList<>();
-    for (ZNRecord record : getRecords(instanceFolderPath, instanceList)) {
-      instanceConfigs.add(new InstanceConfig(record));
-      liveInstances.add(record.getId());
-    }
-    return instanceConfigs;
-  }
-
-  private static List<IdealState> getIdealStates(String idealStateFolderPath, String idealStateList)
-      throws IOException {
-    List<IdealState> idealStates = new ArrayList<>();
-    for (ZNRecord record : getRecords(idealStateFolderPath, idealStateList)) {
-      IdealState idealState = new IdealState(record);
-      try {
-        BuiltInStateModelDefinitions.valueOf(idealState.getStateModelDefRef());
-      } catch (IllegalArgumentException ex) {
-        idealState.setStateModelDefRef("OnlineOffline");
-      }
-      idealStates.add(idealState);
-    }
-    return idealStates;
-  }
-
-  private static List<ZNRecord> getRecords(String folderPath, String list) throws IOException {
-    List<ZNRecord> records = new ArrayList<>();
-    List<String> names = new ArrayList<>();
-    try (BufferedReader br = new BufferedReader(new FileReader(list))) {
-      String sCurrentLine = br.readLine();
-      names.addAll(Arrays.asList(sCurrentLine.split(" ")));
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    for (String name : names) {
-      Path path = Paths.get(folderPath + name);
-      ZNRecord record = (ZNRecord) new ZNRecordSerializer().deserialize(Files.readAllBytes(path));
-      records.add(record);
-    }
-    return records;
-  }
-
-  private Map<String, Map<String, Integer>> getStateCount(Map<String, Map<String, String>> map) {
-    Map<String, Map<String, Integer>> mapStateCount = new HashMap<>();
-    for (String partition : map.keySet()) {
-      Map<String, Integer> stateCount = new HashMap<>();
-      mapStateCount.put(partition, stateCount);
-      for (String node : map.get(partition).keySet()) {
-        String state = map.get(partition).get(node);
-        if (!stateCount.containsKey(state)) {
-          stateCount.put(state, 1);
-        } else {
-          stateCount.put(state, stateCount.get(state) + 1);
-        }
-      }
-    }
-    return mapStateCount;
-  }
-
-  private void verifyMaps(Map<String, Map<String, String>> map,
-      Map<String, Map<String, String>> newMap) throws Exception {
-    // check no partition lose
-    Map<String, Map<String, Integer>> mapStateCount = getStateCount(map);
-    Map<String, Map<String, Integer>> newMapStateCount = getStateCount(newMap);
-    for (String partition : mapStateCount.keySet()) {
-      if (!newMapStateCount.containsKey(partition)) {
-        throw new Exception("mapping does not match");
-      }
-      for (String state : mapStateCount.get(partition).keySet()) {
-        if (!newMapStateCount.get(partition).containsKey(state)
-            || mapStateCount.get(partition).get(state) != newMapStateCount.get(partition)
-            .get(state)) {
-          throw new Exception("state does not match");
-        }
-      }
-      for (String state : newMapStateCount.get(partition).keySet()) {
-        if (!mapStateCount.get(partition).containsKey(state)) {
-          throw new Exception("state does not match");
-        }
-      }
-    }
-    for (String partition : newMapStateCount.keySet()) {
-      if (!mapStateCount.containsKey(partition)) {
-        throw new Exception("mapping does not match");
-      }
-    }
-  }
-
-  private double[] checkMovement(Map<String, Map<String, String>> map,
-      Map<String, Map<String, String>> newMap, Collection<String> deltaNodes, boolean verbose)
-      throws Exception {
-    verifyMaps(map, newMap);
-    int totalChange = 0;
-    int totalTopStateChange = 0;
-    int totalTopStateChangeWithNewDeployment = 0;
-    int totalPartition = 0;
-    int totalTopState = 0;
-
-    for (String partition : map.keySet()) {
-      Map<String, String> origStates = map.get(partition);
-      Map<String, String> newStates = newMap.get(partition);
-      String topStateNode = "", newtopStateNode = "";
-      for (String node : origStates.keySet()) {
-        if (origStates.get(node).equalsIgnoreCase(topState)) {
-          topStateNode = node;
-        }
-      }
-      for (String node : newStates.keySet()) {
-        if (newStates.get(node).equalsIgnoreCase(topState)) {
-          newtopStateNode = node;
-          totalTopState++;
-        }
-      }
-      if (!topStateNode.equalsIgnoreCase(newtopStateNode)) {
-        totalTopStateChange++;
-        if (!origStates.containsKey(newtopStateNode)) {
-          totalTopStateChangeWithNewDeployment++;
-        }
-      }
-    }
-
-    Map<String, Set<String>> list = convertMapping(map);
-    Map<String, Set<String>> newList = convertMapping(newMap);
-
-    Map<String, Integer> addition = new HashMap<>();
-    Map<String, Integer> subtraction = new HashMap<>();
-    for (String instance : newList.keySet()) {
-      Set<String> oldPartitions = list.get(instance);
-      Set<String> newPartitions = newList.get(instance);
-      totalPartition += newPartitions.size();
-      if (oldPartitions == null) {
-        addition.put(instance, newPartitions.size());
-      } else {
-        Set<String> commonPartitions = new HashSet<>(newPartitions);
-        commonPartitions.retainAll(oldPartitions);
-
-        newPartitions.removeAll(commonPartitions);
-
-        addition.put(instance, newPartitions.size());
-
-        oldPartitions.removeAll(commonPartitions);
-        subtraction.put(instance, oldPartitions.size());
-      }
-      totalChange += newPartitions.size();
-      //System.out.println("Changed partition on node: \t" + instance + "\t: \t" + newPartitions.toString());
-    }
-    /*
-      List<String> instances = new ArrayList<>(newList.keySet());
-      Collections.sort(instances);
-      System.out.println("Addition partition count: ");
-      for (String instance : instances) {
-        System.out.println(addition.containsKey(instance) ? addition.get(instance) : 0);
-      }
-
-      System.out.println("Subtraction partition count: ");
-      for (String instance : instances) {
-        System.out.println(subtraction.containsKey(instance) ? subtraction.get(instance) : 0);
-      }
-    */
-
-    int nodeChanged = 0;
-    int necessaryChange = 0;
-    int necessarytopStateChange = 0;
-    for (String instance : deltaNodes) {
-      nodeChanged++;
-      if (list.containsKey(instance)) {
-        necessaryChange += list.get(instance).size();
-        for (Map<String, String> nodeState : map.values()) {
-          if (nodeState.containsKey(instance)) {
-            if (nodeState.get(instance).equalsIgnoreCase(topState)) {
-              necessarytopStateChange++;
-            }
-          }
-        }
-      }
-      if (newList.containsKey(instance)) {
-        necessaryChange += newList.get(instance).size();
-        for (Map<String, String> nodeState : newMap.values()) {
-          if (nodeState.containsKey(instance)) {
-            if (nodeState.get(instance).equalsIgnoreCase(topState)) {
-              necessarytopStateChange++;
-            }
-          }
-        }
-      }
-    }
-
-    if (verbose) {
-      System.out.println(
-          "\t\t\t" + "Total partition change count: \t" + totalChange + "\t/\t" + totalPartition
-              + "\t, rate: \t" + (((float) totalChange) / totalPartition * 100) + "%\t"
-              + "Diff nodes have partition \t" + necessaryChange + "\t, unnecessary change rate: \t"
-              + (((float) totalChange - necessaryChange) / totalPartition * 100)
-              + "%\t, which is \t" + (((float) totalChange - necessaryChange) / totalChange * 100)
-              + "%\t of the movement.");
-    }
-
-    double expectedAverageMv =
-        (double) totalPartition / Math.max(list.size(), newList.size()) * deltaNodes.size();
-
-    return new double[] { nodeChanged, totalChange, (((double) totalChange) / totalPartition * 100),
-        (((double) totalChange - necessaryChange) / totalPartition * 100), totalTopStateChange,
-        (((double) totalTopStateChange) / totalTopState * 100),
-        (((double) totalTopStateChange - necessarytopStateChange) / totalTopState * 100),
-        expectedAverageMv, (((double) totalChange - expectedAverageMv) / totalPartition * 100),
-        totalTopStateChangeWithNewDeployment,
-        (((double) totalTopStateChangeWithNewDeployment) / totalTopState * 100)
-    };
-  }
-
-  private Map<String, Set<String>> convertMapping(Map<String, Map<String, String>> map) {
-    Map<String, Set<String>> list = new HashMap<>();
-    for (String partition : map.keySet()) {
-      for (String instance : map.get(partition).keySet()) {
-        if (!list.containsKey(instance)) {
-          list.put(instance, new HashSet<String>());
-        }
-        list.get(instance).add(partition);
-      }
-    }
-    return list;
-  }
-}


[34/50] [abbrv] helix git commit: HELIX-661: implement GET namespace(s)

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/AbstractHelixResource.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/AbstractHelixResource.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/AbstractHelixResource.java
new file mode 100644
index 0000000..1eac9c2
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/AbstractHelixResource.java
@@ -0,0 +1,79 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.io.IOException;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.server.ServerContext;
+import org.apache.helix.rest.server.resources.AbstractResource;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.tools.ClusterSetup;
+
+
+/**
+ * This class provides methods to access Helix specific objects
+ * such as cluster, instance, job, resource, workflow, etc in
+ * metadata store.
+ */
+public class AbstractHelixResource extends AbstractResource{
+
+  public ZkClient getZkClient() {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getZkClient();
+  }
+
+  public HelixAdmin getHelixAdmin() {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getHelixAdmin();
+  }
+
+  public ClusterSetup getClusterSetup() {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getClusterSetup();
+  }
+
+  public TaskDriver getTaskDriver(String clusterName) {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getTaskDriver(clusterName);
+  }
+
+  public ConfigAccessor getConfigAccessor() {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getConfigAccessor();
+  }
+
+  public HelixDataAccessor getDataAccssor(String clusterName) {
+    ServerContext serverContext = getServerContext();
+    return serverContext.getDataAccssor(clusterName);
+  }
+
+  protected static ZNRecord toZNRecord(String data) throws IOException {
+    return OBJECT_MAPPER.reader(ZNRecord.class).readValue(data);
+  }
+
+  private ServerContext getServerContext() {
+    return (ServerContext) _application.getProperties().get(ContextPropertyKeys.SERVER_CONTEXT.name());
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
new file mode 100644
index 0000000..f6f95b0
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ClusterAccessor.java
@@ -0,0 +1,400 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZKUtil;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.LeaderHistory;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.helix.tools.ClusterSetup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Path("/clusters")
+public class ClusterAccessor extends AbstractHelixResource {
+  private static Logger _logger = LoggerFactory.getLogger(ClusterAccessor.class.getName());
+
+  public enum ClusterProperties {
+    controller,
+    instances,
+    liveInstances,
+    resources,
+    paused,
+    maintenance,
+    messages,
+    stateModelDefinitions,
+    clusters
+  }
+
+  @GET
+  public Response getClusters() {
+    HelixAdmin helixAdmin = getHelixAdmin();
+    List<String> clusters = helixAdmin.getClusters();
+
+    Map<String, List<String>> dataMap = new HashMap<>();
+    dataMap.put(ClusterProperties.clusters.name(), clusters);
+
+    return JSONRepresentation(dataMap);
+  }
+
+  @GET
+  @Path("{clusterId}")
+  public Response getClusterInfo(@PathParam("clusterId") String clusterId) {
+    if (!isClusterExist(clusterId)) {
+      return notFound();
+    }
+
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
+
+    Map<String, Object> clusterInfo = new HashMap<>();
+    clusterInfo.put(Properties.id.name(), clusterId);
+
+    LiveInstance controller = dataAccessor.getProperty(keyBuilder.controllerLeader());
+    if (controller != null) {
+      clusterInfo.put(ClusterProperties.controller.name(), controller.getInstanceName());
+    } else {
+      clusterInfo.put(ClusterProperties.controller.name(), "No Lead Controller!");
+    }
+
+    boolean paused = (dataAccessor.getProperty(keyBuilder.pause()) == null ? false : true);
+    clusterInfo.put(ClusterProperties.paused.name(), paused);
+    boolean maintenance =
+        (dataAccessor.getProperty(keyBuilder.maintenance()) == null ? false : true);
+    clusterInfo.put(ClusterProperties.maintenance.name(), maintenance);
+
+    List<String> idealStates = dataAccessor.getChildNames(keyBuilder.idealStates());
+    clusterInfo.put(ClusterProperties.resources.name(), idealStates);
+    List<String> instances = dataAccessor.getChildNames(keyBuilder.instanceConfigs());
+    clusterInfo.put(ClusterProperties.instances.name(), instances);
+    List<String> liveInstances = dataAccessor.getChildNames(keyBuilder.liveInstances());
+    clusterInfo.put(ClusterProperties.liveInstances.name(), liveInstances);
+
+    return JSONRepresentation(clusterInfo);
+  }
+
+
+  @PUT
+  @Path("{clusterId}")
+  public Response createCluster(@PathParam("clusterId") String clusterId,
+      @DefaultValue("false") @QueryParam("recreate") String recreate) {
+    boolean recreateIfExists = Boolean.valueOf(recreate);
+    ClusterSetup clusterSetup = getClusterSetup();
+
+    try {
+      clusterSetup.addCluster(clusterId, recreateIfExists);
+    } catch (Exception ex) {
+      _logger.error("Failed to create cluster " + clusterId + ", exception: " + ex);
+      return serverError(ex);
+    }
+
+    return created();
+  }
+
+  @DELETE
+  @Path("{clusterId}")
+  public Response deleteCluster(@PathParam("clusterId") String clusterId) {
+    ClusterSetup clusterSetup = getClusterSetup();
+
+    try {
+      clusterSetup.deleteCluster(clusterId);
+    } catch (HelixException ex) {
+      _logger.info(
+          "Failed to delete cluster " + clusterId + ", cluster is still in use. Exception: " + ex);
+      return badRequest(ex.getMessage());
+    } catch (Exception ex) {
+      _logger.error("Failed to delete cluster " + clusterId + ", exception: " + ex);
+      return serverError(ex);
+    }
+
+    return OK();
+  }
+
+  @POST
+  @Path("{clusterId}")
+  public Response updateCluster(@PathParam("clusterId") String clusterId,
+      @QueryParam("command") String commandStr, @QueryParam("superCluster") String superCluster,
+      String content) {
+    Command command;
+    try {
+      command = getCommand(commandStr);
+    } catch (HelixException ex) {
+      return badRequest(ex.getMessage());
+    }
+
+    ClusterSetup clusterSetup = getClusterSetup();
+    HelixAdmin helixAdmin = getHelixAdmin();
+
+    switch (command) {
+    case activate:
+      if (superCluster == null) {
+        return badRequest("Super Cluster name is missing!");
+      }
+      try {
+        clusterSetup.activateCluster(clusterId, superCluster, true);
+      } catch (Exception ex) {
+        _logger.error("Failed to add cluster " + clusterId + " to super cluster " + superCluster);
+        return serverError(ex);
+      }
+      break;
+
+    case expand:
+      try {
+        clusterSetup.expandCluster(clusterId);
+      } catch (Exception ex) {
+        _logger.error("Failed to expand cluster " + clusterId);
+        return serverError(ex);
+      }
+      break;
+
+    case enable:
+      try {
+        helixAdmin.enableCluster(clusterId, true);
+      } catch (Exception ex) {
+        _logger.error("Failed to enable cluster " + clusterId);
+        return serverError(ex);
+      }
+      break;
+
+    case disable:
+      try {
+        helixAdmin.enableCluster(clusterId, false);
+      } catch (Exception ex) {
+        _logger.error("Failed to disable cluster " + clusterId);
+        return serverError(ex);
+      }
+      break;
+    case enableMaintenanceMode:
+      try {
+        helixAdmin.enableMaintenanceMode(clusterId, true, content);
+      } catch (Exception ex) {
+        _logger.error("Failed to enable maintenance mode " + clusterId);
+        return serverError(ex);
+      }
+      break;
+    case disableMaintenanceMode:
+      try {
+        helixAdmin.enableMaintenanceMode(clusterId, false);
+      } catch (Exception ex) {
+        _logger.error("Failed to disable maintenance mode " + clusterId);
+        return serverError(ex);
+      }
+      break;
+    default:
+      return badRequest("Unsupported command " + command);
+    }
+
+    return OK();
+  }
+
+
+  @GET
+  @Path("{clusterId}/configs")
+  public Response getClusterConfig(@PathParam("clusterId") String clusterId) {
+    ConfigAccessor accessor = getConfigAccessor();
+    ClusterConfig config = null;
+    try {
+      config = accessor.getClusterConfig(clusterId);
+    } catch (HelixException ex) {
+      // cluster not found.
+      _logger.info("Failed to get cluster config for cluster " + clusterId
+          + ", cluster not found, Exception: " + ex);
+    } catch (Exception ex) {
+      _logger.error("Failed to get cluster config for cluster " + clusterId + " Exception: " + ex);
+      return serverError(ex);
+    }
+    if (config == null) {
+      return notFound();
+    }
+    return JSONRepresentation(config.getRecord());
+  }
+
+  @POST
+  @Path("{clusterId}/configs")
+  public Response updateClusterConfig(
+      @PathParam("clusterId") String clusterId, @QueryParam("command") String commandStr,
+      String content) {
+    Command command;
+    try {
+      command = getCommand(commandStr);
+    } catch (HelixException ex) {
+      return badRequest(ex.getMessage());
+    }
+
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a valid ZNRecord!");
+    }
+
+    if (!record.getId().equals(clusterId)) {
+      return badRequest("ID does not match the cluster name in input!");
+    }
+
+    ClusterConfig config = new ClusterConfig(record);
+    ConfigAccessor configAccessor = getConfigAccessor();
+    try {
+      switch (command) {
+      case update:
+        configAccessor.updateClusterConfig(clusterId, config);
+        break;
+      case delete: {
+        HelixConfigScope clusterScope =
+            new HelixConfigScopeBuilder(HelixConfigScope.ConfigScopeProperty.CLUSTER)
+                .forCluster(clusterId).build();
+        configAccessor.remove(clusterScope, config.getRecord());
+        }
+        break;
+
+      default:
+        return badRequest("Unsupported command " + commandStr);
+      }
+    } catch (HelixException ex) {
+      return notFound(ex.getMessage());
+    } catch (Exception ex) {
+      _logger.error(
+          "Failed to " + command + " cluster config, cluster " + clusterId + " new config: "
+              + content + ", Exception: " + ex);
+      return serverError(ex);
+    }
+    return OK();
+  }
+
+  @GET
+  @Path("{clusterId}/controller")
+  public Response getClusterController(@PathParam("clusterId") String clusterId) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    Map<String, Object> controllerInfo = new HashMap<>();
+    controllerInfo.put(Properties.id.name(), clusterId);
+
+    LiveInstance leader = dataAccessor.getProperty(dataAccessor.keyBuilder().controllerLeader());
+    if (leader != null) {
+      controllerInfo.put(ClusterProperties.controller.name(), leader.getInstanceName());
+      controllerInfo.putAll(leader.getRecord().getSimpleFields());
+    } else {
+      controllerInfo.put(ClusterProperties.controller.name(), "No Lead Controller!");
+    }
+
+    return JSONRepresentation(controllerInfo);
+  }
+
+  @GET
+  @Path("{clusterId}/controller/history")
+  public Response getClusterControllerHistory(@PathParam("clusterId") String clusterId) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    Map<String, Object> controllerHistory = new HashMap<>();
+    controllerHistory.put(Properties.id.name(), clusterId);
+
+    LeaderHistory history =
+        dataAccessor.getProperty(dataAccessor.keyBuilder().controllerLeaderHistory());
+    if (history != null) {
+      controllerHistory.put(Properties.history.name(), history.getHistoryList());
+    } else {
+      controllerHistory.put(Properties.history.name(), Collections.emptyList());
+    }
+
+    return JSONRepresentation(controllerHistory);
+  }
+
+  @GET
+  @Path("{clusterId}/controller/messages")
+  public Response getClusterControllerMessages(@PathParam("clusterId") String clusterId) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+
+    Map<String, Object> controllerMessages = new HashMap<>();
+    controllerMessages.put(Properties.id.name(), clusterId);
+
+    List<String> messages =
+        dataAccessor.getChildNames(dataAccessor.keyBuilder().controllerMessages());
+    controllerMessages.put(ClusterProperties.messages.name(), messages);
+    controllerMessages.put(Properties.count.name(), messages.size());
+
+    return JSONRepresentation(controllerMessages);
+  }
+
+  @GET
+  @Path("{clusterId}/controller/messages/{messageId}")
+  public Response getClusterControllerMessages(@PathParam("clusterId") String clusterId, @PathParam("messageId") String messageId) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    Message message = dataAccessor.getProperty(
+        dataAccessor.keyBuilder().controllerMessage(messageId));
+    return JSONRepresentation(message.getRecord());
+  }
+
+  @GET
+  @Path("{clusterId}/statemodeldefs")
+  public Response getClusterStateModelDefinitions(@PathParam("clusterId") String clusterId) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    List<String> stateModelDefs =
+        dataAccessor.getChildNames(dataAccessor.keyBuilder().stateModelDefs());
+
+    Map<String, Object> clusterStateModelDefs = new HashMap<>();
+    clusterStateModelDefs.put(Properties.id.name(), clusterId);
+    clusterStateModelDefs.put(ClusterProperties.stateModelDefinitions.name(), stateModelDefs);
+
+    return JSONRepresentation(clusterStateModelDefs);
+  }
+
+  @GET
+  @Path("{clusterId}/statemodeldefs/{statemodel}")
+  public Response getClusterStateModelDefinition(@PathParam("clusterId") String clusterId,
+      @PathParam("statemodel") String statemodel) {
+    HelixDataAccessor dataAccessor = getDataAccssor(clusterId);
+    StateModelDefinition stateModelDef =
+        dataAccessor.getProperty(dataAccessor.keyBuilder().stateModelDef(statemodel));
+
+    return JSONRepresentation(stateModelDef.getRecord());
+  }
+
+  private boolean isClusterExist(String cluster) {
+    ZkClient zkClient = getZkClient();
+    if (ZKUtil.isClusterSetup(cluster, zkClient)) {
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
new file mode 100644
index 0000000..2748dea
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/InstanceAccessor.java
@@ -0,0 +1,545 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.Error;
+import org.apache.helix.model.HealthStat;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.ParticipantHistory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.JsonNodeFactory;
+import org.codehaus.jackson.node.ObjectNode;
+
+@Path("/clusters/{clusterId}/instances")
+public class InstanceAccessor extends AbstractHelixResource {
+  private final static Logger _logger = LoggerFactory.getLogger(InstanceAccessor.class);
+
+  public enum InstanceProperties {
+    instances,
+    online,
+    disabled,
+    config,
+    liveInstance,
+    resource,
+    resources,
+    partitions,
+    errors,
+    new_messages,
+    read_messages,
+    total_message_count,
+    read_message_count,
+    healthreports,
+    instanceTags
+  }
+
+  @GET
+  public Response getInstances(@PathParam("clusterId") String clusterId) {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), JsonNodeFactory.instance.textNode(clusterId));
+
+    ArrayNode instancesNode = root.putArray(InstanceProperties.instances.name());
+    ArrayNode onlineNode = root.putArray(InstanceProperties.online.name());
+    ArrayNode disabledNode = root.putArray(InstanceProperties.disabled.name());
+
+    List<String> instances = accessor.getChildNames(accessor.keyBuilder().instanceConfigs());
+
+    if (instances != null) {
+      instancesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(instances));
+    } else {
+      return notFound();
+    }
+
+    List<String> liveInstances = accessor.getChildNames(accessor.keyBuilder().liveInstances());
+    ClusterConfig clusterConfig = accessor.getProperty(accessor.keyBuilder().clusterConfig());
+
+    for (String instanceName : instances) {
+      InstanceConfig instanceConfig =
+          accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
+      if (instanceConfig != null) {
+        if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
+            && clusterConfig.getDisabledInstances().containsKey(instanceName))) {
+          disabledNode.add(JsonNodeFactory.instance.textNode(instanceName));
+        }
+
+        if (liveInstances.contains(instanceName)){
+          onlineNode.add(JsonNodeFactory.instance.textNode(instanceName));
+        }
+      }
+    }
+
+    return JSONRepresentation(root);
+  }
+
+  @POST
+  public Response updateInstances(@PathParam("clusterId") String clusterId,
+      @QueryParam("command") String command, String content) {
+    Command cmd;
+    try {
+      cmd = Command.valueOf(command);
+    } catch (Exception e) {
+      return badRequest("Invalid command : " + command);
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      JsonNode node = null;
+      if (content.length() != 0) {
+        node = OBJECT_MAPPER.readTree(content);
+      }
+      if (node == null) {
+        return badRequest("Invalid input for content : " + content);
+      }
+      List<String> enableInstances = OBJECT_MAPPER
+          .readValue(node.get(InstanceProperties.instances.name()).toString(),
+              OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class));
+      switch (cmd) {
+      case enable:
+        admin.enableInstance(clusterId, enableInstances, true);
+
+        break;
+      case disable:
+        admin.enableInstance(clusterId, enableInstances, false);
+        break;
+      default:
+        _logger.error("Unsupported command :" + command);
+        return badRequest("Unsupported command :" + command);
+      }
+    } catch (Exception e) {
+      _logger.error("Failed in updating instances : " + content, e);
+      return badRequest(e.getMessage());
+    }
+    return OK();
+  }
+
+  @GET
+  @Path("{instanceName}")
+  public Response getInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    Map<String, Object> instanceMap = new HashMap<>();
+    instanceMap.put(Properties.id.name(), JsonNodeFactory.instance.textNode(instanceName));
+    instanceMap.put(InstanceProperties.liveInstance.name(), null);
+
+    InstanceConfig instanceConfig =
+        accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
+    LiveInstance liveInstance =
+        accessor.getProperty(accessor.keyBuilder().liveInstance(instanceName));
+
+    if (instanceConfig != null) {
+      instanceMap.put(InstanceProperties.config.name(), instanceConfig.getRecord());
+    } else {
+      return notFound();
+    }
+
+    if (liveInstance != null) {
+      instanceMap.put(InstanceProperties.liveInstance.name(), liveInstance.getRecord());
+    }
+
+    return JSONRepresentation(instanceMap);
+  }
+
+  @PUT
+  @Path("{instanceName}")
+  public Response addInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName, String content) {
+    HelixAdmin admin = getHelixAdmin();
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a vaild ZNRecord!");
+    }
+
+    try {
+      admin.addInstance(clusterId, new InstanceConfig(record));
+    } catch (Exception ex) {
+      _logger.error("Error in adding an instance: " + instanceName, ex);
+      return serverError(ex);
+    }
+
+    return OK();
+  }
+
+  @POST
+  @Path("{instanceName}")
+  public Response updateInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName, @QueryParam("command") String command,
+      String content) {
+    Command cmd;
+    try {
+      cmd = Command.valueOf(command);
+    } catch (Exception e) {
+      return badRequest("Invalid command : " + command);
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      JsonNode node = null;
+      if (content.length() != 0) {
+        node = OBJECT_MAPPER.readTree(content);
+      }
+
+      switch (cmd) {
+      case enable:
+        admin.enableInstance(clusterId, instanceName, true);
+        break;
+      case disable:
+        admin.enableInstance(clusterId, instanceName, false);
+        break;
+      case reset:
+        if (!validInstance(node, instanceName)) {
+          return badRequest("Instance names are not match!");
+        }
+        admin.resetPartition(clusterId, instanceName,
+            node.get(InstanceProperties.resource.name()).toString(), (List<String>) OBJECT_MAPPER
+                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
+                    OBJECT_MAPPER.getTypeFactory()
+                        .constructCollectionType(List.class, String.class)));
+        break;
+      case addInstanceTag:
+        if (!validInstance(node, instanceName)) {
+          return badRequest("Instance names are not match!");
+        }
+        for (String tag : (List<String>) OBJECT_MAPPER
+            .readValue(node.get(InstanceProperties.instanceTags.name()).toString(),
+                OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class))) {
+          admin.addInstanceTag(clusterId, instanceName, tag);
+        }
+        break;
+      case removeInstanceTag:
+        if (!validInstance(node, instanceName)) {
+          return badRequest("Instance names are not match!");
+        }
+        for (String tag : (List<String>) OBJECT_MAPPER
+            .readValue(node.get(InstanceProperties.instanceTags.name()).toString(),
+                OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class))) {
+          admin.removeInstanceTag(clusterId, instanceName, tag);
+        }
+        break;
+      case enablePartitions:
+        admin.enablePartition(true, clusterId, instanceName,
+            node.get(InstanceProperties.resource.name()).getTextValue(),
+            (List<String>) OBJECT_MAPPER
+                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
+                    OBJECT_MAPPER.getTypeFactory()
+                        .constructCollectionType(List.class, String.class)));
+        break;
+      case disablePartitions:
+        admin.enablePartition(false, clusterId, instanceName,
+            node.get(InstanceProperties.resource.name()).getTextValue(),
+            (List<String>) OBJECT_MAPPER
+                .readValue(node.get(InstanceProperties.partitions.name()).toString(),
+                    OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class)));
+        break;
+      default:
+        _logger.error("Unsupported command :" + command);
+        return badRequest("Unsupported command :" + command);
+      }
+    } catch (Exception e) {
+      _logger.error("Failed in updating instance : " + instanceName, e);
+      return badRequest(e.getMessage());
+    }
+    return OK();
+  }
+
+  @DELETE
+  @Path("{instanceName}")
+  public Response deleteInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) {
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      InstanceConfig instanceConfig = admin.getInstanceConfig(clusterId, instanceName);
+      admin.dropInstance(clusterId, instanceConfig);
+    } catch (HelixException e) {
+      return badRequest(e.getMessage());
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{instanceName}/configs")
+  public Response getInstanceConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    InstanceConfig instanceConfig =
+        accessor.getProperty(accessor.keyBuilder().instanceConfig(instanceName));
+
+    if (instanceConfig != null) {
+      return JSONRepresentation(instanceConfig.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @PUT
+  @Path("{instanceName}/configs")
+  public Response updateInstanceConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName, String content) throws IOException {
+    HelixAdmin admin = getHelixAdmin();
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a vaild ZNRecord!");
+    }
+
+    try {
+      admin.setInstanceConfig(clusterId, instanceName, new InstanceConfig(record));
+    } catch (Exception ex) {
+      _logger.error("Error in update instance config: " + instanceName, ex);
+      return serverError(ex);
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{instanceName}/resources")
+  public Response getResourcesOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), instanceName);
+    ArrayNode resourcesNode = root.putArray(InstanceProperties.resources.name());
+
+    List<String> sessionIds = accessor.getChildNames(accessor.keyBuilder().sessions(instanceName));
+    if (sessionIds == null || sessionIds.size() == 0) {
+      return null;
+    }
+
+    // Only get resource list from current session id
+    String currentSessionId = sessionIds.get(0);
+
+    List<String> resources =
+        accessor.getChildNames(accessor.keyBuilder().currentStates(instanceName, currentSessionId));
+    if (resources != null && resources.size() > 0) {
+      resourcesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(resources));
+    }
+
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{instanceName}/resources/{resourceName}")
+  public Response getResourceOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName,
+      @PathParam("resourceName") String resourceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    List<String> sessionIds = accessor.getChildNames(accessor.keyBuilder().sessions(instanceName));
+    if (sessionIds == null || sessionIds.size() == 0) {
+      return notFound();
+    }
+
+    // Only get resource list from current session id
+    String currentSessionId = sessionIds.get(0);
+    CurrentState resourceCurrentState = accessor
+        .getProperty(accessor.keyBuilder().currentState(instanceName, currentSessionId, resourceName));
+    if (resourceCurrentState != null) {
+      return JSONRepresentation(resourceCurrentState.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @GET
+  @Path("{instanceName}/errors")
+  public Response getErrorsOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), instanceName);
+    ObjectNode errorsNode = JsonNodeFactory.instance.objectNode();
+
+    List<String> sessionIds =
+        accessor.getChildNames(accessor.keyBuilder().errors(instanceName));
+
+    if (sessionIds == null || sessionIds.size() == 0) {
+      return notFound();
+    }
+
+    for (String sessionId : sessionIds) {
+      List<String> resources =
+          accessor.getChildNames(accessor.keyBuilder().errors(instanceName, sessionId));
+      if (resources != null) {
+        ObjectNode resourcesNode = JsonNodeFactory.instance.objectNode();
+        for (String resourceName : resources) {
+          List<String> partitions = accessor
+              .getChildNames(accessor.keyBuilder().errors(instanceName, sessionId, resourceName));
+          if (partitions != null) {
+            ArrayNode partitionsNode = resourcesNode.putArray(resourceName);
+            partitionsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(partitions));
+          }
+        }
+        errorsNode.put(sessionId, resourcesNode);
+      }
+    }
+    root.put(InstanceProperties.errors.name(), errorsNode);
+
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{instanceName}/errors/{sessionId}/{resourceName}/{partitionName}")
+  public Response getErrorsOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName, @PathParam("sessionId") String sessionId,
+      @PathParam("resourceName") String resourceName,
+      @PathParam("partitionName") String partitionName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    Error error = accessor.getProperty(accessor.keyBuilder()
+        .stateTransitionError(instanceName, sessionId, resourceName, partitionName));
+    if (error != null) {
+      return JSONRepresentation(error.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @GET
+  @Path("{instanceName}/history")
+  public Response getHistoryOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    ParticipantHistory history =
+        accessor.getProperty(accessor.keyBuilder().participantHistory(instanceName));
+    if (history != null) {
+      return JSONRepresentation(history.getRecord());
+    }
+    return notFound();
+  }
+
+  @GET
+  @Path("{instanceName}/messages")
+  public Response getMessagesOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), instanceName);
+    ArrayNode newMessages = root.putArray(InstanceProperties.new_messages.name());
+    ArrayNode readMessages = root.putArray(InstanceProperties.read_messages.name());
+
+
+    List<String> messages =
+        accessor.getChildNames(accessor.keyBuilder().messages(instanceName));
+    if (messages == null || messages.size() == 0) {
+      return notFound();
+    }
+
+    for (String messageName : messages) {
+      Message message = accessor.getProperty(accessor.keyBuilder().message(instanceName, messageName));
+      if (message.getMsgState() == Message.MessageState.NEW) {
+        newMessages.add(messageName);
+      }
+
+      if (message.getMsgState() == Message.MessageState.READ) {
+        readMessages.add(messageName);
+      }
+    }
+
+    root.put(InstanceProperties.total_message_count.name(),
+        newMessages.size() + readMessages.size());
+    root.put(InstanceProperties.read_message_count.name(), readMessages.size());
+
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{instanceName}/messages/{messageId}")
+  public Response getMessageOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName,
+      @PathParam("messageId") String messageId) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    Message message = accessor.getProperty(accessor.keyBuilder().message(instanceName, messageId));
+    if (message != null) {
+      return JSONRepresentation(message.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @GET
+  @Path("{instanceName}/healthreports")
+  public Response getHealthReportsOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), instanceName);
+    ArrayNode healthReportsNode = root.putArray(InstanceProperties.healthreports.name());
+
+    List<String> healthReports =
+        accessor.getChildNames(accessor.keyBuilder().healthReports(instanceName));
+
+    if (healthReports != null && healthReports.size() > 0) {
+      healthReportsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(healthReports));
+    }
+
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{instanceName}/healthreports/{reportName}")
+  public Response getHealthReportsOnInstance(@PathParam("clusterId") String clusterId,
+      @PathParam("instanceName") String instanceName,
+      @PathParam("reportName") String reportName) throws IOException {
+    HelixDataAccessor accessor = getDataAccssor(clusterId);
+    HealthStat healthStat =
+        accessor.getProperty(accessor.keyBuilder().healthReport(instanceName, reportName));
+    if (healthStat != null) {
+      return JSONRepresentation(healthStat);
+    }
+
+    return notFound();
+  }
+
+  private boolean validInstance(JsonNode node, String instanceName) {
+    return instanceName.equals(node.get(Properties.id.name()).getValueAsText());
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
new file mode 100644
index 0000000..2d27f51
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/JobAccessor.java
@@ -0,0 +1,200 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.core.Response;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobContext;
+import org.apache.helix.task.TaskConfig;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.WorkflowConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.JsonNodeFactory;
+import org.codehaus.jackson.node.ObjectNode;
+
+@Path("/clusters/{clusterId}/workflows/{workflowName}/jobs")
+public class JobAccessor extends AbstractHelixResource {
+  private static Logger _logger = LoggerFactory.getLogger(JobAccessor.class.getName());
+
+  public enum JobProperties {
+    Jobs,
+    JobConfig,
+    JobContext,
+    TASK_COMMAND
+  }
+
+  @GET
+  public Response getJobs(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName) {
+    TaskDriver driver = getTaskDriver(clusterId);
+    WorkflowConfig workflowConfig = driver.getWorkflowConfig(workflowName);
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+
+    if (workflowConfig == null) {
+      return badRequest(String.format("Workflow %s is not found!", workflowName));
+    }
+
+    Set<String> jobs = workflowConfig.getJobDag().getAllNodes();
+    root.put(Properties.id.name(), JobProperties.Jobs.name());
+    ArrayNode jobsNode = root.putArray(JobProperties.Jobs.name());
+
+    if (jobs != null) {
+      jobsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(jobs));
+    }
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{jobName}")
+  public Response getJob(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
+    TaskDriver driver = getTaskDriver(clusterId);
+    Map<String, ZNRecord> jobMap = new HashMap<>();
+
+
+    JobConfig jobConfig = driver.getJobConfig(jobName);
+    if (jobConfig != null) {
+      jobMap.put(JobProperties.JobConfig.name(), jobConfig.getRecord());
+    } else {
+      return badRequest(String.format("Job config for %s does not exists", jobName));
+    }
+
+    JobContext jobContext =
+        driver.getJobContext(jobName);
+    jobMap.put(JobProperties.JobContext.name(), null);
+
+    if (jobContext != null) {
+      jobMap.put(JobProperties.JobContext.name(), jobContext.getRecord());
+    }
+
+    return JSONRepresentation(jobMap);
+  }
+
+  @PUT
+  @Path("{jobName}")
+  public Response addJob(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName,
+      String content) {
+    ZNRecord record;
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    try {
+      record = toZNRecord(content);
+      JobConfig.Builder jobConfig = JobAccessor.getJobConfig(record);
+      driver.enqueueJob(workflowName, jobName, jobConfig);
+    } catch (HelixException e) {
+      return badRequest(
+          String.format("Failed to enqueue job %s for reason : %s", jobName, e.getMessage()));
+    } catch (IOException e) {
+      return badRequest(String.format("Invalid input for Job Config of Job : %s", jobName));
+    }
+
+    return OK();
+  }
+
+  @DELETE
+  @Path("{jobName}")
+  public Response deleteJob(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    try {
+      driver.deleteJob(workflowName, jobName);
+    } catch (Exception e) {
+      return badRequest(e.getMessage());
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{jobName}/configs")
+  public Response getJobConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    JobConfig jobConfig = driver.getJobConfig(jobName);
+    if (jobConfig != null) {
+      return JSONRepresentation(jobConfig.getRecord());
+    }
+    return badRequest("Job config for " + jobName + " does not exists");
+  }
+
+  @GET
+  @Path("{jobName}/context")
+  public Response getJobContext(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowName") String workflowName, @PathParam("jobName") String jobName) {
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    JobContext jobContext =
+        driver.getJobContext(jobName);
+    if (jobContext != null) {
+      return JSONRepresentation(jobContext.getRecord());
+    }
+    return badRequest("Job context for " + jobName + " does not exists");
+  }
+
+  protected static JobConfig.Builder getJobConfig(Map<String, String> cfgMap) {
+    return new JobConfig.Builder().fromMap(cfgMap);
+  }
+
+  protected static JobConfig.Builder getJobConfig(ZNRecord record) {
+    JobConfig.Builder jobConfig = new JobConfig.Builder().fromMap(record.getSimpleFields());
+    jobConfig.addTaskConfigMap(getTaskConfigMap(record.getMapFields()));
+
+    return jobConfig;
+  }
+
+  private static Map<String, TaskConfig> getTaskConfigMap(
+      Map<String, Map<String, String>> taskConfigs) {
+    Map<String, TaskConfig> taskConfigsMap = new HashMap<>();
+    if (taskConfigs == null || taskConfigs.isEmpty()) {
+      return Collections.emptyMap();
+    }
+
+    for (Map<String, String> taskConfigMap : taskConfigs.values()) {
+      if (!taskConfigMap.containsKey(JobProperties.TASK_COMMAND.name())) {
+        continue;
+      }
+
+      TaskConfig taskConfig =
+          new TaskConfig(taskConfigMap.get(JobProperties.TASK_COMMAND.name()), taskConfigMap);
+      taskConfigsMap.put(taskConfig.getId(), taskConfig);
+    }
+
+    return taskConfigsMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/MetadataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/MetadataAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/MetadataAccessor.java
new file mode 100644
index 0000000..5757760
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/MetadataAccessor.java
@@ -0,0 +1,45 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Response;
+import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.common.HelixRestUtils;
+import org.apache.helix.rest.server.resources.AbstractResource;
+
+@Path("")
+public class MetadataAccessor extends AbstractResource {
+  @GET
+  public Response getMetadata() {
+    if (HelixRestUtils.isDefaultServlet(_servletRequest.getServletPath())) {
+      // To keep API endpoints to behave the same, if user call /admin/v2/ ,
+      // we will return NotFound
+      return notFound();
+    }
+    // This will be the root of all namespaced servlets, and returns
+    // servlet namespace information
+    HelixRestNamespace namespace = (HelixRestNamespace) _application.getProperties().get(
+        ContextPropertyKeys.METADATA.name());
+    return JSONRepresentation(namespace.getRestInfo());
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
new file mode 100644
index 0000000..04d3536
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/ResourceAccessor.java
@@ -0,0 +1,278 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixException;
+import org.apache.helix.PropertyPathBuilder;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZkClient;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.ResourceConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.JsonNodeFactory;
+import org.codehaus.jackson.node.ObjectNode;
+
+@Path("/clusters/{clusterId}/resources")
+public class ResourceAccessor extends AbstractHelixResource {
+  private final static Logger _logger = LoggerFactory.getLogger(ResourceAccessor.class);
+  public enum ResourceProperties {
+    idealState,
+    idealStates,
+    externalView,
+    externalViews,
+    resourceConfig,
+  }
+
+  @GET
+  public Response getResources(@PathParam("clusterId") String clusterId) {
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    root.put(Properties.id.name(), JsonNodeFactory.instance.textNode(clusterId));
+
+    ZkClient zkClient = getZkClient();
+
+    ArrayNode idealStatesNode = root.putArray(ResourceProperties.idealStates.name());
+    ArrayNode externalViewsNode = root.putArray(ResourceProperties.externalViews.name());
+
+    List<String> idealStates = zkClient.getChildren(PropertyPathBuilder.idealState(clusterId));
+    List<String> externalViews = zkClient.getChildren(PropertyPathBuilder.externalView(clusterId));
+
+    if (idealStates != null) {
+      idealStatesNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(idealStates));
+    } else {
+      return notFound();
+    }
+
+    if (externalViews != null) {
+      externalViewsNode.addAll((ArrayNode) OBJECT_MAPPER.valueToTree(externalViews));
+    }
+
+    return JSONRepresentation(root);
+  }
+
+  @GET
+  @Path("{resourceName}")
+  public Response getResource(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName) throws IOException {
+    ConfigAccessor accessor = getConfigAccessor();
+    HelixAdmin admin = getHelixAdmin();
+
+    ResourceConfig resourceConfig = accessor.getResourceConfig(clusterId, resourceName);
+    IdealState idealState = admin.getResourceIdealState(clusterId, resourceName);
+    ExternalView externalView = admin.getResourceExternalView(clusterId, resourceName);
+
+    Map<String, ZNRecord> resourceMap = new HashMap<>();
+    if (idealState != null) {
+      resourceMap.put(ResourceProperties.idealState.name(), idealState.getRecord());
+    } else {
+      return notFound();
+    }
+
+    resourceMap.put(ResourceProperties.resourceConfig.name(), null);
+    resourceMap.put(ResourceProperties.externalView.name(), null);
+
+    if (resourceConfig != null) {
+      resourceMap.put(ResourceProperties.resourceConfig.name(), resourceConfig.getRecord());
+    }
+
+    if (externalView != null) {
+      resourceMap.put(ResourceProperties.externalView.name(), externalView.getRecord());
+    }
+
+    return JSONRepresentation(resourceMap);
+  }
+
+  @PUT
+  @Path("{resourceName}")
+  public Response addResource(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName,
+      @DefaultValue("-1") @QueryParam("numPartitions") int numPartitions,
+      @DefaultValue("") @QueryParam("stateModelRef") String stateModelRef,
+      @DefaultValue("SEMI_AUTO") @QueryParam("rebalancerMode") String rebalancerMode,
+      @DefaultValue("DEFAULT") @QueryParam("rebalanceStrategy") String rebalanceStrategy,
+      @DefaultValue("0") @QueryParam("bucketSize") int bucketSize,
+      @DefaultValue("-1") @QueryParam("maxPartitionsPerInstance") int maxPartitionsPerInstance,
+      String content) {
+
+    HelixAdmin admin = getHelixAdmin();
+
+    try {
+      if (content.length() != 0) {
+        ZNRecord record;
+        try {
+          record = toZNRecord(content);
+        } catch (IOException e) {
+          _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+          return badRequest("Input is not a vaild ZNRecord!");
+        }
+
+        if (record.getSimpleFields() != null) {
+          admin.addResource(clusterId, resourceName, new IdealState(record));
+        }
+      } else {
+        admin.addResource(clusterId, resourceName, numPartitions, stateModelRef, rebalancerMode,
+            rebalanceStrategy, bucketSize, maxPartitionsPerInstance);
+      }
+    } catch (Exception e) {
+      _logger.error("Error in adding a resource: " + resourceName, e);
+      return serverError(e);
+    }
+
+    return OK();
+  }
+
+  @POST
+  @Path("{resourceName}")
+  public Response updateResource(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName, @QueryParam("command") String command,
+      @DefaultValue("-1") @QueryParam("replicas") int replicas,
+      @DefaultValue("") @QueryParam("keyPrefix") String keyPrefix,
+      @DefaultValue("") @QueryParam("group") String group){
+    Command cmd;
+    try {
+      cmd = Command.valueOf(command);
+    } catch (Exception e) {
+      return badRequest("Invalid command : " + command);
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      switch (cmd) {
+      case enable:
+        admin.enableResource(clusterId, resourceName, true);
+        break;
+      case disable:
+        admin.enableResource(clusterId, resourceName, false);
+        break;
+      case rebalance:
+        if (replicas == -1) {
+          return badRequest("Number of replicas is needed for rebalancing!");
+        }
+        keyPrefix = keyPrefix.length() == 0 ? resourceName : keyPrefix;
+        admin.rebalance(clusterId, resourceName, replicas, keyPrefix, group);
+        break;
+      default:
+        _logger.error("Unsupported command :" + command);
+        return badRequest("Unsupported command :" + command);
+      }
+    } catch (Exception e) {
+      _logger.error("Failed in updating resource : " + resourceName, e);
+      return badRequest(e.getMessage());
+    }
+    return OK();
+  }
+
+  @DELETE
+  @Path("{resourceName}")
+  public Response deleteResource(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName) {
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      admin.dropResource(clusterId, resourceName);
+    } catch (Exception e) {
+      _logger.error("Error in deleting a resource: " + resourceName, e);
+      return serverError();
+    }
+    return OK();
+  }
+
+  @GET
+  @Path("{resourceName}/configs")
+  public Response getResourceConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName) {
+    ConfigAccessor accessor = getConfigAccessor();
+    ResourceConfig resourceConfig = accessor.getResourceConfig(clusterId, resourceName);
+    if (resourceConfig != null) {
+      return JSONRepresentation(resourceConfig.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @POST
+  @Path("{resourceName}/configs")
+  public Response updateResourceConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName, String content) {
+    ZNRecord record;
+    try {
+      record = toZNRecord(content);
+    } catch (IOException e) {
+      _logger.error("Failed to deserialize user's input " + content + ", Exception: " + e);
+      return badRequest("Input is not a vaild ZNRecord!");
+    }
+    ResourceConfig resourceConfig = new ResourceConfig(record);
+    ConfigAccessor configAccessor = getConfigAccessor();
+    try {
+      configAccessor.updateResourceConfig(clusterId, resourceName, resourceConfig);
+    } catch (HelixException ex) {
+      return notFound(ex.getMessage());
+    } catch (Exception ex) {
+      _logger.error(
+          "Failed to update cluster config, cluster " + clusterId + " new config: " + content
+              + ", Exception: " + ex);
+      return serverError(ex);
+    }
+    return OK();
+  }
+
+  @GET
+  @Path("{resourceName}/idealState")
+  public Response getResourceIdealState(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName) {
+    HelixAdmin admin = getHelixAdmin();
+    IdealState idealState = admin.getResourceIdealState(clusterId, resourceName);
+    if (idealState != null) {
+      return JSONRepresentation(idealState.getRecord());
+    }
+
+    return notFound();
+  }
+
+  @GET
+  @Path("{resourceName}/externalView")
+  public Response getResourceExternalView(@PathParam("clusterId") String clusterId,
+      @PathParam("resourceName") String resourceName) {
+    HelixAdmin admin = getHelixAdmin();
+    ExternalView externalView = admin.getResourceExternalView(clusterId, resourceName);
+    if (externalView != null) {
+      return JSONRepresentation(externalView.getRecord());
+    }
+
+    return notFound();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
new file mode 100644
index 0000000..ebd04fd
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/helix/WorkflowAccessor.java
@@ -0,0 +1,325 @@
+package org.apache.helix.rest.server.resources.helix;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.task.JobConfig;
+import org.apache.helix.task.JobDag;
+import org.apache.helix.task.JobQueue;
+import org.apache.helix.task.TaskDriver;
+import org.apache.helix.task.Workflow;
+import org.apache.helix.task.WorkflowConfig;
+import org.apache.helix.task.WorkflowContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.JsonNode;
+import org.codehaus.jackson.map.type.TypeFactory;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.JsonNodeFactory;
+import org.codehaus.jackson.node.ObjectNode;
+import org.codehaus.jackson.node.TextNode;
+
+@Path("/clusters/{clusterId}/workflows")
+public class WorkflowAccessor extends AbstractHelixResource {
+  private static Logger _logger = LoggerFactory.getLogger(WorkflowAccessor.class.getName());
+
+  public enum WorkflowProperties {
+    Workflows,
+    WorkflowConfig,
+    WorkflowContext,
+    Jobs,
+    ParentJobs
+  }
+
+  public enum TaskCommand {
+    stop,
+    resume,
+    clean
+  }
+
+  @GET
+  public Response getWorkflows(@PathParam("clusterId") String clusterId) {
+    TaskDriver taskDriver = getTaskDriver(clusterId);
+    Map<String, WorkflowConfig> workflowConfigMap = taskDriver.getWorkflows();
+    Map<String, List<String>> dataMap = new HashMap<>();
+    dataMap.put(WorkflowProperties.Workflows.name(), new ArrayList<>(workflowConfigMap.keySet()));
+
+    return JSONRepresentation(dataMap);
+  }
+
+  @GET
+  @Path("{workflowId}")
+  public Response getWorkflow(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId) {
+    TaskDriver taskDriver = getTaskDriver(clusterId);
+    WorkflowConfig workflowConfig = taskDriver.getWorkflowConfig(workflowId);
+    WorkflowContext workflowContext = taskDriver.getWorkflowContext(workflowId);
+
+    ObjectNode root = JsonNodeFactory.instance.objectNode();
+    TextNode id = JsonNodeFactory.instance.textNode(workflowId);
+    root.put(Properties.id.name(), id);
+
+    ObjectNode workflowConfigNode = JsonNodeFactory.instance.objectNode();
+    ObjectNode workflowContextNode = JsonNodeFactory.instance.objectNode();
+
+    if (workflowConfig != null) {
+      getWorkflowConfigNode(workflowConfigNode, workflowConfig.getRecord());
+    }
+
+    if (workflowContext != null) {
+      getWorkflowContextNode(workflowContextNode, workflowContext.getRecord());
+    }
+
+    root.put(WorkflowProperties.WorkflowConfig.name(), workflowConfigNode);
+    root.put(WorkflowProperties.WorkflowContext.name(), workflowContextNode);
+
+    JobDag jobDag = workflowConfig.getJobDag();
+    ArrayNode jobs = OBJECT_MAPPER.valueToTree(jobDag.getAllNodes());
+    ObjectNode parentJobs = OBJECT_MAPPER.valueToTree(jobDag.getParentsToChildren());
+    root.put(WorkflowProperties.Jobs.name(), jobs);
+    root.put(WorkflowProperties.ParentJobs.name(), parentJobs);
+
+    return JSONRepresentation(root);
+  }
+
+  @PUT
+  @Path("{workflowId}")
+  public Response createWorkflow(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId, String content) {
+    TaskDriver driver = getTaskDriver(clusterId);
+    Map<String, String> cfgMap;
+    try {
+      JsonNode root = OBJECT_MAPPER.readTree(content);
+      cfgMap = OBJECT_MAPPER
+          .readValue(root.get(WorkflowProperties.WorkflowConfig.name()).toString(),
+              TypeFactory.defaultInstance()
+                  .constructMapType(HashMap.class, String.class, String.class));
+
+      WorkflowConfig workflowConfig = WorkflowConfig.Builder.fromMap(cfgMap).build();
+
+      // Since JobQueue can keep adding jobs, Helix create JobQueue will ignore the jobs
+      if (workflowConfig.isJobQueue()) {
+        driver.start(new JobQueue.Builder(workflowId).setWorkflowConfig(workflowConfig).build());
+        return OK();
+      }
+
+      Workflow.Builder workflow = new Workflow.Builder(workflowId);
+
+      if (root.get(WorkflowProperties.Jobs.name()) != null) {
+        Map<String, JobConfig.Builder> jobConfigs =
+            getJobConfigs((ArrayNode) root.get(WorkflowProperties.Jobs.name()));
+        for (Map.Entry<String, JobConfig.Builder> job : jobConfigs.entrySet()) {
+          workflow.addJob(job.getKey(), job.getValue());
+        }
+      }
+
+      if (root.get(WorkflowProperties.ParentJobs.name()) != null) {
+        Map<String, List<String>> parentJobs = OBJECT_MAPPER
+            .readValue(root.get(WorkflowProperties.ParentJobs.name()).toString(),
+                TypeFactory.defaultInstance()
+                    .constructMapType(HashMap.class, String.class, List.class));
+        for (Map.Entry<String, List<String>> entry : parentJobs.entrySet()) {
+          String parentJob = entry.getKey();
+          for (String childJob : entry.getValue()) {
+            workflow.addParentChildDependency(parentJob, childJob);
+          }
+        }
+      }
+
+      driver.start(workflow.build());
+    } catch (IOException e) {
+      return badRequest(String
+          .format("Invalid input of Workflow %s for reason : %s", workflowId, e.getMessage()));
+    } catch (HelixException e) {
+      return badRequest(String
+          .format("Failed to create workflow %s for reason : %s", workflowId, e.getMessage()));
+    }
+    return OK();
+  }
+
+  @DELETE
+  @Path("{workflowId}")
+  public Response deleteWorkflow(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId) {
+    TaskDriver driver = getTaskDriver(clusterId);
+    try {
+      driver.delete(workflowId);
+    } catch (HelixException e) {
+      return badRequest(String
+          .format("Failed to delete workflow %s for reason : %s", workflowId, e.getMessage()));
+    }
+    return OK();
+  }
+
+  @POST
+  @Path("{workflowId}")
+  public Response updateWorkflow(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId, @QueryParam("command") String command) {
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    try {
+      TaskCommand cmd = TaskCommand.valueOf(command);
+      switch (cmd) {
+      case stop:
+        driver.stop(workflowId);
+        break;
+      case resume:
+        driver.resume(workflowId);
+        break;
+      case clean:
+        driver.cleanupQueue(workflowId);
+        break;
+      default:
+        return badRequest(String.format("Invalid command : %s", command));
+      }
+    } catch (HelixException e) {
+      return badRequest(
+          String.format("Failed to execute operation %s for reason : %s", command, e.getMessage()));
+    } catch (Exception e) {
+      return serverError(e);
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{workflowId}/configs")
+  public Response getWorkflowConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId) {
+    TaskDriver taskDriver = getTaskDriver(clusterId);
+    WorkflowConfig workflowConfig = taskDriver.getWorkflowConfig(workflowId);
+    ObjectNode workflowConfigNode = JsonNodeFactory.instance.objectNode();
+    if (workflowConfig != null) {
+      getWorkflowConfigNode(workflowConfigNode, workflowConfig.getRecord());
+    }
+
+    return JSONRepresentation(workflowConfigNode);
+  }
+
+  @POST
+  @Path("{workflowId}/configs")
+  public Response updateWorkflowConfig(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId, String content) {
+    ZNRecord record;
+    TaskDriver driver = getTaskDriver(clusterId);
+
+    try {
+      record = toZNRecord(content);
+
+      WorkflowConfig workflowConfig = driver.getWorkflowConfig(workflowId);
+      if (workflowConfig == null) {
+        return badRequest(
+            String.format("WorkflowConfig for workflow %s does not exists!", workflowId));
+      }
+
+      workflowConfig.getRecord().update(record);
+      driver.updateWorkflow(workflowId, workflowConfig);
+    } catch (HelixException e) {
+      return badRequest(
+          String.format("Failed to update WorkflowConfig for workflow %s", workflowId));
+    } catch (Exception e) {
+      return badRequest(String.format("Invalid WorkflowConfig for workflow %s", workflowId));
+    }
+
+    return OK();
+  }
+
+  @GET
+  @Path("{workflowId}/context")
+  public Response getWorkflowContext(@PathParam("clusterId") String clusterId,
+      @PathParam("workflowId") String workflowId) {
+    TaskDriver taskDriver = getTaskDriver(clusterId);
+    WorkflowContext workflowContext = taskDriver.getWorkflowContext(workflowId);
+    ObjectNode workflowContextNode = JsonNodeFactory.instance.objectNode();
+    if (workflowContext != null) {
+      getWorkflowContextNode(workflowContextNode, workflowContext.getRecord());
+    }
+
+    return JSONRepresentation(workflowContextNode);
+  }
+
+  private void getWorkflowConfigNode(ObjectNode workflowConfigNode, ZNRecord record) {
+    for (Map.Entry<String, String> entry : record.getSimpleFields().entrySet()) {
+      if (!entry.getKey().equals(WorkflowConfig.WorkflowConfigProperty.Dag)) {
+        workflowConfigNode.put(entry.getKey(), JsonNodeFactory.instance.textNode(entry.getValue()));
+      }
+    }
+  }
+
+  private void getWorkflowContextNode(ObjectNode workflowContextNode, ZNRecord record) {
+    if (record.getMapFields() != null) {
+      for (String fieldName : record.getMapFields().keySet()) {
+        JsonNode node = OBJECT_MAPPER.valueToTree(record.getMapField(fieldName));
+        workflowContextNode.put(fieldName, node);
+      }
+    }
+
+    if (record.getSimpleFields() != null) {
+      for (Map.Entry<String, String> entry : record.getSimpleFields().entrySet()) {
+        workflowContextNode
+            .put(entry.getKey(), JsonNodeFactory.instance.textNode(entry.getValue()));
+      }
+    }
+  }
+
+  private Map<String, JobConfig.Builder> getJobConfigs(ArrayNode root)
+      throws HelixException, IOException {
+    Map<String, JobConfig.Builder> jobConfigsMap = new HashMap<>();
+    for (Iterator<JsonNode> it = root.getElements(); it.hasNext(); ) {
+      JsonNode job = it.next();
+      ZNRecord record = null;
+
+      try {
+        record = toZNRecord(job.toString());
+      } catch (IOException e) {
+        // Ignore the parse since it could be just simple fields
+      }
+
+      if (record == null || record.getSimpleFields().isEmpty()) {
+        Map<String, String> cfgMap = OBJECT_MAPPER.readValue(job.toString(),
+            TypeFactory.defaultInstance()
+                .constructMapType(HashMap.class, String.class, String.class));
+        jobConfigsMap
+            .put(job.get(Properties.id.name()).getTextValue(), JobAccessor.getJobConfig(cfgMap));
+      } else {
+        jobConfigsMap
+            .put(job.get(Properties.id.name()).getTextValue(), JobAccessor.getJobConfig(record));
+      }
+    }
+
+    return jobConfigsMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/main/java/org/apache/helix/rest/server/resources/metadata/NamespacesAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/metadata/NamespacesAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/metadata/NamespacesAccessor.java
new file mode 100644
index 0000000..dd5c5f9
--- /dev/null
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/metadata/NamespacesAccessor.java
@@ -0,0 +1,47 @@
+package org.apache.helix.rest.server.resources.metadata;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Response;
+import org.apache.helix.rest.common.ContextPropertyKeys;
+import org.apache.helix.rest.common.HelixRestNamespace;
+import org.apache.helix.rest.server.resources.AbstractResource;
+
+
+@Path("/namespaces")
+public class NamespacesAccessor extends AbstractResource {
+  @GET
+  public Response getHelixRestNamespaces() {
+    @SuppressWarnings("unchecked")
+    List<HelixRestNamespace> allNamespaces =
+        (List<HelixRestNamespace>) _application.getProperties()
+            .get(ContextPropertyKeys.ALL_NAMESPACES.name());
+    List<Map<String, String>> ret = new ArrayList<>();
+    for (HelixRestNamespace namespace : allNamespaces) {
+      ret.add(namespace.getRestInfo());
+    }
+    return JSONRepresentation(ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
index 94c5f63..763d3e2 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestClusterAccessor.java
@@ -42,7 +42,7 @@ import org.apache.helix.model.MaintenanceSignal;
 import org.apache.helix.rest.common.HelixRestNamespace;
 import org.apache.helix.rest.server.auditlog.AuditLog;
 import org.apache.helix.rest.server.resources.AbstractResource.Command;
-import org.apache.helix.rest.server.resources.ClusterAccessor;
+import org.apache.helix.rest.server.resources.helix.ClusterAccessor;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
index e213bd3..00ffdba 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestHelixRestServer.java
@@ -57,8 +57,7 @@ public class TestHelixRestServer extends AbstractTestClass {
     try {
       List<HelixRestNamespace> invalidManifest3 = new ArrayList<>();
       invalidManifest3.add(
-          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
-              true));
+          new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
       invalidManifest3.add(
           new HelixRestNamespace("DuplicatedName", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR,
               false));
@@ -77,7 +76,7 @@ public class TestHelixRestServer extends AbstractTestClass {
           new HelixRestNamespace("test4-2", HelixRestNamespace.HelixMetadataStoreType.ZOOKEEPER, ZK_ADDR, true));
       HelixRestServer svr = new HelixRestServer(invalidManifest4, 10250, "/", Collections.<AuditLogger>emptyList());
       Assert.assertFalse(true, "InvalidManifest4 test failed");
-    } catch (IllegalArgumentException e) {
+    } catch (IllegalStateException e) {
       // OK
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 947ba49..763f95b 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import javax.ws.rs.client.Entity;
@@ -33,10 +32,9 @@ import javax.ws.rs.core.Response;
 import org.apache.helix.HelixException;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.rest.server.resources.AbstractResource;
-import org.apache.helix.rest.server.resources.InstanceAccessor;
+import org.apache.helix.rest.server.resources.helix.InstanceAccessor;
 import org.codehaus.jackson.JsonNode;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
index eca6836..682039d 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestJobAccessor.java
@@ -27,8 +27,8 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.rest.server.resources.JobAccessor;
-import org.apache.helix.rest.server.resources.WorkflowAccessor;
+import org.apache.helix.rest.server.resources.helix.JobAccessor;
+import org.apache.helix.rest.server.resources.helix.WorkflowAccessor;
 import org.apache.helix.task.JobConfig;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.TaskDriver;

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
index e6f036d..c77f939 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestNamespacedAPIAccess.java
@@ -21,15 +21,22 @@ package org.apache.helix.rest.server;
 
 import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import javax.ws.rs.client.Entity;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.rest.common.HelixRestNamespace;
+import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestNamespacedAPIAccess extends AbstractTestClass {
+  ObjectMapper _mapper = new ObjectMapper();
+
   @Test
   public void testDefaultNamespaceCompatibility() {
     String testClusterName1 = "testClusterForDefaultNamespaceCompatibility1";
@@ -81,4 +88,52 @@ public class TestNamespacedAPIAccess extends AbstractTestClass {
     get(String.format("/clusters/%s", testClusterName), Response.Status.OK.getStatusCode(), false);
   }
 
+  @Test
+  public void testNamespaceServer() throws IOException {
+    // Default endpoints should not have any namespace information returned
+    get("/", Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    // Get invalid namespace should return not found
+    get("/namespaces/invalid-namespace", Response.Status.NOT_FOUND.getStatusCode(), false);
+
+    // list namespace should return a list of all namespaces
+    String body = get("/namespaces", Response.Status.OK.getStatusCode(), true);
+    List<Map<String, String>> namespaceMaps = _mapper
+        .readValue(body, _mapper.getTypeFactory().constructCollectionType(List.class, Map.class));
+    Assert.assertEquals(namespaceMaps.size(), 2);
+
+    Set<String> expectedNamespaceNames = new HashSet<>();
+    expectedNamespaceNames.add(HelixRestNamespace.DEFAULT_NAMESPACE_NAME);
+    expectedNamespaceNames.add(TEST_NAMESPACE);
+
+    for (Map<String, String> namespaceMap : namespaceMaps) {
+      String name = namespaceMap.get(HelixRestNamespace.HelixRestNamespaceProperty.NAME.name());
+      boolean isDefault = Boolean.parseBoolean(
+          namespaceMap.get(HelixRestNamespace.HelixRestNamespaceProperty.IS_DEFAULT.name()));
+      switch (name) {
+      case HelixRestNamespace.DEFAULT_NAMESPACE_NAME:
+        Assert.assertTrue(isDefault);
+        break;
+      case TEST_NAMESPACE:
+        Assert.assertFalse(isDefault);
+        break;
+      default:
+        Assert.assertFalse(true, "Namespace " + name + " is not expected");
+        break;
+      }
+      expectedNamespaceNames.remove(name);
+    }
+    Assert.assertTrue(expectedNamespaceNames.isEmpty());
+
+    // Accessing root of namespaced API endpoint shall return information of that namespace
+    body = get(String.format("/namespaces/%s", HelixRestNamespace.DEFAULT_NAMESPACE_NAME),
+        Response.Status.OK.getStatusCode(), true);
+    Map<String, String> namespace = _mapper.readValue(body,
+        _mapper.getTypeFactory().constructMapType(Map.class, String.class, String.class));
+    Assert.assertEquals(namespace.get(HelixRestNamespace.HelixRestNamespaceProperty.NAME.name()),
+        HelixRestNamespace.DEFAULT_NAMESPACE_NAME);
+    Assert.assertTrue(Boolean.parseBoolean(
+        namespace.get(HelixRestNamespace.HelixRestNamespaceProperty.IS_DEFAULT.name())));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
index 8e116e9..dea4e06 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestResourceAccessor.java
@@ -30,7 +30,7 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.builder.FullAutoModeISBuilder;
-import org.apache.helix.rest.server.resources.ResourceAccessor;
+import org.apache.helix.rest.server.resources.helix.ResourceAccessor;
 import org.codehaus.jackson.JsonNode;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/helix/blob/40710b27/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
index c41101f..ad8894a 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestWorkflowAccessor.java
@@ -8,7 +8,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.rest.server.resources.WorkflowAccessor;
+import org.apache.helix.rest.server.resources.helix.WorkflowAccessor;
 import org.apache.helix.task.JobQueue;
 import org.apache.helix.task.TargetState;
 import org.apache.helix.task.TaskDriver;


[39/50] [abbrv] helix git commit: More refactors to deduplicate code between helix's ZkClient and raw ZkClient.

Posted by jx...@apache.org.
More refactors to deduplicate code between helix's ZkClient and raw ZkClient.

1) Merge all duplicated (and extended) code in helix.manager.zk.ZkClient into helix.manager.zk.zookeeper.ZkClient.
2) Keep helix.manager.zk.ZkClient as a simple wrapper with all constructors and builder.


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

Branch: refs/heads/master
Commit: 310d47660b2c31b569ddfb98058ce5877441095f
Parents: 5ffab62
Author: Lei Xia <lx...@linkedin.com>
Authored: Tue Dec 19 10:22:37 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:57 2018 -0800

----------------------------------------------------------------------
 .../helix/manager/zk/ZkAsyncCallbacks.java      |   6 +-
 .../org/apache/helix/manager/zk/ZkClient.java   | 518 ++----------------
 .../helix/manager/zk/zookeeper/ZkClient.java    | 520 ++++++++++++++-----
 .../monitoring/mbeans/ZkClientMonitor.java      |  40 +-
 .../monitoring/mbeans/TestZkClientMonitor.java  |  10 +-
 5 files changed, 454 insertions(+), 640 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/310d4766/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
index 3fd0d0f..bfdf7bb 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkAsyncCallbacks.java
@@ -136,9 +136,11 @@ public class ZkAsyncCallbacks {
         ZkAsyncCallContext zkCtx = (ZkAsyncCallContext) ctx;
         if (zkCtx._monitor != null) {
           if (zkCtx._isRead) {
-            zkCtx._monitor.recordRead(path, zkCtx._bytes, zkCtx._startTimeMilliSec);
+            zkCtx._monitor.record(path, zkCtx._bytes, zkCtx._startTimeMilliSec,
+                ZkClientMonitor.AccessType.READ);
           } else {
-            zkCtx._monitor.recordWrite(path, zkCtx._bytes, zkCtx._startTimeMilliSec);
+            zkCtx._monitor.record(path, zkCtx._bytes, zkCtx._startTimeMilliSec,
+                ZkClientMonitor.AccessType.WRITE);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/helix/blob/310d4766/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
index 182c77e..618a003 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
@@ -21,47 +21,51 @@ package org.apache.helix.manager.zk;
 
 import org.I0Itec.zkclient.IZkConnection;
 import org.I0Itec.zkclient.ZkConnection;
-import org.I0Itec.zkclient.exception.ZkException;
-import org.I0Itec.zkclient.exception.ZkInterruptedException;
-import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.serialize.SerializableSerializer;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.HelixException;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.*;
-import org.apache.helix.monitoring.mbeans.ZkClientMonitor;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.management.JMException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.Callable;
 
 /**
- * ZKClient does not provide some functionalities, this will be used for quick fixes if
- * any bug found in ZKClient or if we need additional features but can't wait for the new
- * ZkClient jar Ideally we should commit the changes we do here to ZKClient.
+ * This is a wrapper of {@link org.apache.helix.manager.zk.zookeeper.ZkClient},
+ * with additional constructors and builder.
+ *
+ * // TODO: we will need to merge two ZkClient into just one class.
  */
-
 public class ZkClient extends org.apache.helix.manager.zk.zookeeper.ZkClient {
   private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
 
   public static final int DEFAULT_CONNECTION_TIMEOUT = 60 * 1000;
   public static final int DEFAULT_SESSION_TIMEOUT = 30 * 1000;
 
-  private PathBasedZkSerializer _zkSerializer;
-  private ZkClientMonitor _monitor;
-
-  private ZkClient(IZkConnection connection, int connectionTimeout, long operationRetryTimeout,
+  /**
+   *
+   * @param zkConnection
+   *            The Zookeeper connection
+   * @param connectionTimeout
+   *            The connection timeout in milli seconds
+   * @param zkSerializer
+   *            The Zookeeper data serializer
+   * @param operationRetryTimeout
+   *            Most operations are retried in cases like connection loss with the Zookeeper servers. During such failures, this
+   *            <code>operationRetryTimeout</code> decides the maximum amount of time, in milli seconds, each
+   *            operation is retried. A value lesser than 0 is considered as
+   *            "retry forever until a connection has been reestablished".
+   * @param monitorType
+   * @param monitorKey
+   * @param monitorInstanceName
+   *            These 3 inputs are used to name JMX monitor bean name for this ZkClient.
+   *            The JMX bean name will be: HelixZkClient.monitorType.monitorKey.monitorInstanceName.
+   * @param monitorRootPathOnly
+   *            Should only stat of access to root path be reported to JMX bean or path-specific stat be reported too.
+   */
+  public ZkClient(IZkConnection zkConnection, int connectionTimeout, long operationRetryTimeout,
       PathBasedZkSerializer zkSerializer, String monitorType, String monitorKey,
       String monitorInstanceName, boolean monitorRootPathOnly) {
-    super(connection, connectionTimeout, new ByteArraySerializer(), operationRetryTimeout);
-    init(zkSerializer, monitorType, monitorKey, monitorInstanceName, monitorRootPathOnly);
+    super(zkConnection, connectionTimeout, operationRetryTimeout, zkSerializer, monitorType,
+        monitorKey, monitorInstanceName, monitorRootPathOnly);
   }
 
   public ZkClient(IZkConnection connection, int connectionTimeout,
@@ -128,466 +132,16 @@ public class ZkClient extends org.apache.helix.manager.zk.zookeeper.ZkClient {
     this(zkServers, null, null);
   }
 
-  protected void init(PathBasedZkSerializer zkSerializer, String monitorType, String monitorKey,
-      String monitorInstanceName, boolean monitorRootPathOnly) {
-    _zkSerializer = zkSerializer;
-    if (LOG.isTraceEnabled()) {
-      StackTraceElement[] calls = Thread.currentThread().getStackTrace();
-      LOG.trace("created a zkclient. callstack: " + Arrays.asList(calls));
-    }
-    try {
-      if (monitorKey != null && !monitorKey.isEmpty() && monitorType != null && !monitorType
-          .isEmpty()) {
-        _monitor =
-            new ZkClientMonitor(monitorType, monitorKey, monitorInstanceName, monitorRootPathOnly);
-      } else {
-        LOG.info("ZkClient monitor key or type is not provided. Skip monitoring.");
-      }
-    } catch (JMException e) {
-      LOG.error("Error in creating ZkClientMonitor", e);
-    }
-  }
-
-  @Override
-  public void setZkSerializer(ZkSerializer zkSerializer) {
-    _zkSerializer = new BasicZkSerializer(zkSerializer);
-  }
-
-  public void setZkSerializer(PathBasedZkSerializer zkSerializer) {
-    _zkSerializer = zkSerializer;
-  }
-
-  public PathBasedZkSerializer getZkSerializer() {
-    return _zkSerializer;
-  }
-
-  public IZkConnection getConnection() {
-    return _connection;
-  }
-
-  @Override
-  public void close() throws ZkInterruptedException {
-    if (LOG.isTraceEnabled()) {
-      StackTraceElement[] calls = Thread.currentThread().getStackTrace();
-      LOG.trace("closing a zkclient. callStack: " + Arrays.asList(calls));
-    }
-    getEventLock().lock();
-    try {
-      if (_connection == null) {
-        return;
-      }
-      LOG.info("Closing zkclient: " + ((ZkConnection) _connection).getZookeeper());
-      super.close();
-    } catch (ZkInterruptedException e) {
-      /**
-       * Workaround for HELIX-264: calling ZkClient#close() in its own eventThread context will
-       * throw ZkInterruptedException and skip ZkConnection#close()
-       */
-      if (_connection != null) {
-        try {
-          /**
-           * ZkInterruptedException#construct() honors InterruptedException by calling
-           * Thread.currentThread().interrupt(); clear it first, so we can safely close the
-           * zk-connection
-           */
-          Thread.interrupted();
-          _connection.close();
-          /**
-           * restore interrupted status of current thread
-           */
-          Thread.currentThread().interrupt();
-        } catch (InterruptedException e1) {
-          throw new ZkInterruptedException(e1);
-        }
-      }
-    } finally {
-      getEventLock().unlock();
-      if (_monitor != null) {
-        _monitor.unregister();
-      }
-      LOG.info("Closed zkclient");
-    }
-  }
-
-  public boolean isClosed() {
-    return (_connection == null || !_connection.getZookeeperState().isAlive());
-  }
-
-  public Stat getStat(final String path) {
-    long startT = System.currentTimeMillis();
-    try {
-      Stat stat = retryUntilConnected(new Callable<Stat>() {
-
-        @Override
-        public Stat call() throws Exception {
-          Stat stat = ((ZkConnection) _connection).getZookeeper().exists(path, false);
-          return stat;
-        }
-      });
-      recordRead(path, null, startT);
-      return stat;
-    } catch (Exception e) {
-      recordReadFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("exists, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  // override exists(path, watch), so we can record all exists requests
-  @Override
-  protected boolean exists(final String path, final boolean watch) {
-    long startT = System.currentTimeMillis();
-    try {
-      boolean exists = retryUntilConnected(new Callable<Boolean>() {
-        @Override
-        public Boolean call() throws Exception {
-          return _connection.exists(path, watch);
-        }
-      });
-      recordRead(path, null, startT);
-      return exists;
-    } catch (Exception e) {
-      recordReadFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("exists, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  // override getChildren(path, watch), so we can record all getChildren requests
-  @Override
-  protected List<String> getChildren(final String path, final boolean watch) {
-    long startT = System.currentTimeMillis();
-    try {
-      List<String> children = retryUntilConnected(new Callable<List<String>>() {
-        @Override
-        public List<String> call() throws Exception {
-          return _connection.getChildren(path, watch);
-        }
-      });
-      recordRead(path, null, startT);
-      return children;
-    } catch (Exception e) {
-      recordReadFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("getChildren, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T extends Object> T deserialize(byte[] data, String path) {
-    if (data == null) {
-      return null;
-    }
-    return (T) _zkSerializer.deserialize(data, path);
-  }
-
-  // override readData(path, stat, watch), so we can record all read requests
-  @Override
-  @SuppressWarnings("unchecked")
-  protected <T extends Object> T readData(final String path, final Stat stat, final boolean watch) {
-    long startT = System.currentTimeMillis();
-    byte[] data = null;
-    try {
-      data = retryUntilConnected(new Callable<byte[]>() {
-
-        @Override
-        public byte[] call() throws Exception {
-          return _connection.readData(path, stat, watch);
-        }
-      });
-      recordRead(path, data, startT);
-      return (T) deserialize(data, path);
-    } catch (Exception e) {
-      recordReadFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("getData, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public <T extends Object> T readDataAndStat(String path, Stat stat,
-      boolean returnNullIfPathNotExists) {
-    T data = null;
-    try {
-      data = readData(path, stat);
-    } catch (ZkNoNodeException e) {
-      if (!returnNullIfPathNotExists) {
-        throw e;
-      }
-    }
-    return data;
-  }
-
-  public String getServers() {
-    return _connection.getServers();
-  }
-
-  public byte[] serialize(Object data, String path) {
-    return _zkSerializer.serialize(data, path);
-  }
-
-  @Override
-  public void writeData(final String path, Object datat, final int expectedVersion) {
-    long startT = System.currentTimeMillis();
-    try {
-      final byte[] data = serialize(datat, path);
-      checkDataSizeLimit(data);
-      retryUntilConnected(new Callable<Object>() {
-
-        @Override public Object call() throws Exception {
-          _connection.writeData(path, data, expectedVersion);
-          return null;
-        }
-      });
-      recordWrite(path, data, startT);
-    } catch (Exception e) {
-      recordWriteFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("setData, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  public Stat writeDataGetStat(final String path, Object datat, final int expectedVersion)
-      throws InterruptedException {
-    long startT = System.currentTimeMillis();
-    try {
-      final byte[] data = _zkSerializer.serialize(datat, path);
-      checkDataSizeLimit(data);
-      Stat stat = retryUntilConnected(new Callable<Stat>() {
-
-        @Override public Stat call() throws Exception {
-          return ((ZkConnection) _connection).getZookeeper()
-              .setData(path, data, expectedVersion);
-        }
-      });
-      recordWrite(path, data, startT);
-      return stat;
-    } catch (Exception e) {
-      recordWriteFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("setData, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  @Override
-  public String create(final String path, Object datat, final CreateMode mode)
-      throws IllegalArgumentException, ZkException {
-    if (path == null) {
-      throw new NullPointerException("path must not be null.");
-    }
-    long startT = System.currentTimeMillis();
-    try {
-      final byte[] data = datat == null ? null : serialize(datat, path);
-      checkDataSizeLimit(data);
-      String actualPath = retryUntilConnected(new Callable<String>() {
-        @Override
-        public String call() throws Exception {
-          return _connection.create(path, data, mode);
-        }
-      });
-      recordWrite(path, data, startT);
-      return actualPath;
-    } catch (Exception e) {
-      recordWriteFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("create, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-  }
-
-  @Override
-  public boolean delete(final String path) {
-    long startT = System.currentTimeMillis();
-    boolean isDeleted;
-    try {
-      try {
-        retryUntilConnected(new Callable<Object>() {
-
-          @Override
-          public Object call() throws Exception {
-            _connection.delete(path);
-            return null;
-          }
-        });
-        isDeleted = true;
-      } catch (ZkNoNodeException e) {
-        isDeleted = false;
-        LOG.error("Failed to delete path " + path + ", znode does not exist!");
-      }
-      recordWrite(path, null, startT);
-    } catch (Exception e) {
-      recordWriteFailure(path);
-      throw e;
-    } finally {
-      long endT = System.currentTimeMillis();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("delete, path: " + path + ", time: " + (endT - startT) + " ms");
-      }
-    }
-    return isDeleted;
-  }
-
-  public void asyncCreate(final String path, Object datat, final CreateMode mode,
-      final CreateCallbackHandler cb) {
-    final long startT = System.currentTimeMillis();
-    final byte[] data = (datat == null ? null : serialize(datat, path));
-    retryUntilConnected(new Callable<Object>() {
-      @Override public Object call() throws Exception {
-        ((ZkConnection) _connection).getZookeeper().create(path, data, Ids.OPEN_ACL_UNSAFE,
-            // Arrays.asList(DEFAULT_ACL),
-            mode, cb, new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT,
-                data == null ? 0 : data.length, false));
-        return null;
-      }
-    });
-  }
-
-  public void asyncSetData(final String path, Object datat, final int version,
-      final SetDataCallbackHandler cb) {
-    final long startT = System.currentTimeMillis();
-    final byte[] data = serialize(datat, path);
-    retryUntilConnected(new Callable<Object>() {
-      @Override public Object call() throws Exception {
-        ((ZkConnection) _connection).getZookeeper().setData(path, data, version, cb,
-            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT,
-                data == null ? 0 : data.length, false));
-        return null;
-      }
-    });
-  }
-
-  public void asyncGetData(final String path, final GetDataCallbackHandler cb) {
-    final long startT = System.currentTimeMillis();
-    retryUntilConnected(new Callable<Object>() {
-      @Override public Object call() throws Exception {
-        ((ZkConnection) _connection).getZookeeper().getData(path, null, cb,
-            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, true));
-        return null;
-      }
-    });
-  }
-
-  public void asyncExists(final String path, final ExistsCallbackHandler cb) {
-    final long startT = System.currentTimeMillis();
-    retryUntilConnected(new Callable<Object>() {
-      @Override public Object call() throws Exception {
-        ((ZkConnection) _connection).getZookeeper().exists(path, null, cb,
-            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, true));
-        return null;
-      }
-    });
-  }
-
-  public void asyncDelete(final String path, final DeleteCallbackHandler cb) {
-    final long startT = System.currentTimeMillis();
-    retryUntilConnected(new Callable<Object>() {
-      @Override public Object call() throws Exception {
-        ((ZkConnection) _connection).getZookeeper().delete(path, -1, cb,
-            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, false));
-        return null;
-      }
-    });
-  }
-
-  public <T> T retryUntilConnected(final Callable<T> callable) {
-    final ZkConnection zkConnection = (ZkConnection) getConnection();
-    return super.retryUntilConnected(new Callable<T>() {
-      @Override
-      public T call() throws Exception {
-        // Validate that the connection is not null before trigger callback
-        if (zkConnection == null || zkConnection.getZookeeper() == null) {
-          throw new IllegalStateException(
-              "ZkConnection is in invalid state! Please close this ZkClient and create new client.");
-        }
-        return callable.call();
-      }
-    });
-  }
-
-  private void checkDataSizeLimit(byte[] data) {
-    if (data != null && data.length > ZNRecord.SIZE_LIMIT) {
-      LOG.error("Data size larger than 1M, will not write to zk. Data (first 1k): "
-          + new String(data).substring(0, 1024));
-      throw new HelixException("Data size larger than 1M");
-    }
-  }
-
-  @Override public void process(WatchedEvent event) {
-    boolean stateChanged = event.getPath() == null;
-    boolean dataChanged = event.getType() == Event.EventType.NodeDataChanged
-        || event.getType() == Event.EventType.NodeDeleted
-        || event.getType() == Event.EventType.NodeCreated
-        || event.getType() == Event.EventType.NodeChildrenChanged;
-
-    if (_monitor != null) {
-      if (stateChanged) {
-        _monitor.increaseStateChangeEventCounter();
-      }
-      if (dataChanged) {
-        _monitor.increaseDataChangeEventCounter();
-      }
-    }
-
-    super.process(event);
-  }
-
-  private void recordRead(String path, byte[] data, long startTimeMilliSec) {
-    if (_monitor != null) {
-      int dataSize = 0;
-      if (data != null) {
-        dataSize = data.length;
-      }
-      _monitor.recordRead(path, dataSize, startTimeMilliSec);
-    }
+  public ZkClient(final String zkServers, final int sessionTimeout, final int connectionTimeout,
+      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
+    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout, zkSerializer,
+        operationRetryTimeout);
   }
 
-  private void recordWrite(String path, byte[] data, long startTimeMilliSec) {
-    if (_monitor != null) {
-      int dataSize = 0;
-      if (data != null) {
-        dataSize = data.length;
-      }
-      _monitor.recordWrite(path, dataSize, startTimeMilliSec);
-    }
-  }
-
-  private void recordReadFailure(String path) {
-    if (_monitor != null) {
-      _monitor.recordReadFailure(path);
-    }
-  }
-
-  private void recordWriteFailure(String path) {
-    if (_monitor != null) {
-      _monitor.recordWriteFailure(path);
-    }
+  public ZkClient(final IZkConnection zkConnection, final int connectionTimeout,
+      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
+    this(zkConnection, connectionTimeout, operationRetryTimeout,
+        new BasicZkSerializer(zkSerializer), null, null, null, false);
   }
 
   public static class Builder {

http://git-wip-us.apache.org/repos/asf/helix/blob/310d4766/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
index d26a274..4748d6e 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/zookeeper/ZkClient.java
@@ -10,6 +10,7 @@
  */
 package org.apache.helix.manager.zk.zookeeper;
 
+import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -19,8 +20,9 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
-
+import javax.management.JMException;
 import org.I0Itec.zkclient.DataUpdater;
+import org.I0Itec.zkclient.ExceptionUtil;
 import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkConnection;
 import org.I0Itec.zkclient.IZkDataListener;
@@ -33,9 +35,15 @@ import org.I0Itec.zkclient.exception.ZkInterruptedException;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.I0Itec.zkclient.exception.ZkTimeoutException;
-import org.I0Itec.zkclient.ExceptionUtil;
 import org.I0Itec.zkclient.serialize.SerializableSerializer;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.helix.HelixException;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.BasicZkSerializer;
+import org.apache.helix.manager.zk.PathBasedZkSerializer;
+import org.apache.helix.manager.zk.ZkAsyncCallbacks;
+import org.apache.helix.manager.zk.zookeeper.ZkEventThread.ZkEvent;
+import org.apache.helix.monitoring.mbeans.ZkClientMonitor;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
@@ -49,12 +57,12 @@ import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
-import org.apache.helix.manager.zk.zookeeper.ZkEventThread.ZkEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Abstracts the interaction with zookeeper and allows permanent (not just one time) watches on nodes in ZooKeeper
+ * Abstracts the interaction with zookeeper and allows permanent (not just one time) watches on nodes in ZooKeeper.
+ * WARN: Do not use this class directly, use {@link org.apache.helix.manager.zk.ZkClient} instead.
  */
 public class ZkClient implements Watcher {
   private static Logger LOG = LoggerFactory.getLogger(ZkClient.class);
@@ -62,106 +70,51 @@ public class ZkClient implements Watcher {
   protected final IZkConnection _connection;
   protected final long operationRetryTimeoutInMillis;
   private final Map<String, Set<IZkChildListener>> _childListener =
-      new ConcurrentHashMap<String, Set<IZkChildListener>>();
+      new ConcurrentHashMap<>();
   private final ConcurrentHashMap<String, Set<IZkDataListener>> _dataListener =
-      new ConcurrentHashMap<String, Set<IZkDataListener>>();
-  private final Set<IZkStateListener> _stateListener = new CopyOnWriteArraySet<IZkStateListener>();
+      new ConcurrentHashMap<>();
+  private final Set<IZkStateListener> _stateListener = new CopyOnWriteArraySet<>();
   private KeeperState _currentState;
   private final ZkLock _zkEventLock = new ZkLock();
   private boolean _shutdownTriggered;
   private ZkEventThread _eventThread;
   // TODO PVo remove this later
   private Thread _zookeeperEventThread;
-  private ZkSerializer _zkSerializer;
   private volatile boolean _closed;
+  private PathBasedZkSerializer _pathBasedZkSerializer;
+  private ZkClientMonitor _monitor;
 
-  public ZkClient(String serverstring) {
-    this(serverstring, Integer.MAX_VALUE);
-  }
-
-  public ZkClient(String zkServers, int connectionTimeout) {
-    this(new ZkConnection(zkServers), connectionTimeout);
-  }
-
-  public ZkClient(String zkServers, int sessionTimeout, int connectionTimeout) {
-    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout);
-  }
-
-  public ZkClient(String zkServers, int sessionTimeout, int connectionTimeout,
-      ZkSerializer zkSerializer) {
-    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout, zkSerializer);
-  }
-
-  /**
-   *
-   * @param zkServers
-   *            The Zookeeper servers
-   * @param sessionTimeout
-   *            The session timeout in milli seconds
-   * @param connectionTimeout
-   *            The connection timeout in milli seconds
-   * @param zkSerializer
-   *            The Zookeeper data serializer
-   * @param operationRetryTimeout
-   *            Most operations done through this {@link org.I0Itec.zkclient.ZkClient} are retried in cases like
-   *            connection loss with the Zookeeper servers. During such failures, this
-   *            <code>operationRetryTimeout</code> decides the maximum amount of time, in milli seconds, each
-   *            operation is retried. A value lesser than 0 is considered as
-   *            "retry forever until a connection has been reestablished".
-   */
-  public ZkClient(final String zkServers, final int sessionTimeout, final int connectionTimeout,
-      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
-    this(new ZkConnection(zkServers, sessionTimeout), connectionTimeout, zkSerializer,
-        operationRetryTimeout);
-  }
-
-  public ZkClient(IZkConnection connection) {
-    this(connection, Integer.MAX_VALUE);
-  }
-
-  public ZkClient(IZkConnection connection, int connectionTimeout) {
-    this(connection, connectionTimeout, new SerializableSerializer());
-  }
 
-  public ZkClient(IZkConnection zkConnection, int connectionTimeout, ZkSerializer zkSerializer) {
-    this(zkConnection, connectionTimeout, zkSerializer, -1);
-  }
-
-  /**
-   *
-   * @param zkConnection
-   *            The Zookeeper servers
-   * @param connectionTimeout
-   *            The connection timeout in milli seconds
-   * @param zkSerializer
-   *            The Zookeeper data serializer
-   * @param operationRetryTimeout
-   *            Most operations done through this {@link org.I0Itec.zkclient.ZkClient} are retried in cases like
-   *            connection loss with the Zookeeper servers. During such failures, this
-   *            <code>operationRetryTimeout</code> decides the maximum amount of time, in milli seconds, each
-   *            operation is retried. A value lesser than 0 is considered as
-   *            "retry forever until a connection has been reestablished".
-   */
-  public ZkClient(final IZkConnection zkConnection, final int connectionTimeout,
-      final ZkSerializer zkSerializer, final long operationRetryTimeout) {
+  protected ZkClient(IZkConnection zkConnection, int connectionTimeout, long operationRetryTimeout,
+      PathBasedZkSerializer zkSerializer, String monitorType, String monitorKey,
+      String monitorInstanceName, boolean monitorRootPathOnly) {
     if (zkConnection == null) {
       throw new NullPointerException("Zookeeper connection is null!");
     }
     _connection = zkConnection;
-    _zkSerializer = zkSerializer;
+    _pathBasedZkSerializer = zkSerializer;
     this.operationRetryTimeoutInMillis = operationRetryTimeout;
     connect(connectionTimeout, this);
-  }
 
-  public void setZkSerializer(ZkSerializer zkSerializer) {
-    _zkSerializer = zkSerializer;
+    // initiate monitor
+    try {
+      if (monitorKey != null && !monitorKey.isEmpty() && monitorType != null && !monitorType
+          .isEmpty()) {
+        _monitor =
+            new ZkClientMonitor(monitorType, monitorKey, monitorInstanceName, monitorRootPathOnly);
+      } else {
+        LOG.info("ZkClient monitor key or type is not provided. Skip monitoring.");
+      }
+    } catch (JMException e) {
+      LOG.error("Error in creating ZkClientMonitor", e);
+    }
   }
 
   public List<String> subscribeChildChanges(String path, IZkChildListener listener) {
     synchronized (_childListener) {
       Set<IZkChildListener> listeners = _childListener.get(path);
       if (listeners == null) {
-        listeners = new CopyOnWriteArraySet<IZkChildListener>();
+        listeners = new CopyOnWriteArraySet<>();
         _childListener.put(path, listeners);
       }
       listeners.add(listener);
@@ -183,7 +136,7 @@ public class ZkClient implements Watcher {
     synchronized (_dataListener) {
       listeners = _dataListener.get(path);
       if (listeners == null) {
-        listeners = new CopyOnWriteArraySet<IZkDataListener>();
+        listeners = new CopyOnWriteArraySet<>();
         _dataListener.put(path, listeners);
       }
       listeners.add(listener);
@@ -446,7 +399,7 @@ public class ZkClient implements Watcher {
    * Create a node with ACL.
    *
    * @param path
-   * @param data
+   * @param datat
    * @param acl
    * @param mode
    * @return create node's path
@@ -459,21 +412,35 @@ public class ZkClient implements Watcher {
    * @throws RuntimeException
    *             if any other exception occurs
    */
-  public String create(final String path, Object data, final List<ACL> acl, final CreateMode mode) {
+  public String create(final String path, Object datat, final List<ACL> acl, final CreateMode mode)
+      throws IllegalArgumentException, ZkException {
     if (path == null) {
-      throw new NullPointerException("Missing value for path");
+      throw new NullPointerException("Path must not be null.");
     }
     if (acl == null || acl.size() == 0) {
       throw new NullPointerException("Missing value for ACL");
     }
-    final byte[] bytes = data == null ? null : serialize(data);
-
-    return retryUntilConnected(new Callable<String>() {
-      @Override public String call() throws Exception {
-        return _connection.create(path, bytes, acl, mode);
+    long startT = System.currentTimeMillis();
+    try {
+      final byte[] data = datat == null ? null : serialize(datat, path);
+      checkDataSizeLimit(data);
+      String actualPath = retryUntilConnected(new Callable<String>() {
+        @Override
+        public String call() throws Exception {
+          return _connection.create(path, data, acl, mode);
+        }
+      });
+      record(path, data, startT, ZkClientMonitor.AccessType.WRITE);
+      return actualPath;
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.WRITE);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("create, path: " + path + ", time: " + (endT - startT) + " ms");
       }
-    });
-
+    }
   }
 
   /**
@@ -556,16 +523,17 @@ public class ZkClient implements Watcher {
     return create(path, data, acl, CreateMode.EPHEMERAL_SEQUENTIAL);
   }
 
-  @Override public void process(WatchedEvent event) {
+  @Override
+  public void process(WatchedEvent event) {
     LOG.debug("Received event: " + event);
     _zookeeperEventThread = Thread.currentThread();
 
     boolean stateChanged = event.getPath() == null;
     boolean znodeChanged = event.getPath() != null;
-    boolean dataChanged =
-        event.getType() == EventType.NodeDataChanged || event.getType() == EventType.NodeDeleted
-            || event.getType() == EventType.NodeCreated
-            || event.getType() == EventType.NodeChildrenChanged;
+    boolean dataChanged = event.getType() == Event.EventType.NodeDataChanged
+        || event.getType() == Event.EventType.NodeDeleted
+        || event.getType() == Event.EventType.NodeCreated
+        || event.getType() == Event.EventType.NodeChildrenChanged;
 
     getEventLock().lock();
     try {
@@ -604,6 +572,10 @@ public class ZkClient implements Watcher {
         getEventLock().getDataChangedCondition().signalAll();
       }
       getEventLock().unlock();
+
+      // update state change counter.
+      recordStateChange(stateChanged, dataChanged);
+
       LOG.debug("Leaving process event");
     }
   }
@@ -622,13 +594,28 @@ public class ZkClient implements Watcher {
   }
 
   protected List<String> getChildren(final String path, final boolean watch) {
-    return retryUntilConnected(new Callable<List<String>>() {
-      @Override public List<String> call() throws Exception {
-        return _connection.getChildren(path, watch);
+    long startT = System.currentTimeMillis();
+    try {
+      List<String> children = retryUntilConnected(new Callable<List<String>>() {
+        @Override
+        public List<String> call() throws Exception {
+          return _connection.getChildren(path, watch);
+        }
+      });
+      record(path, null, startT, ZkClientMonitor.AccessType.READ);
+      return children;
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.READ);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("getChildren, path: " + path + ", time: " + (endT - startT) + " ms");
       }
-    });
+    }
   }
 
+
   /**
    * Counts number of children for the given path.
    *
@@ -643,16 +630,54 @@ public class ZkClient implements Watcher {
     }
   }
 
+  public boolean exists(final String path) {
+    return exists(path, hasListeners(path));
+  }
+
+
   protected boolean exists(final String path, final boolean watch) {
-    return retryUntilConnected(new Callable<Boolean>() {
-      @Override public Boolean call() throws Exception {
-        return _connection.exists(path, watch);
+    long startT = System.currentTimeMillis();
+    try {
+      boolean exists = retryUntilConnected(new Callable<Boolean>() {
+        @Override
+        public Boolean call() throws Exception {
+          return _connection.exists(path, watch);
+        }
+      });
+      record(path, null, startT, ZkClientMonitor.AccessType.READ);
+      return exists;
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.READ);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("exists, path: " + path + ", time: " + (endT - startT) + " ms");
       }
-    });
+    }
   }
 
-  public boolean exists(final String path) {
-    return exists(path, hasListeners(path));
+  public Stat getStat(final String path) {
+    long startT = System.currentTimeMillis();
+    try {
+      Stat stat = retryUntilConnected(new Callable<Stat>() {
+        @Override
+        public Stat call() throws Exception {
+          Stat stat = ((ZkConnection) _connection).getZookeeper().exists(path, false);
+          return stat;
+        }
+      });
+      record(path, null, startT, ZkClientMonitor.AccessType.READ);
+      return stat;
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.READ);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("exists, path: " + path + ", time: " + (endT - startT) + " ms");
+      }
+    }
   }
 
   private void processStateChanged(WatchedEvent event) {
@@ -826,6 +851,10 @@ public class ZkClient implements Watcher {
     return _dataListener.get(path);
   }
 
+  public IZkConnection getConnection() {
+    return _connection;
+  }
+
   public void waitUntilConnected() throws ZkInterruptedException {
     waitUntilConnected(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
   }
@@ -882,8 +911,8 @@ public class ZkClient implements Watcher {
    * @throws RuntimeException
    *             if any other exception occurs from invoking the Callable
    */
-  public <T> T retryUntilConnected(Callable<T> callable)
-      throws ZkInterruptedException, IllegalArgumentException, ZkException, RuntimeException {
+  public <T> T retryUntilConnected(final Callable<T> callable)
+      throws IllegalArgumentException, ZkException {
     if (_zookeeperEventThread != null && Thread.currentThread() == _zookeeperEventThread) {
       throw new IllegalArgumentException("Must not be done in the zookeeper event thread.");
     }
@@ -893,6 +922,12 @@ public class ZkClient implements Watcher {
         throw new IllegalStateException("ZkClient already closed!");
       }
       try {
+        final ZkConnection zkConnection = (ZkConnection) getConnection();
+        // Validate that the connection is not null before trigger callback
+        if (zkConnection == null || zkConnection.getZookeeper() == null) {
+          throw new IllegalStateException(
+              "ZkConnection is in invalid state! Please close this ZkClient and create new client.");
+        }
         return callable.call();
       } catch (ConnectionLossException e) {
         // we give the event thread some time to update the status to 'Disconnected'
@@ -948,30 +983,58 @@ public class ZkClient implements Watcher {
   }
 
   public boolean delete(final String path) {
+    long startT = System.currentTimeMillis();
+    boolean success;
     try {
-      retryUntilConnected(new Callable<Object>() {
-
-        @Override public Object call() throws Exception {
-          _connection.delete(path);
-          return null;
-        }
-      });
+      try {
+        retryUntilConnected(new Callable<Object>() {
 
-      return true;
-    } catch (ZkNoNodeException e) {
-      return false;
+          @Override
+          public Object call() throws Exception {
+            _connection.delete(path);
+            return null;
+          }
+        });
+        success = true;
+      } catch (ZkNoNodeException e) {
+        success = false;
+        LOG.warn("Failed to delete path " + path + ", znode does not exist!");
+      }
+      record(path, null, startT, ZkClientMonitor.AccessType.WRITE);
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.WRITE);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("delete, path: " + path + ", time: " + (endT - startT) + " ms");
+      }
     }
+    return success;
+  }
+
+  public void setZkSerializer(ZkSerializer zkSerializer) {
+    _pathBasedZkSerializer = new BasicZkSerializer(zkSerializer);
+  }
+
+  public void setZkSerializer(PathBasedZkSerializer zkSerializer) {
+    _pathBasedZkSerializer = zkSerializer;
   }
 
-  private byte[] serialize(Object data) {
-    return _zkSerializer.serialize(data);
+  public PathBasedZkSerializer getZkSerializer() {
+    return _pathBasedZkSerializer;
   }
 
-  @SuppressWarnings("unchecked") private <T extends Object> T derializable(byte[] data) {
+  public byte[] serialize(Object data, String path) {
+    return _pathBasedZkSerializer.serialize(data, path);
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T extends Object> T deserialize(byte[] data, String path) {
     if (data == null) {
       return null;
     }
-    return (T) _zkSerializer.deserialize(data);
+    return (T) _pathBasedZkSerializer.deserialize(data, path);
   }
 
   @SuppressWarnings("unchecked") public <T extends Object> T readData(String path) {
@@ -991,19 +1054,47 @@ public class ZkClient implements Watcher {
     return data;
   }
 
-  @SuppressWarnings("unchecked") public <T extends Object> T readData(String path, Stat stat) {
+  @SuppressWarnings("unchecked")
+  public <T extends Object> T readData(String path, Stat stat) {
     return (T) readData(path, stat, hasListeners(path));
   }
 
-  @SuppressWarnings("unchecked") protected <T extends Object> T readData(final String path,
-      final Stat stat, final boolean watch) {
-    byte[] data = retryUntilConnected(new Callable<byte[]>() {
+  @SuppressWarnings("unchecked")
+  public <T extends Object> T readData(final String path, final Stat stat, final boolean watch) {
+    long startT = System.currentTimeMillis();
+    byte[] data = null;
+    try {
+      data = retryUntilConnected(new Callable<byte[]>() {
 
-      @Override public byte[] call() throws Exception {
-        return _connection.readData(path, stat, watch);
+        @Override public byte[] call() throws Exception {
+          return _connection.readData(path, stat, watch);
+        }
+      });
+      record(path, data, startT, ZkClientMonitor.AccessType.READ);
+      return (T) deserialize(data, path);
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.READ);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("getData, path: " + path + ", time: " + (endT - startT) + " ms");
       }
-    });
-    return (T) derializable(data);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public <T extends Object> T readDataAndStat(String path, Stat stat,
+      boolean returnNullIfPathNotExists) {
+    T data = null;
+    try {
+      data = readData(path, stat);
+    } catch (ZkNoNodeException e) {
+      if (!returnNullIfPathNotExists) {
+        throw e;
+      }
+    }
+    return data;
   }
 
   public void writeData(String path, Object object) {
@@ -1043,16 +1134,104 @@ public class ZkClient implements Watcher {
   }
 
   public Stat writeDataReturnStat(final String path, Object datat, final int expectedVersion) {
-    final byte[] data = serialize(datat);
-    return (Stat) retryUntilConnected(new Callable<Object>() {
+    long startT = System.currentTimeMillis();
+    try {
+      final byte[] data = serialize(datat, path);
+      checkDataSizeLimit(data);
+      final Stat stat = (Stat) retryUntilConnected(new Callable<Object>() {
+        @Override public Object call() throws Exception {
+          return _connection.writeDataReturnStat(path, data, expectedVersion);
+        }
+      });
+      record(path, data, startT, ZkClientMonitor.AccessType.WRITE);
+      return stat;
+    } catch (Exception e) {
+      recordFailure(path, ZkClientMonitor.AccessType.WRITE);
+      throw e;
+    } finally {
+      long endT = System.currentTimeMillis();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("setData, path: " + path + ", time: " + (endT - startT) + " ms");
+      }
+    }
+  }
+
+  public Stat writeDataGetStat(final String path, Object datat, final int expectedVersion) {
+    return writeDataReturnStat(path, datat, expectedVersion);
+  }
+
 
+  public void asyncCreate(final String path, Object datat, final CreateMode mode,
+      final ZkAsyncCallbacks.CreateCallbackHandler cb) {
+    final long startT = System.currentTimeMillis();
+    final byte[] data = (datat == null ? null : serialize(datat, path));
+    retryUntilConnected(new Callable<Object>() {
       @Override public Object call() throws Exception {
-        Stat stat = _connection.writeDataReturnStat(path, data, expectedVersion);
-        return stat;
+        ((ZkConnection) _connection).getZookeeper().create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE,
+            // Arrays.asList(DEFAULT_ACL),
+            mode, cb, new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT,
+                data == null ? 0 : data.length, false));
+        return null;
       }
     });
   }
 
+  // Async Data Accessors
+  public void asyncSetData(final String path, Object datat, final int version,
+      final ZkAsyncCallbacks.SetDataCallbackHandler cb) {
+    final long startT = System.currentTimeMillis();
+    final byte[] data = serialize(datat, path);
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        ((ZkConnection) _connection).getZookeeper().setData(path, data, version, cb,
+            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT,
+                data == null ? 0 : data.length, false));
+        return null;
+      }
+    });
+  }
+
+  public void asyncGetData(final String path, final ZkAsyncCallbacks.GetDataCallbackHandler cb) {
+    final long startT = System.currentTimeMillis();
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        ((ZkConnection) _connection).getZookeeper().getData(path, null, cb,
+            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, true));
+        return null;
+      }
+    });
+  }
+
+  public void asyncExists(final String path, final ZkAsyncCallbacks.ExistsCallbackHandler cb) {
+    final long startT = System.currentTimeMillis();
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        ((ZkConnection) _connection).getZookeeper().exists(path, null, cb,
+            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, true));
+        return null;
+      }
+    });
+  }
+
+  public void asyncDelete(final String path, final ZkAsyncCallbacks.DeleteCallbackHandler cb) {
+    final long startT = System.currentTimeMillis();
+    retryUntilConnected(new Callable<Object>() {
+      @Override public Object call() throws Exception {
+        ((ZkConnection) _connection).getZookeeper().delete(path, -1, cb,
+            new ZkAsyncCallbacks.ZkAsyncCallContext(_monitor, startT, 0, false));
+        return null;
+      }
+    });
+  }
+
+  private void checkDataSizeLimit(byte[] data) {
+    if (data != null && data.length > ZNRecord.SIZE_LIMIT) {
+      LOG.error("Data size larger than 1M, will not write to zk. Data (first 1k): "
+          + new String(data).substring(0, 1024));
+      throw new HelixException("Data size larger than 1M");
+    }
+  }
+
   public void watchForData(final String path) {
     retryUntilConnected(new Callable<Object>() {
       @Override public Object call() throws Exception {
@@ -1153,29 +1332,64 @@ public class ZkClient implements Watcher {
     }
   }
 
+  public String getServers() {
+    return _connection.getServers();
+  }
+
   /**
    * Close the client.
    *
    * @throws ZkInterruptedException
    */
   public void close() throws ZkInterruptedException {
-    if (_closed) {
-      return;
+    if (LOG.isTraceEnabled()) {
+      StackTraceElement[] calls = Thread.currentThread().getStackTrace();
+      LOG.trace("closing a zkclient. callStack: " + Arrays.asList(calls));
     }
-    LOG.debug("Closing ZkClient...");
     getEventLock().lock();
     try {
+      if (_connection == null || _closed) {
+        return;
+      }
+      LOG.info("Closing zkclient: " + ((ZkConnection) _connection).getZookeeper());
       setShutdownTrigger(true);
       _eventThread.interrupt();
       _eventThread.join(2000);
       _connection.close();
       _closed = true;
     } catch (InterruptedException e) {
-      throw new ZkInterruptedException(e);
+      /**
+       * Workaround for HELIX-264: calling ZkClient#close() in its own eventThread context will
+       * throw ZkInterruptedException and skip ZkConnection#close()
+       */
+      if (_connection != null) {
+        try {
+          /**
+           * ZkInterruptedException#construct() honors InterruptedException by calling
+           * Thread.currentThread().interrupt(); clear it first, so we can safely close the
+           * zk-connection
+           */
+          Thread.interrupted();
+          _connection.close();
+          /**
+           * restore interrupted status of current thread
+           */
+          Thread.currentThread().interrupt();
+        } catch (InterruptedException e1) {
+          throw new ZkInterruptedException(e1);
+        }
+      }
     } finally {
       getEventLock().unlock();
+      if (_monitor != null) {
+        _monitor.unregister();
+      }
+      LOG.info("Closed zkclient");
     }
-    LOG.debug("Closing ZkClient...done");
+  }
+
+  public boolean isClosed() {
+    return (_connection == null || !_connection.getZookeeperState().isAlive());
   }
 
   private void reconnect() {
@@ -1223,4 +1437,30 @@ public class ZkClient implements Watcher {
       }
     });
   }
+
+  // operations to update monitor's counters
+  private void record(String path, byte[] data, long startTimeMilliSec, ZkClientMonitor.AccessType accessType) {
+    if (_monitor != null) {
+      int dataSize = (data != null) ? data.length : 0;
+      _monitor.record(path, dataSize, startTimeMilliSec, accessType);
+    }
+  }
+
+  private void recordFailure(String path, ZkClientMonitor.AccessType accessType) {
+    if (_monitor != null) {
+      _monitor.recordFailure(path, accessType);
+    }
+  }
+
+  private void recordStateChange(boolean stateChanged, boolean dataChanged) {
+    // update state change counter.
+    if (_monitor != null) {
+      if (stateChanged) {
+        _monitor.increaseStateChangeEventCounter();
+      }
+      if (dataChanged) {
+        _monitor.increaseDataChangeEventCounter();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/310d4766/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
index 639fd8a..6cdf6e7 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientMonitor.java
@@ -31,6 +31,11 @@ public class ZkClientMonitor implements ZkClientMonitorMBean {
   public static final String MONITOR_TYPE = "Type";
   public static final String MONITOR_KEY = "Key";
 
+  public enum AccessType {
+    READ,
+    WRITE
+  }
+
   private ObjectName _objectName;
   private String _sensorName;
 
@@ -117,19 +122,30 @@ public class ZkClientMonitor implements ZkClientMonitorMBean {
     }
   }
 
-  public void recordReadFailure(String path) {
-    record(path, 0, 0, true, true);
-  }
-
-  public void recordRead(String path, int dataSize, long startTimeMilliSec) {
-    record(path, dataSize, System.currentTimeMillis() - startTimeMilliSec, false, true);
-  }
-
-  public void recordWriteFailure(String path) {
-    record(path, 0, 0, true, false);
+  public void record(String path, int dataSize, long startTimeMilliSec, AccessType accessType) {
+    switch (accessType) {
+    case READ:
+      record(path, dataSize, System.currentTimeMillis() - startTimeMilliSec, false, true);
+      return;
+    case WRITE:
+      record(path, dataSize, System.currentTimeMillis() - startTimeMilliSec, false, false);
+      return;
+
+    default:
+      return;
+    }
   }
 
-  public void recordWrite(String path, int dataSize, long startTimeMilliSec) {
-    record(path, dataSize, System.currentTimeMillis() - startTimeMilliSec, false, false);
+  public void recordFailure(String path, AccessType accessType) {
+    switch (accessType) {
+    case READ:
+      record(path, 0, 0, true, true);
+      return;
+    case WRITE:
+      record(path, 0, 0, true, false);
+      return;
+    default:
+      return;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/310d4766/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestZkClientMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestZkClientMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestZkClientMonitor.java
index 8bf136e..1b8099c 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestZkClientMonitor.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestZkClientMonitor.java
@@ -98,18 +98,20 @@ public class TestZkClientMonitor {
     long eventCount = (long) _beanServer.getAttribute(name, "DataChangeEventCounter");
     Assert.assertEquals(eventCount, 1);
 
-    monitor.recordRead("TEST/IDEALSTATES/myResource", 0, System.currentTimeMillis() - 10);
+    monitor.record("TEST/IDEALSTATES/myResource", 0, System.currentTimeMillis() - 10,
+        ZkClientMonitor.AccessType.READ);
     Assert.assertEquals((long) _beanServer.getAttribute(rootName, "ReadCounter"), 1);
     Assert.assertEquals((long) _beanServer.getAttribute(idealStateName, "ReadCounter"), 1);
     Assert.assertTrue((long) _beanServer.getAttribute(rootName, "ReadLatencyGauge.Max") >= 10);
-    monitor.recordRead("TEST/INSTANCES/testDB0", 0, System.currentTimeMillis() - 15);
+    monitor.record("TEST/INSTANCES/testDB0", 0, System.currentTimeMillis() - 15,
+        ZkClientMonitor.AccessType.READ);
     Assert.assertEquals((long) _beanServer.getAttribute(rootName, "ReadCounter"), 2);
     Assert.assertEquals((long) _beanServer.getAttribute(instancesName, "ReadCounter"), 1);
     Assert.assertEquals((long) _beanServer.getAttribute(idealStateName, "ReadCounter"), 1);
     Assert.assertTrue((long) _beanServer.getAttribute(rootName, "ReadTotalLatencyCounter") >= 25);
 
-    monitor.recordWrite("TEST/INSTANCES/node_1/CURRENTSTATES/session_1/Resource", 5,
-        System.currentTimeMillis() - 10);
+    monitor.record("TEST/INSTANCES/node_1/CURRENTSTATES/session_1/Resource", 5,
+        System.currentTimeMillis() - 10, ZkClientMonitor.AccessType.WRITE);
     Assert.assertEquals((long) _beanServer.getAttribute(rootName, "WriteCounter"), 1);
     Assert.assertEquals((long) _beanServer.getAttribute(currentStateName, "WriteCounter"), 1);
     Assert.assertEquals((long) _beanServer.getAttribute(currentStateName, "WriteBytesCounter"), 5);


[44/50] [abbrv] helix git commit: [helix-front] Upgrade Angular to 5; Material to 5; ngx-datatable to 11

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/node-viewer/node-viewer.component.spec.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/node-viewer/node-viewer.component.spec.ts b/helix-front/client/app/shared/node-viewer/node-viewer.component.spec.ts
index 40f57b9..d66b1ee 100644
--- a/helix-front/client/app/shared/node-viewer/node-viewer.component.spec.ts
+++ b/helix-front/client/app/shared/node-viewer/node-viewer.component.spec.ts
@@ -1,8 +1,6 @@
 import { async, ComponentFixture, TestBed } from '@angular/core/testing';
 import { NO_ERRORS_SCHEMA } from '@angular/core';
-import { MaterialModule } from '@angular/material';
-import { NoopAnimationsModule } from '@angular/platform-browser/animations';
-import { RouterTestingModule } from '@angular/router/testing';
+import { TestingModule } from '../../../testing/testing.module';
 
 import { NodeViewerComponent } from './node-viewer.component';
 
@@ -13,9 +11,7 @@ describe('NodeViewerComponent', () => {
   beforeEach(async(() => {
     TestBed.configureTestingModule({
       imports: [
-        MaterialModule,
-        NoopAnimationsModule,
-        RouterTestingModule
+        TestingModule
       ],
       declarations: [ NodeViewerComponent ],
       schemas: [

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/node-viewer/node-viewer.component.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/node-viewer/node-viewer.component.ts b/helix-front/client/app/shared/node-viewer/node-viewer.component.ts
index 9842568..316cbbf 100644
--- a/helix-front/client/app/shared/node-viewer/node-viewer.component.ts
+++ b/helix-front/client/app/shared/node-viewer/node-viewer.component.ts
@@ -1,6 +1,6 @@
 import { Component, OnInit, Input, Output, ViewChild, ViewEncapsulation, EventEmitter } from '@angular/core';
 import { ActivatedRoute } from '@angular/router';
-import { MdDialog } from '@angular/material';
+import { MatDialog } from '@angular/material';
 
 import * as _ from 'lodash';
 
@@ -63,6 +63,7 @@ export class NodeViewerComponent implements OnInit {
   protected _obj: any;
   protected node: Node;
 
+  headerHeight = Settings.tableHeaderHeight;
   rowHeight = Settings.tableRowHeight;
   sorts = [
     { prop: 'name', dir: 'asc'}
@@ -117,7 +118,7 @@ export class NodeViewerComponent implements OnInit {
   }
 
   constructor(
-    protected dialog: MdDialog,
+    protected dialog: MatDialog,
     protected route: ActivatedRoute
   ) { }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/shared.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/shared.module.ts b/helix-front/client/app/shared/shared.module.ts
index 017a637..0aa75d1 100644
--- a/helix-front/client/app/shared/shared.module.ts
+++ b/helix-front/client/app/shared/shared.module.ts
@@ -1,13 +1,13 @@
 import { NgModule } from '@angular/core';
 import { CommonModule } from '@angular/common';
 import { RouterModule } from '@angular/router';
-import { MaterialModule } from '@angular/material';
 import { FlexLayoutModule } from '@angular/flex-layout';
 import { FormsModule } from '@angular/forms';
 
 import { NgxDatatableModule } from '@swimlane/ngx-datatable';
 import { NgxJsonViewerModule } from 'ngx-json-viewer';
 
+import { MaterialModule } from './material.module';
 import { HelperService } from './helper.service';
 import { InputDialogComponent } from './dialog/input-dialog/input-dialog.component';
 import { DetailHeaderComponent } from './detail-header/detail-header.component';

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/shared/state-label/state-label.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/state-label/state-label.component.html b/helix-front/client/app/shared/state-label/state-label.component.html
index 6641a15..8b98d7e 100644
--- a/helix-front/client/app/shared/state-label/state-label.component.html
+++ b/helix-front/client/app/shared/state-label/state-label.component.html
@@ -3,5 +3,5 @@
   'state-label-ready': isReady,
   'state-label-not-ready': isReady == false
 }">
-  {{ state }}
+  {{ state || 'NO STATE' }}
 </span>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/workflow/workflow-detail/workflow-detail.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/workflow/workflow-detail/workflow-detail.component.html b/helix-front/client/app/workflow/workflow-detail/workflow-detail.component.html
index 38b3128..848c6c2 100644
--- a/helix-front/client/app/workflow/workflow-detail/workflow-detail.component.html
+++ b/helix-front/client/app/workflow/workflow-detail/workflow-detail.component.html
@@ -1,7 +1,9 @@
-<md-toolbar class="mat-elevation-z1">
-  <hi-detail-header [cluster]="clusterName" [workflow]="workflow?.name"></hi-detail-header>
-  <md-toolbar-row class="information">
-    <a md-mini-fab routerLink="../"><md-icon>arrow_back</md-icon></a>
+<mat-toolbar class="mat-elevation-z1">
+  <mat-toolbar-row>
+    <hi-detail-header [cluster]="clusterName" [workflow]="workflow?.name"></hi-detail-header>
+  </mat-toolbar-row>
+  <mat-toolbar-row class="information">
+    <a mat-mini-fab routerLink="../"><mat-icon>arrow_back</mat-icon></a>
     <hi-key-value-pairs [obj]="workflow">
       <hi-key-value-pair name="Capacity" prop="config.capacity"></hi-key-value-pair>
       <hi-key-value-pair name="Target State" prop="config.TargetState"></hi-key-value-pair>
@@ -10,10 +12,10 @@
       <hi-key-value-pair name="Failure Threshold" prop="config.FailureThreshold"></hi-key-value-pair>
       <hi-key-value-pair name="Expiry" prop="config.Expiry"></hi-key-value-pair>
     </hi-key-value-pairs>
-  </md-toolbar-row>
-</md-toolbar>
+  </mat-toolbar-row>
+</mat-toolbar>
 <section fxLayout="column" fxLayoutAlign="center center">
-  <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-spinner *ngIf="isLoading"></mat-spinner>
   <section class="content" fxFlexFill>
     <ngx-json-viewer [json]="workflow"></ngx-json-viewer>
   </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/app/workflow/workflow-list/workflow-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/workflow/workflow-list/workflow-list.component.html b/helix-front/client/app/workflow/workflow-list/workflow-list.component.html
index f3b642f..9c4222c 100644
--- a/helix-front/client/app/workflow/workflow-list/workflow-list.component.html
+++ b/helix-front/client/app/workflow/workflow-list/workflow-list.component.html
@@ -1,15 +1,15 @@
 <section fxLayout="column" fxLayoutAlign="center center">
-  <md-spinner *ngIf="isLoading"></md-spinner>
+  <mat-spinner *ngIf="isLoading"></mat-spinner>
   <section fxFlexFill>
     <section *ngIf="!isLoading && workflows.length == 0" class="empty">
       There's no workflow here.
     </section>
-    <md-nav-list>
+    <mat-nav-list>
       <a *ngFor="let name of workflows"
-        md-list-item
+        mat-list-item
         [routerLink]="[name]">
-        <div md-line>{{ name }}</div>
+        <div mat-line>{{ name }}</div>
       </a>
-    </md-nav-list>
+    </mat-nav-list>
   </section>
 </section>

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/styles.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/styles.scss b/helix-front/client/styles.scss
index 2161b18..9a64f47 100644
--- a/helix-front/client/styles.scss
+++ b/helix-front/client/styles.scss
@@ -18,6 +18,10 @@
       &.active,
       &.active .datatable-row-group {
         background-color: #0077B5 !important;
+
+        .datatable-body-cell {
+          color: #FFF !important;
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/testing/testing.module.ts
----------------------------------------------------------------------
diff --git a/helix-front/client/testing/testing.module.ts b/helix-front/client/testing/testing.module.ts
index 36ba18d..632a3c7 100644
--- a/helix-front/client/testing/testing.module.ts
+++ b/helix-front/client/testing/testing.module.ts
@@ -1,7 +1,8 @@
 import { NgModule } from '@angular/core';
 import { HttpModule } from '@angular/http';
-import { MaterialModule } from '@angular/material';
+import { MaterialModule } from '../app/shared/material.module';
 import { RouterTestingModule } from '@angular/router/testing';
+import { NoopAnimationsModule } from '@angular/platform-browser/animations';
 
 import { HelperService } from '../app/shared/helper.service';
 import { HelperServiceStub } from './stubs';
@@ -10,7 +11,8 @@ import { HelperServiceStub } from './stubs';
   imports: [
     HttpModule,
     MaterialModule,
-    RouterTestingModule
+    RouterTestingModule,
+    NoopAnimationsModule
   ],
   providers: [
     {
@@ -21,7 +23,8 @@ import { HelperServiceStub } from './stubs';
   exports: [
     HttpModule,
     MaterialModule,
-    RouterTestingModule
+    RouterTestingModule,
+    NoopAnimationsModule
   ]
 })
 export class TestingModule { }

http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/client/tsconfig.app.json
----------------------------------------------------------------------
diff --git a/helix-front/client/tsconfig.app.json b/helix-front/client/tsconfig.app.json
index 5e2507d..126454c 100644
--- a/helix-front/client/tsconfig.app.json
+++ b/helix-front/client/tsconfig.app.json
@@ -9,5 +9,9 @@
   "exclude": [
     "test.ts",
     "**/*.spec.ts"
+  ],
+  "include": [
+    "**/*",
+    "../node_modules/ngx-json-viewer/index.ts"
   ]
 }


[22/50] [abbrv] helix git commit: Add capability to specify different session store

Posted by jx...@apache.org.
Add capability to specify different session store


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

Branch: refs/heads/master
Commit: b1540f08d72c9c564f3e9b6e9744a2042485fa75
Parents: 093f7ab
Author: Vivo Xu <vx...@linkedin.com>
Authored: Mon Nov 27 17:47:41 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:56 2018 -0800

----------------------------------------------------------------------
 helix-front/server/app.ts               |  8 +++++++-
 helix-front/server/config.ts            |  6 ++++--
 helix-front/server/controllers/helix.ts |  9 ++-------
 helix-front/server/controllers/user.ts  | 18 +++++++++++-------
 4 files changed, 24 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/b1540f08/helix-front/server/app.ts
----------------------------------------------------------------------
diff --git a/helix-front/server/app.ts b/helix-front/server/app.ts
index 5d9568c..b35aef6 100644
--- a/helix-front/server/app.ts
+++ b/helix-front/server/app.ts
@@ -8,7 +8,7 @@ import * as http from 'http';
 import * as https from 'https';
 import * as session from 'express-session';
 
-import { SSL } from './config';
+import { SSL, SESSION_STORE } from './config';
 import setRoutes from './routes';
 
 const app = express();
@@ -21,6 +21,7 @@ app.use('/', express.static(path.join(__dirname, '../public')));
 app.use(bodyParser.json());
 app.use(bodyParser.urlencoded({ extended: true }));
 app.use(session({
+  store: SESSION_STORE,
   secret: 'helix',
   resave: true,
   saveUninitialized: true,
@@ -39,6 +40,11 @@ server.listen(app.get('port'), () => {
   console.log(`App is listening on port ${ app.get('port') } as HTTP`);
 });
 
+process.on('uncaughtException', function(err){
+  console.error('uncaughtException: ' + err.message);
+  console.error(err.stack);
+});
+
 // setup SSL
 if (SSL.port > 0 && fs.existsSync(SSL.keyfile) && fs.existsSync(SSL.certfile)) {
   let credentials: any = {

http://git-wip-us.apache.org/repos/asf/helix/blob/b1540f08/helix-front/server/config.ts
----------------------------------------------------------------------
diff --git a/helix-front/server/config.ts b/helix-front/server/config.ts
index c49c5f7..1f49dd2 100644
--- a/helix-front/server/config.ts
+++ b/helix-front/server/config.ts
@@ -4,6 +4,8 @@ export const HELIX_ENDPOINTS = {
   }]
 };
 
+export const SESSION_STORE = undefined;
+
 export const SSL = {
   port: 0,
   keyfile: '',
@@ -12,6 +14,6 @@ export const SSL = {
   cafiles: []
 };
 
-export function IsAdmin(username: string) {
-  return username === 'root';
+export function CheckAdmin(username: string, callback: (boolean) => void) {
+  callback(username === 'root');
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/b1540f08/helix-front/server/controllers/helix.ts
----------------------------------------------------------------------
diff --git a/helix-front/server/controllers/helix.ts b/helix-front/server/controllers/helix.ts
index ee8b412..9d7fcf5 100644
--- a/helix-front/server/controllers/helix.ts
+++ b/helix-front/server/controllers/helix.ts
@@ -2,7 +2,7 @@ import { Request, Response, Router } from 'express';
 
 import * as request from 'request';
 
-import { HELIX_ENDPOINTS, IsAdmin } from '../config';
+import { HELIX_ENDPOINTS } from '../config';
 
 export class HelixCtrl {
 
@@ -25,7 +25,7 @@ export class HelixCtrl {
 
     const user = req.session.username;
     const method = req.method.toLowerCase();
-    if (method != 'get' && !IsAdmin(user)) {
+    if (method != 'get' && !req.session.isAdmin) {
       res.status(403).send('Forbidden');
       return;
     }
@@ -58,11 +58,6 @@ export class HelixCtrl {
     } else {
       res.status(404).send('Not found');
     }
-
-    process.on('uncaughtException', function(err){
-      console.error('uncaughtException: ' + err.message);
-      console.error(err.stack);
-    });
   }
 
   protected list(req: Request, res: Response) {

http://git-wip-us.apache.org/repos/asf/helix/blob/b1540f08/helix-front/server/controllers/user.ts
----------------------------------------------------------------------
diff --git a/helix-front/server/controllers/user.ts b/helix-front/server/controllers/user.ts
index e12b08f..84d2c11 100644
--- a/helix-front/server/controllers/user.ts
+++ b/helix-front/server/controllers/user.ts
@@ -2,7 +2,7 @@ import { Request, Response, Router } from 'express';
 
 import * as request from 'request';
 
-import { IsAdmin } from '../config';
+import { CheckAdmin } from '../config';
 
 export class UserCtrl {
 
@@ -16,11 +16,15 @@ export class UserCtrl {
   protected authorize(req: Request, res: Response) {
     if (req.query.name) {
       req.session.username = req.query.name;
-      if (req.query.url) {
-        res.redirect(req.query.url);
-      } else {
-        res.redirect('/');
-      }
+      CheckAdmin(req.session.username, (isAdmin: boolean) => {
+        req.session.isAdmin = isAdmin;
+
+        if (req.query.url) {
+          res.redirect(req.query.url);
+        } else {
+          res.redirect('/');
+        }
+      });
     } else {
       res.status(401).send('Unauthorized');
     }
@@ -31,6 +35,6 @@ export class UserCtrl {
   }
 
   protected can(req: Request, res: Response) {
-    res.json(IsAdmin(req.session.username));
+    res.json(req.session.isAdmin ? true : false);
   }
 }


[20/50] [abbrv] helix git commit: Temporary disable batch enable/disable API feature

Posted by jx...@apache.org.
Temporary disable batch enable/disable API feature


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

Branch: refs/heads/master
Commit: 2f3a56f73eb8d184c9d371a74ee2e195a6372b72
Parents: de2e3e8
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Nov 16 12:18:39 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:48 2018 -0800

----------------------------------------------------------------------
 .../org/apache/helix/manager/zk/ZKHelixAdmin.java     | 14 ++++++++++++--
 .../helix/integration/TestBatchEnableInstances.java   |  8 ++++----
 2 files changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/2f3a56f7/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index 1af881e..546352c 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -195,13 +195,18 @@ public class ZKHelixAdmin implements HelixAdmin {
       final boolean enabled) {
     BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
     enableSingleInstance(clusterName, instanceName, enabled, baseAccessor);
-    enableBatchInstances(clusterName, Collections.singletonList(instanceName), enabled,
-        baseAccessor);
+    // TODO: Reenable this after storage node bug fixed.
+    // enableBatchInstances(clusterName, Collections.singletonList(instanceName), enabled, baseAccessor);
+
   }
 
   @Override
   public void enableInstance(String clusterName, List<String> instances,
       boolean enabled) {
+    // TODO: Reenable this after storage node bug fixed.
+    if (true) {
+      throw new HelixException("Current batch enable/disable instances are temporarily disabled!");
+    }
     BaseDataAccessor<ZNRecord> baseAccessor = new ZkBaseDataAccessor<>(_zkClient);
     if (enabled) {
       for (String instance : instances) {
@@ -1296,6 +1301,11 @@ public class ZKHelixAdmin implements HelixAdmin {
 
   private void enableBatchInstances(final String clusterName, final List<String> instances,
       final boolean enabled, BaseDataAccessor<ZNRecord> baseAccessor) {
+    // TODO : Due to Espresso storage node depends on map field. Current disable the feature now
+    // include tests.
+    if (true) {
+      throw new HelixException("Current batch enable/disable instances are temporarily disabled!");
+    }
 
     String path = PropertyPathBuilder.clusterConfig(clusterName);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/2f3a56f7/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
index eab5c32..1c0a0f3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchEnableInstances.java
@@ -25,7 +25,7 @@ public class TestBatchEnableInstances extends TaskTestBase {
     _accessor = new ConfigAccessor(_gZkClient);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testOldEnableDisable() throws InterruptedException {
     _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), false);
@@ -41,7 +41,7 @@ public class TestBatchEnableInstances extends TaskTestBase {
         .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), true);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testBatchEnableDisable() throws InterruptedException {
     _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
         Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),
@@ -60,7 +60,7 @@ public class TestBatchEnableInstances extends TaskTestBase {
         true);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testOldDisableBatchEnable() throws InterruptedException {
     _gSetupTool.getClusterManagementTool()
         .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), false);
@@ -83,7 +83,7 @@ public class TestBatchEnableInstances extends TaskTestBase {
         .enableInstance(CLUSTER_NAME, _participants[0].getInstanceName(), true);
   }
 
-  @Test
+  @Test (enabled = false)
   public void testBatchDisableOldEnable() throws InterruptedException {
     _gSetupTool.getClusterManagementTool().enableInstance(CLUSTER_NAME,
         Arrays.asList(_participants[0].getInstanceName(), _participants[1].getInstanceName()),


[30/50] [abbrv] helix git commit: [helix-front] Added pager into configuration page and partition placement table

Posted by jx...@apache.org.
[helix-front] Added pager into configuration page and partition placement table


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

Branch: refs/heads/master
Commit: 1f7417ac45595dc594060b3c566f6deda1bd083f
Parents: d9052a3
Author: Vivo Xu <vx...@linkedin.com>
Authored: Mon Dec 11 11:37:32 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:25 2018 -0800

----------------------------------------------------------------------
 .../partition-list.component.html               | 27 ++++++++++++++
 .../partition-list.component.scss               |  5 +++
 .../shared/data-table/data-table.component.html | 20 ++++++++++-
 .../node-viewer/node-viewer.component.html      | 38 ++++++++++++++++++--
 helix-front/package.json                        |  2 +-
 5 files changed, 88 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/1f7417ac/helix-front/client/app/resource/partition-list/partition-list.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-list/partition-list.component.html b/helix-front/client/app/resource/partition-list/partition-list.component.html
index 64119bc..238d5e8 100644
--- a/helix-front/client/app/resource/partition-list/partition-list.component.html
+++ b/helix-front/client/app/resource/partition-list/partition-list.component.html
@@ -9,6 +9,7 @@
     [footerHeight]="rowHeight"
     [rows]="partitions"
     [sorts]="sorts"
+    [limit]="20"
     selectionType="single"
     (select)="onSelect($event)">
     <ngx-datatable-column
@@ -49,6 +50,32 @@
         <hi-partition-detail [clusterName]="resource.cluster" [partition]="row"></hi-partition-detail>
       </ng-template>
     </ngx-datatable-row-detail>
+    <ngx-datatable-footer>
+      <ng-template
+        ngx-datatable-footer-template
+        let-rowCount="rowCount"
+        let-pageSize="pageSize"
+        let-curPage="curPage">
+        <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
+          <section>
+            {{ rowCount }} total
+          </section>
+          <section>
+            <datatable-pager
+              [pagerLeftArrowIcon]="'datatable-icon-left'"
+              [pagerRightArrowIcon]="'datatable-icon-right'"
+              [pagerPreviousIcon]="'datatable-icon-prev'"
+              [pagerNextIcon]="'datatable-icon-skip'"
+              [page]="curPage"
+              [size]="pageSize"
+              [count]="rowCount"
+              [hidden]="!((rowCount / pageSize) > 1)"
+              (change)="partitionsTable.onFooterPage($event)">
+            </datatable-pager>
+          </section>
+        </section>
+      </ng-template>
+    </ngx-datatable-footer>
   </ngx-datatable>
   <div *ngIf="!canAnalyse()" class="message" fxLayout="column" fxLayoutAlign="center center">
     <md-spinner *ngIf="isLoading"></md-spinner>

http://git-wip-us.apache.org/repos/asf/helix/blob/1f7417ac/helix-front/client/app/resource/partition-list/partition-list.component.scss
----------------------------------------------------------------------
diff --git a/helix-front/client/app/resource/partition-list/partition-list.component.scss b/helix-front/client/app/resource/partition-list/partition-list.component.scss
index 493f45e..3903a76 100644
--- a/helix-front/client/app/resource/partition-list/partition-list.component.scss
+++ b/helix-front/client/app/resource/partition-list/partition-list.component.scss
@@ -11,3 +11,8 @@ div.message {
 .status-not-ready {
   color: mat-color(mat-palette($mat-orange), darker);
 }
+
+.footer {
+  width: 100%;
+  padding: 0 20px;
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/1f7417ac/helix-front/client/app/shared/data-table/data-table.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/data-table/data-table.component.html b/helix-front/client/app/shared/data-table/data-table.component.html
index 8ae7bbe..e665361 100644
--- a/helix-front/client/app/shared/data-table/data-table.component.html
+++ b/helix-front/client/app/shared/data-table/data-table.component.html
@@ -1,4 +1,5 @@
 <ngx-datatable
+  #dataTable
   class="material"
   [headerHeight]="rowHeight"
   rowHeight="auto"
@@ -37,7 +38,11 @@
     </ng-template>
   </ngx-datatable-column>
   <ngx-datatable-footer>
-    <ng-template ngx-datatable-footer-template let-rowCount="rowCount">
+    <ng-template
+      ngx-datatable-footer-template
+      let-rowCount="rowCount"
+      let-pageSize="pageSize"
+      let-curPage="curPage">
       <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
         <button md-button *ngIf="insertable" (click)="onCreate()">
           <md-icon>add</md-icon>
@@ -46,6 +51,19 @@
         <section>
           {{ rowCount }} total
         </section>
+        <section>
+          <datatable-pager
+            [pagerLeftArrowIcon]="'datatable-icon-left'"
+            [pagerRightArrowIcon]="'datatable-icon-right'"
+            [pagerPreviousIcon]="'datatable-icon-prev'"
+            [pagerNextIcon]="'datatable-icon-skip'"
+            [page]="curPage"
+            [size]="pageSize"
+            [count]="rowCount"
+            [hidden]="!((rowCount / pageSize) > 1)"
+            (change)="dataTable.onFooterPage($event)">
+          </datatable-pager>
+        </section>
       </section>
     </ng-template>
   </ngx-datatable-footer>

http://git-wip-us.apache.org/repos/asf/helix/blob/1f7417ac/helix-front/client/app/shared/node-viewer/node-viewer.component.html
----------------------------------------------------------------------
diff --git a/helix-front/client/app/shared/node-viewer/node-viewer.component.html b/helix-front/client/app/shared/node-viewer/node-viewer.component.html
index 4b5b253..006761b 100644
--- a/helix-front/client/app/shared/node-viewer/node-viewer.component.html
+++ b/helix-front/client/app/shared/node-viewer/node-viewer.component.html
@@ -103,7 +103,11 @@
               </ng-template>
             </ngx-datatable-column>
             <ngx-datatable-footer>
-              <ng-template ngx-datatable-footer-template let-rowCount="rowCount">
+              <ng-template
+                ngx-datatable-footer-template
+                let-rowCount="rowCount"
+                let-pageSize="pageSize"
+                let-curPage="curPage">
                 <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
                   <button md-button *ngIf="editable" (click)="onCreate('list')">
                     <md-icon>add</md-icon>
@@ -112,6 +116,19 @@
                   <section>
                     {{ rowCount }} total
                   </section>
+                  <section>
+                    <datatable-pager
+                      [pagerLeftArrowIcon]="'datatable-icon-left'"
+                      [pagerRightArrowIcon]="'datatable-icon-right'"
+                      [pagerPreviousIcon]="'datatable-icon-prev'"
+                      [pagerNextIcon]="'datatable-icon-skip'"
+                      [page]="curPage"
+                      [size]="pageSize"
+                      [count]="rowCount"
+                      [hidden]="!((rowCount / pageSize) > 1)"
+                      (change)="listTable.onFooterPage($event)">
+                    </datatable-pager>
+                  </section>
                 </section>
               </ng-template>
             </ngx-datatable-footer>
@@ -167,7 +184,11 @@
               </ng-template>
             </ngx-datatable-column>
             <ngx-datatable-footer>
-              <ng-template ngx-datatable-footer-template let-rowCount="rowCount">
+              <ng-template
+                ngx-datatable-footer-template
+                let-rowCount="rowCount"
+                let-pageSize="pageSize"
+                let-curPage="curPage">
                 <section class="footer" fxLayout="row" fxLayoutAlign="space-between center">
                   <button md-button *ngIf="editable" (click)="onCreate('map')">
                     <md-icon>add</md-icon>
@@ -176,6 +197,19 @@
                   <section>
                     {{ rowCount }} total
                   </section>
+                  <section>
+                    <datatable-pager
+                      [pagerLeftArrowIcon]="'datatable-icon-left'"
+                      [pagerRightArrowIcon]="'datatable-icon-right'"
+                      [pagerPreviousIcon]="'datatable-icon-prev'"
+                      [pagerNextIcon]="'datatable-icon-skip'"
+                      [page]="curPage"
+                      [size]="pageSize"
+                      [count]="rowCount"
+                      [hidden]="!((rowCount / pageSize) > 1)"
+                      (change)="mapTable.onFooterPage($event)">
+                    </datatable-pager>
+                  </section>
                 </section>
               </ng-template>
             </ngx-datatable-footer>

http://git-wip-us.apache.org/repos/asf/helix/blob/1f7417ac/helix-front/package.json
----------------------------------------------------------------------
diff --git a/helix-front/package.json b/helix-front/package.json
index 7873329..567d39c 100644
--- a/helix-front/package.json
+++ b/helix-front/package.json
@@ -7,7 +7,7 @@
   "scripts": {
     "ng": "ng",
     "build": "rm -rf dist && mkdir dist && ng build -aot -prod && tsc -p server",
-    "start": "concurrently \"ng serve -pc proxy.conf.json\" \"tsc -w -p server\" \"nodemon dist/server/app.js\"",
+    "start": "concurrently -r \"ng serve -pc proxy.conf.json\" \"tsc -w -p server\" \"nodemon dist/server/app.js\"",
     "prod": "npm run build && node dist/server/app.js",
     "test": "ng test",
     "lint": "ng lint",


[31/50] [abbrv] helix git commit: Change error record dump threshold to 100, and remove log if there is no paths to dump.

Posted by jx...@apache.org.
Change error record dump threshold to 100, and remove log if there is no paths to dump.


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

Branch: refs/heads/master
Commit: 9b9da195412b2dccc8414d7cd6f804414f527cff
Parents: 1f7417a
Author: Lei Xia <lx...@linkedin.com>
Authored: Wed Dec 13 14:42:00 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:32:29 2018 -0800

----------------------------------------------------------------------
 .../apache/helix/manager/zk/ZKHelixManager.java |  2 +-
 .../helix/monitoring/ZKPathDataDumpTask.java    | 20 +++++++++++---------
 2 files changed, 12 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/9b9da195/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 60b9a3f..dee9adf 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
@@ -145,7 +145,7 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
       long period = 15 * 60 * 1000;
       long timeThresholdNoChangeForStatusUpdates = 15 * 60 * 1000; // 15 minutes
       long timeThresholdNoChangeForErrors = 24 * 60 * 60 * 1000; // 1 day
-      int maximumNumberOfLeafNodesAllowed = 10000;
+      int maximumNumberOfLeafNodesAllowed = 100;
 
       if (_timer == null) {
         LOG.info("Start StatusDumpTask");

http://git-wip-us.apache.org/repos/asf/helix/blob/9b9da195/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
index 1b723d9..0b70e41 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/ZKPathDataDumpTask.java
@@ -138,16 +138,18 @@ public class ZKPathDataDumpTask extends TimerTask {
       }
     }
 
-    // dump
-    LOG.info("Dump statusUpdates and errors records for pahts: " + dumpPaths);
-    List<ZNRecord> dumpRecords = accessor.get(dumpPaths, null, 0);
-    for (ZNRecord record : dumpRecords) {
-      if (record != null) {
-        LOG.info(new String(_jsonSerializer.serialize(record)));
+    if (!dumpPaths.isEmpty()) {
+      LOG.info("Dump statusUpdates and errors records for paths: " + dumpPaths);
+      List<ZNRecord> dumpRecords = accessor.get(dumpPaths, null, 0);
+      for (ZNRecord record : dumpRecords) {
+        if (record != null) {
+          LOG.info(new String(_jsonSerializer.serialize(record)));
+        }
       }
-    }
 
-    // clean up
-    accessor.remove(dumpPaths, 0);
+      // clean up
+      accessor.remove(dumpPaths, 0);
+      LOG.info("Remove statusUpdates and errors records for paths: " + dumpPaths);
+    }
   }
 }


[41/50] [abbrv] helix git commit: Add Workflow and Job latency metrics

Posted by jx...@apache.org.
Add Workflow and Job latency metrics

To understand the workflow and job execution pattern, adding these metrics to monitor the process.

Only succeeded workflows and jobs will be recorded. Otherwise the data is not clean for workflow or job latency.


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

Branch: refs/heads/master
Commit: 52d3bb83c6c73316099f94de4d732e8c36c7171d
Parents: 4d2734e
Author: Junkai Xue <jx...@linkedin.com>
Authored: Tue Jan 2 15:52:30 2018 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:33:07 2018 -0800

----------------------------------------------------------------------
 .../monitoring/mbeans/ClusterStatusMonitor.java | 13 ++++++--
 .../helix/monitoring/mbeans/JobMonitor.java     | 32 ++++++++++++++++++++
 .../monitoring/mbeans/JobMonitorMBean.java      | 12 ++++++++
 .../monitoring/mbeans/WorkflowMonitor.java      | 32 +++++++++++++++++++-
 .../monitoring/mbeans/WorkflowMonitorMBean.java | 12 ++++++++
 .../org/apache/helix/task/JobRebalancer.java    |  6 ++--
 .../org/apache/helix/task/TaskRebalancer.java   |  7 +++--
 7 files changed, 105 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
index 61f4ce1..2a99341 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
@@ -515,11 +515,14 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
       updateWorkflowGauges(workflowConfigMap.get(workflow), currentState);
     }
   }
-
   public void updateWorkflowCounters(WorkflowConfig workflowConfig, TaskState to) {
+    updateWorkflowCounters(workflowConfig, to, -1L);
+  }
+
+  public void updateWorkflowCounters(WorkflowConfig workflowConfig, TaskState to, long latency) {
     String workflowType = workflowConfig.getWorkflowType();
     workflowType = preProcessWorkflow(workflowType);
-    _perTypeWorkflowMonitorMap.get(workflowType).updateWorkflowCounters(to);
+    _perTypeWorkflowMonitorMap.get(workflowType).updateWorkflowCounters(to, latency);
   }
 
   private void updateWorkflowGauges(WorkflowConfig workflowConfig, TaskState current) {
@@ -568,9 +571,13 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
   }
 
   public void updateJobCounters(JobConfig jobConfig, TaskState to) {
+    updateJobCounters(jobConfig, to, -1L);
+  }
+
+  public void updateJobCounters(JobConfig jobConfig, TaskState to, long latency) {
     String jobType = jobConfig.getJobType();
     jobType = preProcessJobMonitor(jobType);
-    _perTypeJobMonitorMap.get(jobType).updateJobCounters(to);
+    _perTypeJobMonitorMap.get(jobType).updateJobCounters(to, latency);
   }
 
   private void updateJobGauges(String jobType, TaskState current) {

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitor.java
index 91f0b73..39108cf 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitor.java
@@ -28,6 +28,7 @@ public class JobMonitor implements JobMonitorMBean {
 
   private static final String JOB_KEY = "Job";
   private static final Logger LOG = LoggerFactory.getLogger(JobMonitor.class);
+  private static final long DEFAULT_RESET_INTERVAL_MS = 60 * 60 * 1000; // 1 hour
 
   private String _clusterName;
   private String _jobType;
@@ -38,6 +39,9 @@ public class JobMonitor implements JobMonitorMBean {
   private long _existingJobGauge;
   private long _queuedJobGauge;
   private long _runningJobGauge;
+  private long _maximumJobLatencyGauge;
+  private long _jobLatencyCount;
+  private long _lastResetTime;
 
   public JobMonitor(String clusterName, String jobType) {
     _clusterName = clusterName;
@@ -48,6 +52,9 @@ public class JobMonitor implements JobMonitorMBean {
     _existingJobGauge = 0L;
     _queuedJobGauge = 0L;
     _runningJobGauge = 0L;
+    _lastResetTime = System.currentTimeMillis();
+    _jobLatencyCount = 0L;
+    _maximumJobLatencyGauge = 0L;
   }
 
   @Override
@@ -81,6 +88,16 @@ public class JobMonitor implements JobMonitorMBean {
   }
 
   @Override
+  public long getMaximumJobLatencyGauge() {
+    return _maximumJobLatencyGauge;
+  }
+
+  @Override
+  public long getJobLatencyCount() {
+    return _jobLatencyCount;
+  }
+
+  @Override
   public String getSensorName() {
     return String.format("%s.%s.%s", _clusterName, JOB_KEY, _jobType);
   }
@@ -93,15 +110,26 @@ public class JobMonitor implements JobMonitorMBean {
    * Update job counters with transition state
    * @param to The to state of job, cleaned by ZK when it is null
    */
+
   public void updateJobCounters(TaskState to) {
+    updateJobCounters(to, 0);
+  }
+
+  public void updateJobCounters(TaskState to, long latency) {
     // TODO maybe use separate TIMED_OUT counter later
     if (to.equals(TaskState.FAILED) || to.equals(TaskState.TIMED_OUT)) {
       _failedJobCount++;
     } else if (to.equals(TaskState.COMPLETED)) {
       _successfullJobCount++;
+
+      // Only count succeeded jobs
+      _maximumJobLatencyGauge = Math.max(_maximumJobLatencyGauge, latency);
+      _jobLatencyCount += latency > 0 ? latency : 0;
     } else if (to.equals(TaskState.ABORTED)) {
       _abortedJobCount++;
     }
+
+
   }
 
   /**
@@ -111,6 +139,10 @@ public class JobMonitor implements JobMonitorMBean {
     _queuedJobGauge = 0L;
     _existingJobGauge = 0L;
     _runningJobGauge = 0L;
+    if (_lastResetTime + DEFAULT_RESET_INTERVAL_MS < System.currentTimeMillis()) {
+      _lastResetTime = System.currentTimeMillis();
+      _maximumJobLatencyGauge = 0L;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitorMBean.java
index 5d30ec9..23e4a93 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/JobMonitorMBean.java
@@ -61,4 +61,16 @@ public interface JobMonitorMBean extends SensorNameProvider {
    * @return
    */
   public long getRunningJobGauge();
+
+  /**
+   * Get maximum latency of jobs running time. It will be cleared every hour
+   * @return
+   */
+  public long getMaximumJobLatencyGauge();
+
+  /**
+   * Get job latency counter.
+   * @return
+   */
+  public long getJobLatencyCount();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
index 00f75d4..dc3bc5a 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitor.java
@@ -23,6 +23,7 @@ import org.apache.helix.task.TaskState;
 
 public class WorkflowMonitor implements WorkflowMonitorMBean {
   private static final String WORKFLOW_KEY = "Workflow";
+  private static final long DEFAULT_RESET_INTERVAL_MS = 60 * 60 * 1000; // 1 hour
 
   private String _clusterName;
   private String _workflowType;
@@ -33,6 +34,9 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
   private long _existingWorkflowGauge;
   private long _queuedWorkflowGauge;
   private long _runningWorkflowGauge;
+  private long _totalWorkflowLatencyCount;
+  private long _maximumWorkflowLatencyGauge;
+  private long _lastResetTime;
 
 
   public WorkflowMonitor(String clusterName, String workflowType) {
@@ -44,6 +48,9 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
     _existingWorkflowGauge = 0L;
     _queuedWorkflowGauge = 0L;
     _runningWorkflowGauge = 0L;
+    _totalWorkflowLatencyCount = 0L;
+    _maximumWorkflowLatencyGauge = 0L;
+    _lastResetTime = System.currentTimeMillis();
   }
 
   @Override
@@ -76,6 +83,16 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
     return _runningWorkflowGauge;
   }
 
+  @Override
+  public long getWorkflowLatencyCount() {
+    return _totalWorkflowLatencyCount;
+  }
+
+  @Override
+  public long getMaximumWorkflowLatencyGauge() {
+    return _maximumWorkflowLatencyGauge;
+  }
+
   @Override public String getSensorName() {
     return String.format("%s.%s.%s", _clusterName, WORKFLOW_KEY, _workflowType);
   }
@@ -88,11 +105,20 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
    * Update workflow with transition state
    * @param to The to state of a workflow
    */
+
   public void updateWorkflowCounters(TaskState to) {
-   if (to.equals(TaskState.FAILED)) {
+    updateWorkflowCounters(to, 0);
+  }
+
+  public void updateWorkflowCounters(TaskState to, long latency) {
+    if (to.equals(TaskState.FAILED)) {
       _failedWorkflowCount++;
     } else if (to.equals(TaskState.COMPLETED)) {
       _successfulWorkflowCount++;
+
+      // Only record latency larger than 0 and succeeded workflows
+      _maximumWorkflowLatencyGauge = Math.max(_maximumWorkflowLatencyGauge, latency);
+      _totalWorkflowLatencyCount += latency > 0 ? latency : 0;
     }
   }
 
@@ -104,6 +130,10 @@ public class WorkflowMonitor implements WorkflowMonitorMBean {
     _existingWorkflowGauge = 0L;
     _runningWorkflowGauge = 0L;
     _queuedWorkflowGauge = 0L;
+    if (_lastResetTime + DEFAULT_RESET_INTERVAL_MS < System.currentTimeMillis()) {
+      _lastResetTime = System.currentTimeMillis();
+      _maximumWorkflowLatencyGauge = 0;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
index dcd633d..2558e5b 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/WorkflowMonitorMBean.java
@@ -60,4 +60,16 @@ public interface WorkflowMonitorMBean extends SensorNameProvider {
    * @return
    */
   public long getRunningWorkflowGauge();
+
+  /**
+   * Get workflow latency count
+   * @return
+   */
+  public long getWorkflowLatencyCount();
+
+  /**
+   * Get maximum workflow latency gauge. It will be reset in 1 hour.
+   * @return
+   */
+  public long getMaximumWorkflowLatencyGauge();
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
index 5a17c6b..51da264 100644
--- a/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/JobRebalancer.java
@@ -432,7 +432,8 @@ public class JobRebalancer extends TaskRebalancer {
     if (isJobComplete(jobCtx, allPartitions, jobCfg)) {
       markJobComplete(jobResource, jobCtx, workflowConfig, workflowCtx,
           cache.getJobConfigMap());
-      _clusterStatusMonitor.updateJobCounters(jobCfg, TaskState.COMPLETED);
+      _clusterStatusMonitor.updateJobCounters(jobCfg, TaskState.COMPLETED,
+          jobCtx.getFinishTime() - jobCtx.getStartTime());
       _rebalanceScheduler.removeScheduledRebalance(jobResource);
       TaskUtil.cleanupJobIdealStateExtView(_manager.getHelixDataAccessor(), jobResource);
       return buildEmptyAssignment(jobResource, currStateOutput);
@@ -620,8 +621,7 @@ public class JobRebalancer extends TaskRebalancer {
         jobContext.setPartitionState(pId, TaskPartitionState.TASK_ABORTED);
       }
     }
-    _clusterStatusMonitor
-        .updateJobCounters(jobConfigMap.get(jobName), TaskState.FAILED);
+    _clusterStatusMonitor.updateJobCounters(jobConfigMap.get(jobName), TaskState.FAILED);
     _rebalanceScheduler.removeScheduledRebalance(jobName);
     TaskUtil.cleanupJobIdealStateExtView(_manager.getHelixDataAccessor(), jobName);
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/52d3bb83/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
index 9890a0a..3d3f86e 100644
--- a/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/task/TaskRebalancer.java
@@ -82,6 +82,7 @@ public abstract class TaskRebalancer implements Rebalancer, MappingCalculator {
           for (String jobToFail : cfg.getJobDag().getAllNodes()) {
             if (ctx.getJobState(jobToFail) == TaskState.IN_PROGRESS) {
               ctx.setJobState(jobToFail, TaskState.ABORTED);
+              // Skip aborted jobs latency since they are not accurate latency for job running time
               _clusterStatusMonitor
                   .updateJobCounters(jobConfigMap.get(jobToFail), TaskState.ABORTED);
             }
@@ -89,14 +90,16 @@ public abstract class TaskRebalancer implements Rebalancer, MappingCalculator {
           return true;
         }
       }
-      if (jobState != TaskState.COMPLETED && jobState != TaskState.FAILED && jobState != TaskState.TIMED_OUT) {
+      if (jobState != TaskState.COMPLETED && jobState != TaskState.FAILED
+          && jobState != TaskState.TIMED_OUT) {
         incomplete = true;
       }
     }
 
     if (!incomplete && cfg.isTerminable()) {
       ctx.setWorkflowState(TaskState.COMPLETED);
-      _clusterStatusMonitor.updateWorkflowCounters(cfg, TaskState.COMPLETED);
+      _clusterStatusMonitor.updateWorkflowCounters(cfg, TaskState.COMPLETED,
+          ctx.getFinishTime() - ctx.getStartTime());
       return true;
     }
 


[23/50] [abbrv] helix git commit: Hot fix to reduce the chance of flip-flop issue in the legacy/default AutoRebalanceStrategy.

Posted by jx...@apache.org.
Hot fix to reduce the chance of flip-flop issue in the legacy/default AutoRebalanceStrategy.


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

Branch: refs/heads/master
Commit: ae13411c8dd6fcf532d07c0d525eb717741f7f60
Parents: b1540f0
Author: Lei Xia <lx...@linkedin.com>
Authored: Mon Dec 4 13:40:41 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:59 2018 -0800

----------------------------------------------------------------------
 .../rebalancer/DelayedAutoRebalancer.java       | 22 ++++++++++--
 .../controller/stages/ClusterDataCache.java     | 38 ++++++++++++++++++++
 2 files changed, 58 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/ae13411c/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index 5ebb57d..7ad2eb2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -32,6 +32,7 @@ import java.util.Set;
 
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
@@ -57,7 +58,22 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
   public IdealState computeNewIdealState(String resourceName,
       IdealState currentIdealState, CurrentStateOutput currentStateOutput,
       ClusterDataCache clusterData) {
-    List<String> allPartitions = new ArrayList<String>(currentIdealState.getPartitionSet());
+
+    // Looking for cached ideal mapping for this resource, if it is already there, do not recompute it again.
+    // The cached mapping will be cleared in ClusterDataCache if there is anything changed in cluster state that can
+    // cause the potential changes in ideal state.
+    // this will avoid flip-flop issue we saw in AutoRebalanceStrategy.
+    ZNRecord znRecord = clusterData.getCachedIdealMapping(resourceName);
+    if (znRecord != null) {
+      // TODO: only apply to legacy Auto-RebalanceStrategy at this time, need to apply to any strategy in future.
+      if (currentIdealState.getRebalanceStrategy().equals(AutoRebalanceStrategy.class.getName())) {
+        LOG.info("Use cached idealstate for " + resourceName);
+        IdealState idealState = new IdealState(znRecord);
+        return idealState;
+      }
+    }
+
+    List<String> allPartitions = new ArrayList<>(currentIdealState.getPartitionSet());
     if (allPartitions.size() == 0) {
       LOG.info("Partition count is 0 for resource " + resourceName
           + ", stop calculate ideal mapping for the resource.");
@@ -181,7 +197,9 @@ public class DelayedAutoRebalancer extends AbstractRebalancer {
       LOG.debug("finalMapping: " + finalMapping);
     }
 
-    return generateNewIdealState(resourceName, currentIdealState, finalMapping);
+    IdealState idealState = generateNewIdealState(resourceName, currentIdealState, finalMapping);
+    clusterData.setCachedIdealMapping(resourceName, idealState.getRecord());
+    return idealState;
   }
 
   private IdealState generateNewIdealState(String resourceName, IdealState currentIdealState,

http://git-wip-us.apache.org/repos/asf/helix/blob/ae13411c/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index 5b4aa83..e663fd4 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -101,8 +101,13 @@ public class ClusterDataCache {
   private Map<PropertyKey, CurrentState> _currentStateCache = Maps.newHashMap();
 
   // maintain a cache of bestPossible assignment across pipeline runs
+  // TODO: this is only for customRebalancer, remove it and merge it with _idealMappingCache.
   private Map<String, ResourceAssignment>  _resourceAssignmentCache = Maps.newHashMap();
 
+
+  // maintain a cache of idealmapping (preference list) for full-auto resource across pipeline runs
+  private Map<String, ZNRecord>  _idealMappingCache = Maps.newHashMap();
+
   private Map<ChangeType, Boolean> _propertyDataChangedMap;
 
   private Map<String, Integer> _participantActiveTaskCount = new HashMap<>();
@@ -985,8 +990,41 @@ public class ClusterDataCache {
   }
 
 
+  /**
+   * Get cached resourceAssignment (ideal mapping) for a resource
+   *
+   * @param resource
+   *
+   * @return
+   */
+  public ZNRecord getCachedIdealMapping(String resource) {
+    return _idealMappingCache.get(resource);
+  }
+
+  /**
+   * Get cached idealmapping
+   *
+   * @return
+   */
+  public Map<String, ZNRecord> getCachedIdealMapping() {
+    return Collections.unmodifiableMap(_idealMappingCache);
+  }
+
+  /**
+   * Cache resourceAssignment (ideal mapping) for a resource
+   *
+   * @param resource
+   *
+   * @return
+   */
+  public void setCachedIdealMapping(String resource, ZNRecord mapping) {
+    _idealMappingCache.put(resource, mapping);
+  }
+
+
   public void clearCachedResourceAssignments() {
     _resourceAssignmentCache.clear();
+    _idealMappingCache.clear();
   }
 
   /**


[05/50] [abbrv] helix git commit: Add P2P (Participant-to-Participant) state-transition message support in Helix controller.

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/messaging/p2pMessage/TestP2PStateTransitionMessages.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/p2pMessage/TestP2PStateTransitionMessages.java b/helix-core/src/test/java/org/apache/helix/messaging/p2pMessage/TestP2PStateTransitionMessages.java
new file mode 100644
index 0000000..1a21ef9
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/messaging/p2pMessage/TestP2PStateTransitionMessages.java
@@ -0,0 +1,176 @@
+package org.apache.helix.messaging.p2pMessage;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.List;
+import java.util.Map;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.pipeline.Pipeline;
+import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.BaseStageTest;
+import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
+import org.apache.helix.controller.stages.BestPossibleStateOutput;
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.controller.stages.IntermediateStateCalcStage;
+import org.apache.helix.controller.stages.MessageGenerationPhase;
+import org.apache.helix.controller.stages.MessageSelectionStage;
+import org.apache.helix.controller.stages.MessageSelectionStageOutput;
+import org.apache.helix.controller.stages.MessageThrottleStage;
+import org.apache.helix.controller.stages.ReadClusterDataStage;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MasterSlaveSMD;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+public class TestP2PStateTransitionMessages extends BaseStageTest {
+  String db = "testDB";
+  int numPartition = 1;
+  int numReplica = 3;
+
+
+  private void preSetup() {
+    setupIdealState(3, new String[]{db}, numPartition, numReplica, IdealState.RebalanceMode.SEMI_AUTO,
+        BuiltInStateModelDefinitions.MasterSlave.name());
+    setupStateModel();
+    setupInstances(3);
+    setupLiveInstances(3);
+  }
+
+  @Test
+  public void testP2PMessageEnabled() throws Exception {
+    preSetup();
+    ClusterConfig clusterConfig = new ClusterConfig(_clusterName);
+    clusterConfig.enableP2PMessage(true);
+    setClusterConfig(clusterConfig);
+
+    testP2PMessage(clusterConfig, true);
+  }
+
+  @Test
+  public void testP2PMessageDisabled() throws Exception {
+    preSetup();
+    testP2PMessage(null, false);
+  }
+
+  private void testP2PMessage(ClusterConfig clusterConfig, Boolean p2pMessageEnabled) throws Exception {
+    Map<String, Resource> resourceMap =
+        getResourceMap(new String[]{db}, numPartition, BuiltInStateModelDefinitions.MasterSlave.name(), clusterConfig,
+            null);
+
+    event.addAttribute(AttributeName.RESOURCES.name(), resourceMap);
+    event.addAttribute(AttributeName.RESOURCES_TO_REBALANCE.name(), resourceMap);
+    event.addAttribute(AttributeName.CURRENT_STATE.name(), new CurrentStateOutput());
+    event.addAttribute(AttributeName.helixmanager.name(), manager);
+
+    Pipeline pipeline = createPipeline();
+    pipeline.handle(event);
+
+    BestPossibleStateOutput bestPossibleStateOutput = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+
+    CurrentStateOutput currentStateOutput = populateCurrentStateFromBestPossible(bestPossibleStateOutput);
+    event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput);
+
+    Partition p = new Partition(db + "_0");
+
+    String masterInstance =
+        getTopStateInstance(bestPossibleStateOutput.getInstanceStateMap(db, p), MasterSlaveSMD.States.MASTER.name());
+    Assert.assertNotNull(masterInstance);
+
+    admin.enableInstance(_clusterName, masterInstance, false);
+    ClusterDataCache cache = event.getAttribute(AttributeName.ClusterDataCache.name());
+    cache.notifyDataChange(HelixConstants.ChangeType.INSTANCE_CONFIG);
+
+    pipeline.handle(event);
+
+    bestPossibleStateOutput = event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+
+    MessageSelectionStageOutput messageOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    List<Message> messages = messageOutput.getMessages(db, p);
+
+    Assert.assertEquals(messages.size(), 1);
+    Message message = messages.get(0);
+    Assert.assertEquals(message.getTgtName(), masterInstance);
+    Assert.assertEquals(message.getFromState(), MasterSlaveSMD.States.MASTER.name());
+    Assert.assertEquals(message.getToState(), MasterSlaveSMD.States.SLAVE.name());
+
+    if (p2pMessageEnabled) {
+      Assert.assertEquals(message.getRelayMessages().entrySet().size(), 1);
+      String newMasterInstance =
+          getTopStateInstance(bestPossibleStateOutput.getInstanceStateMap(db, p), MasterSlaveSMD.States.MASTER.name());
+
+      Message relayMessage = message.getRelayMessage(newMasterInstance);
+      Assert.assertNotNull(relayMessage);
+      Assert.assertEquals(relayMessage.getMsgSubType(), Message.MessageType.RELAYED_MESSAGE.name());
+      Assert.assertEquals(relayMessage.getTgtName(), newMasterInstance);
+      Assert.assertEquals(relayMessage.getRelaySrcHost(), masterInstance);
+      Assert.assertEquals(relayMessage.getFromState(), MasterSlaveSMD.States.SLAVE.name());
+      Assert.assertEquals(relayMessage.getToState(), MasterSlaveSMD.States.MASTER.name());
+    } else {
+      Assert.assertTrue(message.getRelayMessages().entrySet().isEmpty());
+    }
+  }
+
+  private String getTopStateInstance(Map<String, String> instanceStateMap, String topState) {
+    String masterInstance = null;
+    for (Map.Entry<String, String> e : instanceStateMap.entrySet()) {
+      if (topState.equals(e.getValue())) {
+        masterInstance = e.getKey();
+      }
+    }
+
+    return masterInstance;
+  }
+
+  private CurrentStateOutput populateCurrentStateFromBestPossible(BestPossibleStateOutput bestPossibleStateOutput) {
+    CurrentStateOutput currentStateOutput = new CurrentStateOutput();
+    for (String resource : bestPossibleStateOutput.getResourceStatesMap().keySet()) {
+      PartitionStateMap partitionStateMap = bestPossibleStateOutput.getPartitionStateMap(resource);
+      for (Partition p : partitionStateMap.partitionSet()) {
+        Map<String, String> stateMap = partitionStateMap.getPartitionMap(p);
+
+        for (Map.Entry<String, String> e : stateMap.entrySet()) {
+          currentStateOutput.setCurrentState(resource, p, e.getKey(), e.getValue());
+        }
+      }
+    }
+    return currentStateOutput;
+  }
+
+  private Pipeline createPipeline() {
+    Pipeline pipeline = new Pipeline("test");
+    pipeline.addStage(new ReadClusterDataStage());
+    pipeline.addStage(new BestPossibleStateCalcStage());
+    pipeline.addStage(new IntermediateStateCalcStage());
+    pipeline.addStage(new MessageGenerationPhase());
+    pipeline.addStage(new MessageSelectionStage());
+    pipeline.addStage(new MessageThrottleStage());
+
+    return pipeline;
+  }
+}

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java b/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
index f84565b..77da401 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
@@ -150,7 +150,7 @@ public class MockBaseDataAccessor implements BaseDataAccessor<ZNRecord> {
 
   @Override
   public List<ZNRecord> get(List<String> paths, List<Stat> stats, int options) {
-    List<ZNRecord> records = new ArrayList<ZNRecord>();
+    List<ZNRecord> records = new ArrayList<>();
     for (int i = 0; i < paths.size(); i++) {
       ZNRecord record = get(paths.get(i), stats.get(i), options);
       records.add(record);
@@ -160,7 +160,7 @@ public class MockBaseDataAccessor implements BaseDataAccessor<ZNRecord> {
 
   @Override
   public List<ZNRecord> getChildren(String parentPath, List<Stat> stats, int options) {
-    List<ZNRecord> children = new ArrayList<ZNRecord>();
+    List<ZNRecord> children = new ArrayList<>();
     for (String key : _recordMap.keySet()) {
       if (key.startsWith(parentPath)) {
         String[] keySplit = key.split("\\/");
@@ -182,7 +182,7 @@ public class MockBaseDataAccessor implements BaseDataAccessor<ZNRecord> {
 
   @Override
   public List<String> getChildNames(String parentPath, int options) {
-    List<String> child = new ArrayList<String>();
+    List<String> child = new ArrayList<>();
     for (String key : _recordMap.keySet()) {
       if (key.startsWith(parentPath)) {
         String[] keySplit = key.split("\\/");

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
index 8679007..037d92b 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
@@ -215,8 +216,19 @@ public class MockHelixAdmin implements HelixAdmin {
 
   }
 
-  @Override public void enableInstance(String clusterName, String instanceName, boolean enabled) {
+  @Override
+  public void enableInstance(String clusterName, String instanceName, boolean enabled) {
+    String instanceConfigsPath = PropertyPathBuilder.instanceConfig(clusterName);
+    if (!_baseDataAccessor.exists(instanceConfigsPath, 0)) {
+      _baseDataAccessor.create(instanceConfigsPath, new ZNRecord(instanceName), 0);
+    }
+
+    String instanceConfigPath = instanceConfigsPath + "/" + instanceName;
 
+    ZNRecord  record = (ZNRecord) _baseDataAccessor.get(instanceConfigPath, null, 0);
+    InstanceConfig instanceConfig = new InstanceConfig(record);
+    instanceConfig.setInstanceEnabled(enabled);
+    _baseDataAccessor.set(instanceConfigPath, instanceConfig.getRecord(), 0);
   }
 
   @Override public void enableInstance(String clusterName, List<String> instances,

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
index 8c05626..143f3c0 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestDisableResourceMbean.java
@@ -28,6 +28,8 @@ import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -42,7 +44,8 @@ import java.util.Map;
 public class TestDisableResourceMbean extends ZkUnitTestBase {
   private MBeanServerConnection _mbeanServer = ManagementFactory.getPlatformMBeanServer();
 
-  @Test public void testDisableResourceMonitoring() throws Exception {
+  @Test
+  public void testDisableResourceMonitoring() throws Exception {
     final int NUM_PARTICIPANTS = 2;
     String clusterName = TestHelper.getTestClassName() + "_" + TestHelper.getTestMethodName();
     System.out.println("START " + clusterName + " at " + new Date(System.currentTimeMillis()));
@@ -83,7 +86,9 @@ public class TestDisableResourceMbean extends ZkUnitTestBase {
         new ClusterControllerManager(ZK_ADDR, clusterName, "controller_0");
     controller.syncStart();
 
-    Thread.sleep(300);
+    HelixClusterVerifier clusterVerifier =
+        new BestPossibleExternalViewVerifier.Builder(clusterName).setZkClient(_gZkClient).build();
+    Assert.assertTrue(clusterVerifier.verify());
 
     // Verify the bean was created for TestDB0, but not for TestDB1.
     Assert.assertTrue(_mbeanServer.isRegistered(getMbeanName("TestDB0", clusterName)));

http://git-wip-us.apache.org/repos/asf/helix/blob/d0a3c0d1/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestTopStateHandoffMetrics.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestTopStateHandoffMetrics.java b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestTopStateHandoffMetrics.java
index 6f1b083..51b048d 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestTopStateHandoffMetrics.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestTopStateHandoffMetrics.java
@@ -47,15 +47,13 @@ public class TestTopStateHandoffMetrics extends BaseStageTest {
   public final static String TEST_RESOURCE = "TestResource";
   public final static String PARTITION = "PARTITION";
 
-
   public void preSetup() {
     setupLiveInstances(3);
     setupStateModel();
     Resource resource = new Resource(TEST_RESOURCE);
     resource.setStateModelDefRef("MasterSlave");
     resource.addPartition(PARTITION);
-    event.addAttribute(AttributeName.RESOURCES.name(),
-        Collections.singletonMap(TEST_RESOURCE, resource));
+    event.addAttribute(AttributeName.RESOURCES.name(), Collections.singletonMap(TEST_RESOURCE, resource));
     event.addAttribute(AttributeName.clusterStatusMonitor.name(), new ClusterStatusMonitor("TestCluster"));
   }
 


[14/50] [abbrv] helix git commit: Support new API for getChildren with retry logic.

Posted by jx...@apache.org.
Support new API for getChildren with retry logic.

Current getChildren will remove the znode from list if it has not been read. It could return partial result of application expected. It will be a problem for applcations needs completed data.

New API will support retry logic. If it failed to read all the data from the ZK in retry count. Helix will throw an exception.

TODO: Helix will change the old API's behaivor when Helix start migrating APIs.


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

Branch: refs/heads/master
Commit: 79c96ba6b75c240c601e42ca7031ad1f855e1bc9
Parents: d1cbfdb
Author: Junkai Xue <jx...@linkedin.com>
Authored: Thu Nov 9 15:05:04 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:21 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/helix/BaseDataAccessor.java | 21 +++++++
 .../helix/manager/zk/ZkBaseDataAccessor.java    | 63 ++++++++++++++++----
 .../manager/zk/ZkCacheBaseDataAccessor.java     | 11 +++-
 .../apache/helix/mock/MockBaseDataAccessor.java |  7 +++
 4 files changed, 89 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/79c96ba6/helix-core/src/main/java/org/apache/helix/BaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/BaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/BaseDataAccessor.java
index a8f2907..c42c700 100644
--- a/helix-core/src/main/java/org/apache/helix/BaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/BaseDataAccessor.java
@@ -142,6 +142,11 @@ public interface BaseDataAccessor<T> {
 
   /**
    * Get the children under a parent path using async api
+   *
+   * For this API, if some of child node is failed to read, Helix will return the data of read
+   * nodes. So user may get partial data. No exception will be thrown even if it is failed to read
+   * all the data.
+   *
    * @param parentPath path to the immediate parent ZNode
    * @param stats Zookeeper Stat objects corresponding to each child
    * @param options Set the type of ZNode see the valid values in {@link AccessOption}
@@ -150,6 +155,22 @@ public interface BaseDataAccessor<T> {
   List<T> getChildren(String parentPath, List<Stat> stats, int options);
 
   /**
+   * Get the children under a parent path using async api
+   *
+   * If some of child node is failed to read, Helix will do the retry within retry count. If the
+   * result still cannot be retrieved completely, Helix will throw an HelixException.
+   *
+   * @param parentPath path to the immediate parent ZNode
+   * @param stats Zookeeper Stat objects corresponding to each child
+   * @param options Set the type of ZNode see the valid values in {@link AccessOption}
+   * @param retryCount The number of retries that data is not completed read
+   * @param retryInterval The interval between two retries
+   * @return A list of children of the parent ZNode
+   */
+  List<T> getChildren(String parentPath, List<Stat> stats, int options, int retryCount,
+      int retryInterval) throws HelixException;
+
+  /**
    * Returns the child names given a parent path
    * @param parentPath path to the immediate parent ZNode
    * @param options Set the type of ZNode see the valid values in {@link AccessOption}

http://git-wip-us.apache.org/repos/asf/helix/blob/79c96ba6/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
index 842f0c0..0ccda43 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
@@ -24,7 +24,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-
 import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
@@ -34,6 +33,7 @@ import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.DeleteCallbackHandler;
@@ -42,12 +42,12 @@ import org.apache.helix.manager.zk.ZkAsyncCallbacks.GetDataCallbackHandler;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.SetDataCallbackHandler;
 import org.apache.helix.store.zk.ZNode;
 import org.apache.helix.util.HelixUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.DataTree;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
   enum RetCode {
@@ -331,13 +331,13 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
     boolean[] needRead = new boolean[paths.size()];
     Arrays.fill(needRead, true);
 
-    return get(paths, stats, needRead);
+    return get(paths, stats, needRead, false);
   }
 
   /**
    * async get
    */
-  List<T> get(List<String> paths, List<Stat> stats, boolean[] needRead) {
+  List<T> get(List<String> paths, List<Stat> stats, boolean[] needRead, boolean throwException) {
     if (paths == null || paths.size() == 0) {
       return Collections.emptyList();
     }
@@ -373,7 +373,7 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
 
       // construct return results
       List<T> records = new ArrayList<T>(Collections.<T> nCopies(paths.size(), null));
-
+      StringBuilder nodeFailToRead = new StringBuilder();
       for (int i = 0; i < paths.size(); i++) {
         if (!needRead[i])
           continue;
@@ -386,9 +386,14 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
           if (stats != null) {
             stats.set(i, cb._stat);
           }
+        } else if (Code.get(cb.getRc()) != Code.NONODE && throwException) {
+          throw new HelixException(String.format("Failed to read node %s", paths.get(i)));
+        } else {
+          nodeFailToRead.append(paths + ",");
         }
       }
-
+      LOG.warn(String.format("Fail to read nodes for paths : %s",
+          nodeFailToRead.toString().substring(nodeFailToRead.length() - 1)));
       return records;
     } finally {
       long endT = System.nanoTime();
@@ -401,9 +406,42 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
 
   /**
    * asyn getChildren
+   * The retryCount and retryInterval will be ignored.
    */
+  // TODO: Change the behavior of getChildren when Helix starts migrating API.
   @Override
   public List<T> getChildren(String parentPath, List<Stat> stats, int options) {
+    return getChildren(parentPath, stats, options, false);
+  }
+
+
+  @Override
+  public List<T> getChildren(String parentPath, List<Stat> stats, int options, int retryCount,
+      int retryInterval) throws HelixException {
+    int readCount = retryCount + 1;
+    while (readCount > 0) {
+      try {
+        readCount--;
+        List<T> records = getChildren(parentPath, stats, options, true);
+        return records;
+      } catch (HelixException e) {
+        if (readCount == 0) {
+          throw new HelixException(String.format("Failed to get full list of %s", parentPath), e);
+        }
+        try {
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException interruptedException) {
+          throw new HelixException("Fail to interrupt the sleep", interruptedException);
+        }
+      }
+    }
+
+    // Impossible to reach end
+    return null;
+  }
+
+  private List<T> getChildren(String parentPath, List<Stat> stats, int options,
+      boolean throwException) {
     try {
       // prepare child paths
       List<String> childNames = getChildNames(parentPath, options);
@@ -411,15 +449,17 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
         return Collections.emptyList();
       }
 
-      List<String> paths = new ArrayList<String>();
+      List<String> paths = new ArrayList<>();
       for (String childName : childNames) {
         String path = parentPath + "/" + childName;
         paths.add(path);
       }
 
       // remove null record
-      List<Stat> curStats = new ArrayList<Stat>(paths.size());
-      List<T> records = get(paths, curStats, options);
+      List<Stat> curStats = new ArrayList<>(paths.size());
+      boolean[] needRead = new boolean[paths.size()];
+      Arrays.fill(needRead, true);
+      List<T> records = get(paths, curStats, needRead, throwException);
       Iterator<T> recordIter = records.iterator();
       Iterator<Stat> statIter = curStats.iterator();
       while (statIter.hasNext()) {
@@ -806,7 +846,8 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
 
         // asycn read all data
         List<Stat> curStats = new ArrayList<Stat>();
-        List<T> curDataList = get(paths, curStats, Arrays.copyOf(needUpdate, needUpdate.length));
+        List<T> curDataList =
+            get(paths, curStats, Arrays.copyOf(needUpdate, needUpdate.length), false);
 
         // async update
         List<T> newDataList = new ArrayList<T>();

http://git-wip-us.apache.org/repos/asf/helix/blob/79c96ba6/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
index 748e090..18abe91 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
@@ -36,6 +36,7 @@ import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor.RetCode;
@@ -600,7 +601,7 @@ public class ZkCacheBaseDataAccessor<T> implements HelixPropertyStore<T> {
       if (needRead) {
         cache.lockWrite();
         try {
-          List<T> readRecords = _baseAccessor.get(serverPaths, readStats, needReads);
+          List<T> readRecords = _baseAccessor.get(serverPaths, readStats, needReads, false);
           for (int i = 0; i < size; i++) {
             if (needReads[i]) {
               records.set(i, readRecords.get(i));
@@ -672,7 +673,7 @@ public class ZkCacheBaseDataAccessor<T> implements HelixPropertyStore<T> {
       return null;
     }
 
-    List<String> paths = new ArrayList<String>();
+    List<String> paths = new ArrayList<>();
     for (String childName : childNames) {
       String path = parentPath.equals("/") ? "/" + childName : parentPath + "/" + childName;
       paths.add(path);
@@ -682,6 +683,12 @@ public class ZkCacheBaseDataAccessor<T> implements HelixPropertyStore<T> {
   }
 
   @Override
+  public List<T> getChildren(String parentPath, List<Stat> stats, int options,
+      int retryCount, int retryInterval) throws HelixException {
+    return getChildren(parentPath, stats, options);
+  }
+
+  @Override
   public void subscribeDataChanges(String path, IZkDataListener listener) {
     String serverPath = prependChroot(path);
 

http://git-wip-us.apache.org/repos/asf/helix/blob/79c96ba6/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java b/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
index 77da401..099b3fc 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockBaseDataAccessor.java
@@ -29,6 +29,7 @@ import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
 import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
 import org.apache.zookeeper.data.Stat;
 
@@ -181,6 +182,12 @@ public class MockBaseDataAccessor implements BaseDataAccessor<ZNRecord> {
   }
 
   @Override
+  public List<ZNRecord> getChildren(String parentPath, List<Stat> stats, int options,
+      int retryCount, int retryInterval) throws HelixException {
+    return getChildren(parentPath, stats, options);
+  }
+
+  @Override
   public List<String> getChildNames(String parentPath, int options) {
     List<String> child = new ArrayList<>();
     for (String key : _recordMap.keySet()) {


[16/50] [abbrv] helix git commit: REST support for batch API enable and disable

Posted by jx...@apache.org.
REST support for batch API enable and disable


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

Branch: refs/heads/master
Commit: d03dca1ee360d1ac8dd3d04bbcf50d8e5623c0e7
Parents: 62752b2
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Nov 15 17:31:00 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:30 2018 -0800

----------------------------------------------------------------------
 .../helix/manager/zk/ZkBaseDataAccessor.java    |  6 ++-
 .../rest/server/resources/InstanceAccessor.java | 41 ++++++++++++++++++++
 .../helix/rest/server/TestInstanceAccessor.java | 29 ++++++++++++++
 3 files changed, 74 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d03dca1e/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
index 0ccda43..f8678e5 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
@@ -392,8 +392,10 @@ public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
           nodeFailToRead.append(paths + ",");
         }
       }
-      LOG.warn(String.format("Fail to read nodes for paths : %s",
-          nodeFailToRead.toString().substring(nodeFailToRead.length() - 1)));
+      if (nodeFailToRead.length() > 0) {
+        LOG.warn(String.format("Fail to read nodes for paths : %s",
+            nodeFailToRead.toString().substring(nodeFailToRead.length() - 1)));
+      }
       return records;
     } finally {
       long endT = System.nanoTime();

http://git-wip-us.apache.org/repos/asf/helix/blob/d03dca1e/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
index 5b25c3e..0099097 100644
--- a/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
+++ b/helix-rest/src/main/java/org/apache/helix/rest/server/resources/InstanceAccessor.java
@@ -112,6 +112,47 @@ public class InstanceAccessor extends AbstractResource {
     return JSONRepresentation(root);
   }
 
+  @POST
+  public Response updateInstances(@PathParam("clusterId") String clusterId,
+      @QueryParam("command") String command, String content) {
+    Command cmd;
+    try {
+      cmd = Command.valueOf(command);
+    } catch (Exception e) {
+      return badRequest("Invalid command : " + command);
+    }
+
+    HelixAdmin admin = getHelixAdmin();
+    try {
+      JsonNode node = null;
+      if (content.length() != 0) {
+        node = OBJECT_MAPPER.readTree(content);
+      }
+      if (node == null) {
+        return badRequest("Invalid input for content : " + content);
+      }
+      List<String> enableInstances = OBJECT_MAPPER
+          .readValue(node.get(InstanceProperties.instances.name()).toString(),
+              OBJECT_MAPPER.getTypeFactory().constructCollectionType(List.class, String.class));
+      switch (cmd) {
+      case enable:
+        admin.enableInstance(clusterId, enableInstances, true);
+
+        break;
+      case disable:
+        admin.enableInstance(clusterId, enableInstances, false);
+        break;
+      default:
+        _logger.error("Unsupported command :" + command);
+        return badRequest("Unsupported command :" + command);
+      }
+    } catch (Exception e) {
+      _logger.error("Failed in updating instances : " + content, e);
+      return badRequest(e.getMessage());
+    }
+    return OK();
+  }
+
   @GET
   @Path("{instanceName}")
   public Response getInstance(@PathParam("clusterId") String clusterId,

http://git-wip-us.apache.org/repos/asf/helix/blob/d03dca1e/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
index 5b3f605..7e1b4cb 100644
--- a/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
+++ b/helix-rest/src/test/java/org/apache/helix/rest/server/TestInstanceAccessor.java
@@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import javax.ws.rs.client.Entity;
@@ -30,6 +32,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.apache.helix.HelixException;
 import org.apache.helix.TestHelper;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.rest.server.resources.AbstractResource;
 import org.apache.helix.rest.server.resources.InstanceAccessor;
@@ -133,5 +136,31 @@ public class TestInstanceAccessor extends AbstractTestClass {
         Response.Status.OK.getStatusCode());
     Assert.assertEquals(_configAccessor.getInstanceConfig(CLUSTER_NAME, INSTANCE_NAME).getTags(),
         ImmutableList.of("tag2"));
+
+    // Batch disable instances
+    List<String> instancesToDisable = Arrays.asList(
+        new String[] { CLUSTER_NAME + "localhost_12918", CLUSTER_NAME + "localhost_12919",
+            CLUSTER_NAME + "localhost_12920"
+        });
+    entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(
+        ImmutableMap.of(InstanceAccessor.InstanceProperties.instances.name(), instancesToDisable)),
+        MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/instances", ImmutableMap.of("command", "disable"), entity,
+        Response.Status.OK.getStatusCode());
+    ClusterConfig clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
+    Assert.assertEquals(clusterConfig.getDisabledInstances().keySet(),
+        new HashSet<>(instancesToDisable));
+
+    instancesToDisable = Arrays
+        .asList(new String[] { CLUSTER_NAME + "localhost_12918", CLUSTER_NAME + "localhost_12920"
+        });
+    entity = Entity.entity(OBJECT_MAPPER.writeValueAsString(
+        ImmutableMap.of(InstanceAccessor.InstanceProperties.instances.name(), instancesToDisable)),
+        MediaType.APPLICATION_JSON_TYPE);
+    post("clusters/" + CLUSTER_NAME + "/instances", ImmutableMap.of("command", "enable"), entity,
+        Response.Status.OK.getStatusCode());
+    clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
+    Assert.assertEquals(clusterConfig.getDisabledInstances().keySet(),
+        new HashSet<>(Arrays.asList(CLUSTER_NAME + "localhost_12919")));
   }
 }


[13/50] [abbrv] helix git commit: Switch pause cluster to maintenance mode when hit maximal offline instance limit

Posted by jx...@apache.org.
Switch pause cluster to maintenance mode when hit maximal offline instance limit


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

Branch: refs/heads/master
Commit: d1cbfdba3e932d8ecebd9fdba86b6174de690973
Parents: a7477c3
Author: Junkai Xue <jx...@linkedin.com>
Authored: Wed Nov 8 11:24:54 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:17 2018 -0800

----------------------------------------------------------------------
 .../helix/controller/stages/BestPossibleStateCalcStage.java       | 3 ++-
 .../helix/controller/stages/IntermediateStateCalcStage.java       | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/d1cbfdba/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 4fb8cd7..e96f0f3 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -180,7 +180,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
             "Offline Instances count %d greater than allowed count %d. Stop rebalance pipeline and pause the cluster %s",
             offlineCount, maxOfflineInstancesAllowed, cache.getClusterName());
         if (manager != null) {
-          manager.getClusterManagmentTool().enableCluster(manager.getClusterName(), false, errMsg);
+          manager.getClusterManagmentTool()
+              .enableMaintenanceMode(manager.getClusterName(), true, errMsg);
         } else {
           logger.error("Failed to pause cluster, HelixManager is not set!");
         }

http://git-wip-us.apache.org/repos/asf/helix/blob/d1cbfdba/helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
index 27eab81..2bd0229 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
@@ -179,7 +179,7 @@ public class IntermediateStateCalcStage extends AbstractBaseStage {
                 instance, maxPartitionPerInstance, cache.getClusterName());
             if (manager != null) {
               manager.getClusterManagmentTool()
-                  .enableCluster(manager.getClusterName(), false, errMsg);
+                  .enableMaintenanceMode(manager.getClusterName(), true, errMsg);
             } else {
               logger.error("Failed to pause cluster, HelixManager is not set!");
             }


[09/50] [abbrv] helix git commit: use SlidingTimeWindownReservoir for histogram stats

Posted by jx...@apache.org.
use SlidingTimeWindownReservoir for histogram stats


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

Branch: refs/heads/master
Commit: 3a73b0f30336c4610a501af979d55c4d25344214
Parents: 2f791a6
Author: hrzhang <hr...@linkedin.com>
Authored: Tue Nov 7 11:38:36 2017 -0800
Committer: Junkai Xue <jx...@linkedin.com>
Committed: Wed Jan 24 18:31:00 2018 -0800

----------------------------------------------------------------------
 .../monitoring/mbeans/ClusterEventMonitor.java  | 35 ++++++--
 .../monitoring/mbeans/ClusterStatusMonitor.java |  2 +-
 .../monitoring/mbeans/HelixCallbackMonitor.java | 20 +++--
 .../mbeans/MessageLatencyMonitor.java           | 18 ++--
 .../monitoring/mbeans/ResourceMonitor.java      | 86 +++++++++++---------
 .../monitoring/mbeans/ZkClientPathMonitor.java  | 58 +++++++------
 .../dynamicMBeans/DynamicMBeanProvider.java     | 17 +---
 .../TestClusterEventStatusMonitor.java          | 68 +++++++++++++++-
 8 files changed, 201 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
index e7f09ea..8c77466 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterEventMonitor.java
@@ -19,6 +19,9 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import java.util.concurrent.TimeUnit;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
@@ -35,20 +38,16 @@ public class ClusterEventMonitor extends DynamicMBeanProvider {
     TotalProcessed
   }
 
-  private static final long RESET_INTERVAL = 1000 * 60 * 10; // 1 hour
   private static final String CLUSTEREVENT_DN_KEY = "ClusterEventStatus";
   private static final String EVENT_DN_KEY = "eventName";
   private static final String PHASE_DN_KEY = "phaseName";
 
   private final String _phaseName;
 
-  private SimpleDynamicMetric<Long> _totalDuration =
-      new SimpleDynamicMetric("TotalDurationCounter", 0l);
-  private SimpleDynamicMetric<Long> _maxDuration =
-      new SimpleDynamicMetric("MaxSingleDurationGauge", 0l);
-  private SimpleDynamicMetric<Long> _count = new SimpleDynamicMetric("EventCounter", 0l);
-  private HistogramDynamicMetric _duration = new HistogramDynamicMetric("DurationGauge",
-      _metricRegistry.histogram(getMetricName("DurationGauge")));
+  private SimpleDynamicMetric<Long> _totalDuration;
+  private SimpleDynamicMetric<Long> _maxDuration;
+  private SimpleDynamicMetric<Long> _count;
+  private HistogramDynamicMetric _duration;
 
   private long _lastResetTime;
   private ClusterStatusMonitor _clusterStatusMonitor;
@@ -56,13 +55,31 @@ public class ClusterEventMonitor extends DynamicMBeanProvider {
   public ClusterEventMonitor(ClusterStatusMonitor clusterStatusMonitor, String phaseName) {
     _phaseName = phaseName;
     _clusterStatusMonitor = clusterStatusMonitor;
+
+    _duration = new HistogramDynamicMetric("DurationGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _count = new SimpleDynamicMetric("EventCounter", 0l);
+    _maxDuration = new SimpleDynamicMetric("MaxSingleDurationGauge", 0l);
+    _totalDuration = new SimpleDynamicMetric("TotalDurationCounter", 0l);
+  }
+
+  public ClusterEventMonitor(ClusterStatusMonitor clusterStatusMonitor, String phaseName,
+      int histogramTimeWindowMs) {
+    _phaseName = phaseName;
+    _clusterStatusMonitor = clusterStatusMonitor;
+
+    _duration = new HistogramDynamicMetric("DurationGauge", new Histogram(
+        new SlidingTimeWindowReservoir(histogramTimeWindowMs, TimeUnit.MILLISECONDS)));
+    _count = new SimpleDynamicMetric("EventCounter", 0l);
+    _maxDuration = new SimpleDynamicMetric("MaxSingleDurationGauge", 0l);
+    _totalDuration = new SimpleDynamicMetric("TotalDurationCounter", 0l);
   }
 
   public void reportDuration(long duration) {
     _totalDuration.updateValue(_totalDuration.getValue() + duration);
     _count.updateValue(_count.getValue() + 1);
     _duration.updateValue(duration);
-    if (_lastResetTime + RESET_INTERVAL <= System.currentTimeMillis() ||
+    if (_lastResetTime + DEFAULT_RESET_INTERVAL_MS <= System.currentTimeMillis() ||
         duration > _maxDuration.getValue()) {
       _maxDuration.updateValue(duration);
       _lastResetTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
index c644d10..61f4ce1 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ClusterStatusMonitor.java
@@ -69,7 +69,7 @@ public class ClusterStatusMonitor implements ClusterStatusMonitorMBean {
   private final ConcurrentHashMap<String, InstanceMonitor> _instanceMbeanMap = new ConcurrentHashMap<>();
 
   // phaseName -> eventMonitor
-  private final ConcurrentHashMap<String, ClusterEventMonitor> _clusterEventMbeanMap =
+  protected final ConcurrentHashMap<String, ClusterEventMonitor> _clusterEventMbeanMap =
       new ConcurrentHashMap<>();
 
   /**

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
index 0dbafb4..0fc2001 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/HelixCallbackMonitor.java
@@ -19,6 +19,8 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.InstanceType;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
@@ -28,6 +30,7 @@ import org.apache.helix.monitoring.mbeans.dynamicMBeans.SimpleDynamicMetric;
 
 import javax.management.JMException;
 import java.util.ArrayList;
+import java.util.concurrent.TimeUnit;
 import java.util.List;
 
 public class HelixCallbackMonitor extends DynamicMBeanProvider {
@@ -42,14 +45,11 @@ public class HelixCallbackMonitor extends DynamicMBeanProvider {
   private final String _clusterName;
   private final String _instanceName;
 
-  private SimpleDynamicMetric<Long> _counter = new SimpleDynamicMetric("Counter", 0l);
-  private SimpleDynamicMetric<Long> _unbatchedCounter =
-      new SimpleDynamicMetric("UnbatchedCounter", 0l);
-  private SimpleDynamicMetric<Long> _totalLatencyCounter =
-      new SimpleDynamicMetric("LatencyCounter", 0l);
+  private SimpleDynamicMetric<Long> _counter;
+  private SimpleDynamicMetric<Long> _unbatchedCounter;
+  private SimpleDynamicMetric<Long> _totalLatencyCounter;
 
-  private HistogramDynamicMetric _latencyGauge = new HistogramDynamicMetric("LatencyGauge",
-      _metricRegistry.histogram(getMetricName("LatencyGauge")));
+  private HistogramDynamicMetric _latencyGauge;
 
   public HelixCallbackMonitor(InstanceType type, String clusterName, String instanceName,
       HelixConstants.ChangeType changeType) throws JMException {
@@ -62,6 +62,12 @@ public class HelixCallbackMonitor extends DynamicMBeanProvider {
     _sensorName = String
         .format("%s.%s.%s.%s", MonitorDomainNames.HelixCallback.name(), type.name(), clusterName,
             changeType.name());
+
+    _latencyGauge = new HistogramDynamicMetric("LatencyGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _totalLatencyCounter = new SimpleDynamicMetric("LatencyCounter", 0l);
+    _unbatchedCounter = new SimpleDynamicMetric("UnbatchedCounter", 0l);
+    _counter = new SimpleDynamicMetric("Counter", 0l);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
index f1c1039..dac5826 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MessageLatencyMonitor.java
@@ -19,6 +19,9 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import java.util.concurrent.TimeUnit;
 import org.apache.helix.model.Message;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
@@ -35,19 +38,20 @@ public class MessageLatencyMonitor extends DynamicMBeanProvider {
   private final String _domainName;
   private final String _participantName;
 
-  private SimpleDynamicMetric<Long> _totalMessageCount =
-      new SimpleDynamicMetric("TotalMessageCount", 0l);
-  private SimpleDynamicMetric<Long> _totalMessageLatency =
-      new SimpleDynamicMetric("TotalMessageLatency", 0l);
-  private HistogramDynamicMetric _messageLatencyGauge =
-      new HistogramDynamicMetric("MessageLatencyGauge",
-          _metricRegistry.histogram(getMetricName("MessageLatencyGauge")));
+  private SimpleDynamicMetric<Long> _totalMessageCount;
+  private SimpleDynamicMetric<Long> _totalMessageLatency;
+  private HistogramDynamicMetric _messageLatencyGauge;
 
   public MessageLatencyMonitor(String domainName, String participantName) throws JMException {
     _domainName = domainName;
     _participantName = participantName;
     _sensorName = String.format("%s.%s", ParticipantMessageMonitor.PARTICIPANT_STATUS_KEY,
         "MessageLatency");
+
+    _messageLatencyGauge = new HistogramDynamicMetric("MessagelatencyGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _totalMessageLatency = new SimpleDynamicMetric("TotalMessageLatency", 0l);
+    _totalMessageCount = new SimpleDynamicMetric("TotalMessageCount", 0l);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
index 3318ddd..662f323 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ResourceMonitor.java
@@ -19,6 +19,9 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import java.util.concurrent.TimeUnit;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -33,46 +36,27 @@ import javax.management.ObjectName;
 import java.util.*;
 
 public class ResourceMonitor extends DynamicMBeanProvider {
-  private static final long RESET_TIME_RANGE = 1000 * 60 * 60; // 1 hour
 
   // Gauges
-  private SimpleDynamicMetric<Integer> _numOfPartitions =
-      new SimpleDynamicMetric("PartitionGauge", 0);
-  private SimpleDynamicMetric<Integer> _numOfPartitionsInExternalView =
-      new SimpleDynamicMetric("ExternalViewPartitionGauge", 0);
-  private SimpleDynamicMetric<Integer> _numOfErrorPartitions =
-      new SimpleDynamicMetric("ErrorPartitionGauge", 0);
-  private SimpleDynamicMetric<Integer> _numNonTopStatePartitions =
-      new SimpleDynamicMetric("MissingTopStatePartitionGauge", 0);
-  private SimpleDynamicMetric<Long> _numLessMinActiveReplicaPartitions =
-      new SimpleDynamicMetric("MissingMinActiveReplicaPartitionGauge", 0l);
-  private SimpleDynamicMetric<Long> _numLessReplicaPartitions =
-      new SimpleDynamicMetric("MissingReplicaPartitionGauge", 0l);
-  private SimpleDynamicMetric<Long> _numPendingRecoveryRebalancePartitions =
-      new SimpleDynamicMetric("PendingRecoveryRebalancePartitionGauge", 0l);
-  private SimpleDynamicMetric<Long> _numPendingLoadRebalancePartitions =
-      new SimpleDynamicMetric("PendingLoadRebalancePartitionGauge", 0l);
-  private SimpleDynamicMetric<Long> _numRecoveryRebalanceThrottledPartitions =
-      new SimpleDynamicMetric("RecoveryRebalanceThrottledPartitionGauge", 0l);
-  private SimpleDynamicMetric<Long> _numLoadRebalanceThrottledPartitions =
-      new SimpleDynamicMetric("LoadRebalanceThrottledPartitionGauge", 0l);
-  private SimpleDynamicMetric<Integer> _externalViewIdealStateDiff =
-      new SimpleDynamicMetric("DifferenceWithIdealStateGauge", 0);
+  private SimpleDynamicMetric<Integer> _numOfPartitions;
+  private SimpleDynamicMetric<Integer> _numOfPartitionsInExternalView;
+  private SimpleDynamicMetric<Integer> _numOfErrorPartitions;
+  private SimpleDynamicMetric<Integer> _numNonTopStatePartitions;
+  private SimpleDynamicMetric<Long> _numLessMinActiveReplicaPartitions;
+  private SimpleDynamicMetric<Long> _numLessReplicaPartitions;
+  private SimpleDynamicMetric<Long> _numPendingRecoveryRebalancePartitions;
+  private SimpleDynamicMetric<Long> _numPendingLoadRebalancePartitions;
+  private SimpleDynamicMetric<Long> _numRecoveryRebalanceThrottledPartitions;
+  private SimpleDynamicMetric<Long> _numLoadRebalanceThrottledPartitions;
+  private SimpleDynamicMetric<Integer> _externalViewIdealStateDiff;
 
   // Counters
-  private SimpleDynamicMetric<Long> _successfulTopStateHandoffDurationCounter =
-      new SimpleDynamicMetric("SuccessfulTopStateHandoffDurationCounter", 0l);
-  private SimpleDynamicMetric<Long> _successTopStateHandoffCounter =
-      new SimpleDynamicMetric("SucceededTopStateHandoffCounter", 0l);
-  private SimpleDynamicMetric<Long> _failedTopStateHandoffCounter =
-      new SimpleDynamicMetric("FailedTopStateHandoffCounter", 0l);
-  private SimpleDynamicMetric<Long> _maxSinglePartitionTopStateHandoffDuration =
-      new SimpleDynamicMetric("MaxSinglePartitionTopStateHandoffDurationGauge", 0l);
-  private HistogramDynamicMetric _partitionTopStateHandoffDurationGauge =
-      new HistogramDynamicMetric("PartitionTopStateHandoffDurationGauge", _metricRegistry
-          .histogram(getMetricName("PartitionTopStateHandoffDurationGauge")));
-  private SimpleDynamicMetric<Long> _totalMessageReceived =
-      new SimpleDynamicMetric("TotalMessageReceived", 0l);
+  private SimpleDynamicMetric<Long> _successfulTopStateHandoffDurationCounter;
+  private SimpleDynamicMetric<Long> _successTopStateHandoffCounter;
+  private SimpleDynamicMetric<Long> _failedTopStateHandoffCounter;
+  private SimpleDynamicMetric<Long> _maxSinglePartitionTopStateHandoffDuration;
+  private HistogramDynamicMetric _partitionTopStateHandoffDurationGauge;
+  private SimpleDynamicMetric<Long> _totalMessageReceived;
 
   private String _tag = ClusterStatusMonitor.DEFAULT_TAG;
   private long _lastResetTime;
@@ -114,6 +98,34 @@ public class ResourceMonitor extends DynamicMBeanProvider {
     _clusterName = clusterName;
     _resourceName = resourceName;
     _initObjectName = objectName;
+
+    _externalViewIdealStateDiff = new SimpleDynamicMetric("DifferenceWithIdealStateGauge", 0l);
+    _numLoadRebalanceThrottledPartitions =
+        new SimpleDynamicMetric("LoadRebalanceThrottledPartitionGauge", 0l);
+    _numRecoveryRebalanceThrottledPartitions =
+        new SimpleDynamicMetric("RecoveryRebalanceThrottledPartitionGauge", 0l);
+    _numPendingLoadRebalancePartitions =
+        new SimpleDynamicMetric("PendingLoadRebalancePartitionGauge", 0l);
+    _numPendingRecoveryRebalancePartitions =
+        new SimpleDynamicMetric("PendingRecoveryRebalancePartitionGauge", 0l);
+    _numLessReplicaPartitions = new SimpleDynamicMetric("MissingReplicaPartitionGauge", 0l);
+    _numLessMinActiveReplicaPartitions =
+        new SimpleDynamicMetric("MissingMinActiveReplicaPartitionGauge", 0l);
+    _numNonTopStatePartitions = new SimpleDynamicMetric("MissingTopStatePartitionGauge", 0l);
+    _numOfErrorPartitions = new SimpleDynamicMetric("ErrorPartitionGauge", 0l);
+    _numOfPartitionsInExternalView = new SimpleDynamicMetric("ExternalViewPartitionGauge", 0l);
+    _numOfPartitions = new SimpleDynamicMetric("PartitionGauge", 0l);
+
+    _partitionTopStateHandoffDurationGauge =
+        new HistogramDynamicMetric("PartitionTopStateHandoffDurationGauge", new Histogram(
+            new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _totalMessageReceived = new SimpleDynamicMetric("TotalMessageReceived", 0l);
+    _maxSinglePartitionTopStateHandoffDuration =
+        new SimpleDynamicMetric("MaxSinglePartitionTopStateHandoffDurationGauge", 0l);
+    _failedTopStateHandoffCounter = new SimpleDynamicMetric("FailedTopStateHandoffCounter", 0l);
+    _successTopStateHandoffCounter = new SimpleDynamicMetric("SucceededTopStateHandoffCounter", 0l);
+    _successfulTopStateHandoffDurationCounter =
+        new SimpleDynamicMetric("SuccessfulTopStateHandoffDurationCounter", 0l);
   }
 
   @Override
@@ -341,7 +353,7 @@ public class ResourceMonitor extends DynamicMBeanProvider {
   }
 
   public void resetMaxTopStateHandoffGauge() {
-    if (_lastResetTime + RESET_TIME_RANGE <= System.currentTimeMillis()) {
+    if (_lastResetTime + DEFAULT_RESET_INTERVAL_MS <= System.currentTimeMillis()) {
       _maxSinglePartitionTopStateHandoffDuration.updateValue(0l);
       _lastResetTime = System.currentTimeMillis();
     }

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
index e87738b..bc6a36b 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/ZkClientPathMonitor.java
@@ -19,6 +19,9 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowReservoir;
+import java.util.concurrent.TimeUnit;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
 import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
@@ -62,30 +65,19 @@ public class ZkClientPathMonitor extends DynamicMBeanProvider {
     }
   }
 
-  private SimpleDynamicMetric<Long> _readCounter = new SimpleDynamicMetric("ReadCounter", 0l);
-  private SimpleDynamicMetric<Long> _writeCounter = new SimpleDynamicMetric("WriteCounter", 0l);
-  private SimpleDynamicMetric<Long> _readBytesCounter =
-      new SimpleDynamicMetric("ReadBytesCounter", 0l);
-  private SimpleDynamicMetric<Long> _writeBytesCounter =
-      new SimpleDynamicMetric("WriteBytesCounter", 0l);
-  private SimpleDynamicMetric<Long> _readFailureCounter =
-      new SimpleDynamicMetric("ReadFailureCounter", 0l);
-  private SimpleDynamicMetric<Long> _writeFailureCounter =
-      new SimpleDynamicMetric("WriteFailureCounter", 0l);
-  private SimpleDynamicMetric<Long> _readTotalLatencyCounter =
-      new SimpleDynamicMetric("ReadTotalLatencyCounter", 0l);
-  private SimpleDynamicMetric<Long> _writeTotalLatencyCounter =
-      new SimpleDynamicMetric("WriteTotalLatencyCounter", 0l);
-
-  private HistogramDynamicMetric _readLatencyGauge = new HistogramDynamicMetric("ReadLatencyGauge",
-      _metricRegistry.histogram(getMetricName("ReadLatencyGauge")));
-  private HistogramDynamicMetric _writeLatencyGauge =
-      new HistogramDynamicMetric("WriteLatencyGauge",
-          _metricRegistry.histogram(getMetricName("WriteLatencyGauge")));
-  private HistogramDynamicMetric _readBytesGauge = new HistogramDynamicMetric("ReadBytesGauge",
-      _metricRegistry.histogram(getMetricName("ReadBytesGauge")));
-  private HistogramDynamicMetric _writeBytesGauge = new HistogramDynamicMetric("WriteBytesGauge",
-      _metricRegistry.histogram(getMetricName("WriteBytesGauge")));
+  private SimpleDynamicMetric<Long> _readCounter;
+  private SimpleDynamicMetric<Long> _writeCounter;
+  private SimpleDynamicMetric<Long> _readBytesCounter;
+  private SimpleDynamicMetric<Long> _writeBytesCounter;
+  private SimpleDynamicMetric<Long> _readFailureCounter;
+  private SimpleDynamicMetric<Long> _writeFailureCounter;
+  private SimpleDynamicMetric<Long> _readTotalLatencyCounter;
+  private SimpleDynamicMetric<Long> _writeTotalLatencyCounter;
+
+  private HistogramDynamicMetric _readLatencyGauge;
+  private HistogramDynamicMetric _writeLatencyGauge;
+  private HistogramDynamicMetric _readBytesGauge;
+  private HistogramDynamicMetric _writeBytesGauge;
 
   @Override
   public String getSensorName() {
@@ -101,6 +93,24 @@ public class ZkClientPathMonitor extends DynamicMBeanProvider {
     _sensorName = String
         .format("%s.%s.%s.%s", MonitorDomainNames.HelixZkClient.name(), monitorType, monitorKey,
             path.name());
+
+    _writeTotalLatencyCounter = new SimpleDynamicMetric("WriteTotalLatencyCounter", 0l);
+    _readTotalLatencyCounter = new SimpleDynamicMetric("ReadTotalLatencyCounter", 0l);
+    _writeFailureCounter = new SimpleDynamicMetric("WriteFailureCounter", 0l);
+    _readFailureCounter = new SimpleDynamicMetric("ReadFailureCounter", 0l);
+    _writeBytesCounter = new SimpleDynamicMetric("WriteBytesCounter", 0l);
+    _readBytesCounter = new SimpleDynamicMetric("ReadBytesCounter", 0l);
+    _writeCounter = new SimpleDynamicMetric("WriteCounter", 0l);
+    _readCounter = new SimpleDynamicMetric("ReadCounter", 0l);
+
+    _readLatencyGauge = new HistogramDynamicMetric("ReadLatencyGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _writeLatencyGauge = new HistogramDynamicMetric("WriteLatencyGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _readBytesGauge = new HistogramDynamicMetric("ReadBytesGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
+    _writeBytesGauge = new HistogramDynamicMetric("WriteBytesGauge", new Histogram(
+        new SlidingTimeWindowReservoir(DEFAULT_RESET_INTERVAL_MS, TimeUnit.MILLISECONDS)));
   }
 
   public ZkClientPathMonitor register() throws JMException {

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/dynamicMBeans/DynamicMBeanProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/dynamicMBeans/DynamicMBeanProvider.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/dynamicMBeans/DynamicMBeanProvider.java
index cc97b3b..b44c63c 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/dynamicMBeans/DynamicMBeanProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/dynamicMBeans/DynamicMBeanProvider.java
@@ -19,9 +19,6 @@ package org.apache.helix.monitoring.mbeans.dynamicMBeans;
  * under the License.
  */
 
-import com.codahale.metrics.Metric;
-import com.codahale.metrics.MetricFilter;
-import com.codahale.metrics.MetricRegistry;
 import org.apache.helix.HelixException;
 import org.apache.helix.monitoring.SensorNameProvider;
 import org.apache.helix.monitoring.mbeans.MBeanRegistrar;
@@ -36,7 +33,7 @@ import java.util.*;
  */
 public abstract class DynamicMBeanProvider implements DynamicMBean, SensorNameProvider {
   protected final Logger _logger = LoggerFactory.getLogger(getClass());
-  protected static final MetricRegistry _metricRegistry = new MetricRegistry();
+  protected static final long DEFAULT_RESET_INTERVAL_MS = 60 * 60 * 1000; // Reset time every hour
   private static String SENSOR_NAME_TAG = "SensorName";
   private static String DEFAULT_DESCRIPTION =
       "Information on the management interface of the MBean";
@@ -86,11 +83,6 @@ public abstract class DynamicMBeanProvider implements DynamicMBean, SensorNamePr
     doRegister(dynamicMetrics, null, objectName);
   }
 
-  protected String getMetricName(String metricName) {
-    return MetricRegistry
-        .name(getClass().getSimpleName(), Integer.toHexString(hashCode()), metricName);
-  }
-
   /**
    * Update the Dynamic MBean provider with new metric list.
    *
@@ -145,13 +137,6 @@ public abstract class DynamicMBeanProvider implements DynamicMBean, SensorNamePr
    * After unregistered, the MBean can't be registered again, a new monitor has be to created.
    */
   public synchronized void unregister() {
-    final String metricNamePrefix = getMetricName(null);
-    _metricRegistry.removeMatching(new MetricFilter() {
-      @Override
-      public boolean matches(String name, Metric metric) {
-        return name.startsWith(metricNamePrefix);
-      }
-    });
     MBeanRegistrar.unregister(_objectName);
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/3a73b0f3/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterEventStatusMonitor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterEventStatusMonitor.java b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterEventStatusMonitor.java
index eb4f94b..b607add 100644
--- a/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterEventStatusMonitor.java
+++ b/helix-core/src/test/java/org/apache/helix/monitoring/TestClusterEventStatusMonitor.java
@@ -22,8 +22,10 @@ package org.apache.helix.monitoring;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import javax.management.AttributeNotFoundException;
 import javax.management.InstanceNotFoundException;
+import javax.management.JMException;
 import javax.management.MBeanException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
@@ -40,21 +42,43 @@ import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class TestClusterEventStatusMonitor {
+  private static final int TEST_SLIDING_WINDOW_MS = 2000; // 2s window for testing
+
+  private class ClusterStatusMonitorForTest extends ClusterStatusMonitor {
+    public ClusterStatusMonitorForTest(String clusterName) {
+      super(clusterName);
+    }
+    public ConcurrentHashMap<String, ClusterEventMonitor> getClusterEventMBean() {
+      return _clusterEventMbeanMap;
+    }
+  }
 
   @Test()
   public void test()
       throws InstanceNotFoundException, MalformedObjectNameException, NullPointerException,
       IOException, InterruptedException, MBeanException, AttributeNotFoundException,
-      ReflectionException {
+      ReflectionException{
     System.out.println("START TestClusterEventStatusMonitor");
     String clusterName = "TestCluster";
-    ClusterStatusMonitor monitor = new ClusterStatusMonitor(clusterName);
+    ClusterStatusMonitorForTest monitor = new ClusterStatusMonitorForTest(clusterName);
 
     MBeanServer _server = ManagementFactory.getPlatformMBeanServer();
     Set<ObjectInstance> mbeans =
         _server.queryMBeans(new ObjectName("ClusterStatus:Cluster=TestCluster,eventName=ClusterEvent,*"), null);
     Assert.assertEquals(mbeans.size(), 0);
 
+    // Customize event monitors for testing
+    try {
+      this.addTestEventMonitor(monitor, ClusterEventMonitor.PhaseName.Callback.name());
+      this.addTestEventMonitor(monitor, ClusterEventMonitor.PhaseName.InQueue.name());
+      this.addTestEventMonitor(monitor, BestPossibleStateCalcStage.class.getSimpleName());
+      this.addTestEventMonitor(monitor, ReadClusterDataStage.class.getSimpleName());
+      this.addTestEventMonitor(monitor, IntermediateStateCalcStage.class.getSimpleName());
+      this.addTestEventMonitor(monitor, TaskAssignmentStage.class.getSimpleName());
+    } catch (JMException jme) {
+      Assert.assertTrue(false, "Failed to customize event monitors");
+    }
+
     int count = 5;
     Long totalDuration = 0L;
     for (int i = 1; i <= count; i++) {
@@ -77,9 +101,39 @@ public class TestClusterEventStatusMonitor {
       Long maxDuration = (Long) _server.getAttribute(mbean.getObjectName(), "MaxSingleDurationGauge");
       Long eventCount = (Long) _server.getAttribute(mbean.getObjectName(), "EventCounter");
 
+      Double pct75th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct75th");
+      Double pct95th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct95th");
+      Double pct99th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct99th");
+      Long max = (Long) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Max");
+      Double stddev = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.StdDev");
+
       Assert.assertEquals(duration, totalDuration);
       Assert.assertEquals(maxDuration, Long.valueOf(100 * count));
       Assert.assertEquals(eventCount, Long.valueOf(count));
+      Assert.assertTrue(Math.abs(pct75th - 450.0) < 1);
+      Assert.assertTrue(Math.abs(pct95th - 500.0) < 1);
+      Assert.assertTrue(Math.abs(pct99th - 500.0) < 1);
+      Assert.assertTrue(max == 500);
+      Assert.assertTrue(Math.abs(stddev - 158.0) < 0.2);
+    }
+
+    System.out.println("\nWaiting for time window to expire\n");
+    Thread.sleep(TEST_SLIDING_WINDOW_MS);
+
+    // Since sliding window has expired, just make sure histograms have its values reset
+    for (ObjectInstance mbean : mbeans) {
+      Double pct75th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct75th");
+      Double pct95th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct95th");
+      Double pct99th = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Pct99th");
+      Long max = (Long) _server.getAttribute(mbean.getObjectName(), "DurationGauge.Max");
+      Double stddev = (Double) _server.getAttribute(mbean.getObjectName(), "DurationGauge.StdDev");
+
+      Assert.assertTrue(pct75th == 0.0);
+      Assert.assertTrue(pct95th == 0.0);
+      Assert.assertTrue(pct99th == 0.0);
+      Assert.assertTrue(max == 0);
+      Assert.assertTrue(stddev == 0.0);
+
     }
 
     monitor.reset();
@@ -91,4 +145,14 @@ public class TestClusterEventStatusMonitor {
 
     System.out.println("END TestParticipantMonitor");
   }
+
+  private void addTestEventMonitor(ClusterStatusMonitorForTest monitor, String phaseName) throws
+      JMException {
+    ConcurrentHashMap<String, ClusterEventMonitor> mbean = monitor.getClusterEventMBean();
+    ClusterEventMonitor eventMonitor = new ClusterEventMonitor(monitor, phaseName,
+        TEST_SLIDING_WINDOW_MS);
+    eventMonitor.register();
+    mbean.put(phaseName, eventMonitor);
+  }
+
 }


[43/50] [abbrv] helix git commit: [helix-front] Upgrade Angular to 5; Material to 5; ngx-datatable to 11

Posted by jx...@apache.org.
http://git-wip-us.apache.org/repos/asf/helix/blob/b3ecd2a7/helix-front/package-lock.json
----------------------------------------------------------------------
diff --git a/helix-front/package-lock.json b/helix-front/package-lock.json
new file mode 100644
index 0000000..29476b5
--- /dev/null
+++ b/helix-front/package-lock.json
@@ -0,0 +1,11094 @@
+{
+  "name": "helix-front",
+  "version": "1.1.0",
+  "lockfileVersion": 1,
+  "requires": true,
+  "dependencies": {
+    "@angular-devkit/build-optimizer": {
+      "version": "0.0.36",
+      "resolved": "https://registry.npmjs.org/@angular-devkit/build-optimizer/-/build-optimizer-0.0.36.tgz",
+      "integrity": "sha512-EFFF7hBbVoTOzYfXuSlGhcDr8neafmwuBAIkzAekEjzik7OaTLq7LPG7As+ebed9ll+3DAGypnrpdIE1Tp/H/A==",
+      "dev": true,
+      "requires": {
+        "loader-utils": "1.1.0",
+        "source-map": "0.5.7",
+        "typescript": "2.6.2",
+        "webpack-sources": "1.1.0"
+      },
+      "dependencies": {
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        },
+        "typescript": {
+          "version": "2.6.2",
+          "resolved": "https://registry.npmjs.org/typescript/-/typescript-2.6.2.tgz",
+          "integrity": "sha1-PFtv1/beCRQmkCfwPAlGdY92c6Q=",
+          "dev": true
+        }
+      }
+    },
+    "@angular-devkit/core": {
+      "version": "0.0.22",
+      "resolved": "https://registry.npmjs.org/@angular-devkit/core/-/core-0.0.22.tgz",
+      "integrity": "sha512-zxrNtTiv60liye/GGeRMnnGgLgAWoqlMTfPLMW0D1qJ4bbrPHtme010mpxS3QL4edcDtQseyXSFCnEkuo2MrRw==",
+      "dev": true,
+      "requires": {
+        "source-map": "0.5.7"
+      },
+      "dependencies": {
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        }
+      }
+    },
+    "@angular-devkit/schematics": {
+      "version": "0.0.42",
+      "resolved": "https://registry.npmjs.org/@angular-devkit/schematics/-/schematics-0.0.42.tgz",
+      "integrity": "sha512-elTiNL0Nx9oin2pfZTvMBU/d9sgutXaZe8n3xm2p7jfqQZry5MYYFES4hq+WIJjtV/X9gAniafncEpxuF7ikYw==",
+      "dev": true,
+      "requires": {
+        "@angular-devkit/core": "0.0.22",
+        "@ngtools/json-schema": "1.1.0",
+        "@schematics/schematics": "0.0.11",
+        "minimist": "1.2.0",
+        "rxjs": "5.5.5"
+      }
+    },
+    "@angular/animations": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/animations/-/animations-5.1.1.tgz",
+      "integrity": "sha512-PHLBWDnAzr5b5l52pk5ZYmv/6m0YUe2ICwu5dmbS0d8Kf5dXadMphAWCDbljMF+djGyZeFq2/dQ/t7ygYl3YuA==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/cdk": {
+      "version": "5.0.1",
+      "resolved": "https://registry.npmjs.org/@angular/cdk/-/cdk-5.0.1.tgz",
+      "integrity": "sha512-uK4Vyaf06J8KqePzq35BxMHRGolt35EnbZf9wjCs7eYaghbQ7Pk2xUGoynu5Lj1wAOn5N1/C1nT2/aAH/EE2rw==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/cli": {
+      "version": "1.6.1",
+      "resolved": "https://registry.npmjs.org/@angular/cli/-/cli-1.6.1.tgz",
+      "integrity": "sha512-cdXfUR1by7nCgg9H9HijV8aI36AmceMubBZ/k+UgICMd8DlkPCVBIWcUXLdoH/l6nuXb3gCucfJoThznlakNMw==",
+      "dev": true,
+      "requires": {
+        "@angular-devkit/build-optimizer": "0.0.36",
+        "@angular-devkit/schematics": "0.0.42",
+        "@ngtools/json-schema": "1.1.0",
+        "@ngtools/webpack": "1.9.1",
+        "@schematics/angular": "0.1.11",
+        "autoprefixer": "6.7.7",
+        "chalk": "2.2.2",
+        "circular-dependency-plugin": "4.3.0",
+        "common-tags": "1.5.1",
+        "copy-webpack-plugin": "4.3.0",
+        "core-object": "3.1.5",
+        "css-loader": "0.28.7",
+        "cssnano": "3.10.0",
+        "denodeify": "1.2.1",
+        "ember-cli-string-utils": "1.1.0",
+        "exports-loader": "0.6.4",
+        "extract-text-webpack-plugin": "3.0.2",
+        "file-loader": "1.1.5",
+        "fs-extra": "4.0.3",
+        "glob": "7.1.2",
+        "html-webpack-plugin": "2.30.1",
+        "istanbul-instrumenter-loader": "2.0.0",
+        "karma-source-map-support": "1.2.0",
+        "less": "2.7.3",
+        "less-loader": "4.0.5",
+        "license-webpack-plugin": "1.1.1",
+        "loader-utils": "1.1.0",
+        "lodash": "4.17.4",
+        "memory-fs": "0.4.1",
+        "minimatch": "3.0.4",
+        "node-modules-path": "1.0.1",
+        "node-sass": "4.5.0",
+        "nopt": "4.0.1",
+        "opn": "5.1.0",
+        "portfinder": "1.0.13",
+        "postcss-custom-properties": "6.2.0",
+        "postcss-loader": "2.0.9",
+        "postcss-url": "7.3.0",
+        "raw-loader": "0.5.1",
+        "resolve": "1.5.0",
+        "rxjs": "5.5.5",
+        "sass-loader": "6.0.6",
+        "semver": "5.4.1",
+        "silent-error": "1.1.0",
+        "source-map-loader": "0.2.3",
+        "source-map-support": "0.4.18",
+        "style-loader": "0.13.2",
+        "stylus": "0.54.5",
+        "stylus-loader": "3.0.1",
+        "uglifyjs-webpack-plugin": "1.1.4",
+        "url-loader": "0.6.2",
+        "webpack": "3.10.0",
+        "webpack-dev-middleware": "1.12.2",
+        "webpack-dev-server": "2.9.7",
+        "webpack-merge": "4.1.1",
+        "webpack-sources": "1.1.0",
+        "webpack-subresource-integrity": "1.0.3",
+        "zone.js": "0.8.18"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "3.2.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.0.tgz",
+          "integrity": "sha512-NnSOmMEYtVR2JVMIGTzynRkkaxtiq1xnFBcdQD/DnNCYPoEPsVJhM98BDyaoNOQIi7p4okdi3E27eN7GQbsUug==",
+          "dev": true,
+          "requires": {
+            "color-convert": "1.9.1"
+          }
+        },
+        "chalk": {
+          "version": "2.2.2",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.2.2.tgz",
+          "integrity": "sha512-LvixLAQ4MYhbf7hgL4o5PeK32gJKvVzDRiSNIApDofQvyhl8adgG2lJVXn4+ekQoK7HL9RF8lqxwerpe0x2pCw==",
+          "dev": true,
+          "requires": {
+            "ansi-styles": "3.2.0",
+            "escape-string-regexp": "1.0.5",
+            "supports-color": "4.5.0"
+          }
+        },
+        "nopt": {
+          "version": "4.0.1",
+          "resolved": "https://registry.npmjs.org/nopt/-/nopt-4.0.1.tgz",
+          "integrity": "sha1-0NRoWv1UFRk8jHUFYC0NF81kR00=",
+          "dev": true,
+          "requires": {
+            "abbrev": "1.1.1",
+            "osenv": "0.1.4"
+          }
+        },
+        "supports-color": {
+          "version": "4.5.0",
+          "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-4.5.0.tgz",
+          "integrity": "sha1-vnoN5ITexcXN34s9WRJQRJEvY1s=",
+          "dev": true,
+          "requires": {
+            "has-flag": "2.0.0"
+          }
+        }
+      }
+    },
+    "@angular/common": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/common/-/common-5.1.1.tgz",
+      "integrity": "sha512-SFRzdDthoiKaMLuV+TAwjKXFWwTRFGuidlWC3BhUf8/HzNSePAdvfdQcqbEaE5buMn403OV105S9Tyx5tILQeA==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/compiler": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/compiler/-/compiler-5.1.1.tgz",
+      "integrity": "sha512-k4J2kRiBjtjkDcDut2JVUpqQGLJWd8j3Don+swzZHuEklbLmsVRGM6u/fmH0K9TMwKHtC5Ycap8kj4bWXUYfwg==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/compiler-cli": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/compiler-cli/-/compiler-cli-5.1.1.tgz",
+      "integrity": "sha512-X3n1V0fAsZzJDRLM2OPiOri8rrQ2ILFS0VDqPdHMa1HbpF0ZKe1Yyux2rhGSbS83a1Eanx6RqfDkrUalKEprbw==",
+      "dev": true,
+      "requires": {
+        "chokidar": "1.7.0",
+        "minimist": "1.2.0",
+        "reflect-metadata": "0.1.10",
+        "tsickle": "0.25.5"
+      }
+    },
+    "@angular/core": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/core/-/core-5.1.1.tgz",
+      "integrity": "sha512-8HJ0lNM5Z+pf+JfOl5mAWgNfrdtnMhVcEGCEniJAQweKOfYCziuyB0ALkX/Q6jGmd2IshR36SarwCYEc5ttt/w==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/flex-layout": {
+      "version": "2.0.0-beta.12",
+      "resolved": "https://registry.npmjs.org/@angular/flex-layout/-/flex-layout-2.0.0-beta.12.tgz",
+      "integrity": "sha512-QTOKZxehYTh8fj64V/pNVWNbfNtebSbssyMIXiGJuHTzfyF7GYdRmtjoR2pNpllycz3rE5NYX77EB140Y6BCnw==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/forms": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/forms/-/forms-5.1.1.tgz",
+      "integrity": "sha512-4iN/8N0DgnV82XIb/8PqlFIGrog8BHJlzQ9sdAlpT29biPFezFpqpsXkjLBouBc7oBFTgoyXMgWDj8IGRmwLGQ==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/http": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/http/-/http-5.1.1.tgz",
+      "integrity": "sha512-oeiLX00TaFlGS5Y4EAGnxxVitN8T9X8olhSC+XDDAAL3JHTAyh4dj7me8vNZk1VaqPFa9AXu4D34vu1Zsm0c1g==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/material": {
+      "version": "5.0.1",
+      "resolved": "https://registry.npmjs.org/@angular/material/-/material-5.0.1.tgz",
+      "integrity": "sha512-k95i58ZIVneLE61a5JliM10NSasy9P5C2JJUESo3s/rxt9dq/9XOWpUvNCy49OHYBRFJBlsyrLM6E2V7/tmq4w==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/platform-browser": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/platform-browser/-/platform-browser-5.1.1.tgz",
+      "integrity": "sha512-QpkNXoO2pqURQJxXPhZo6RFeirKbr56O0SwoMpYfXGGN1qEIicoWZHobCUTp7/jvjx5Xjc7886Fvu/qJrE7wVA==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/platform-browser-dynamic": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/platform-browser-dynamic/-/platform-browser-dynamic-5.1.1.tgz",
+      "integrity": "sha512-xnin1eK5nF7EO4tYZvRlhT28DyhL3p4NKWsZQwfqyBwSF0T2mJ1vjhjCZVT0MmaOyt5D+0eUkHIhBDqeZyBMMQ==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@angular/router": {
+      "version": "5.1.1",
+      "resolved": "https://registry.npmjs.org/@angular/router/-/router-5.1.1.tgz",
+      "integrity": "sha512-96mBZS1b1Dt7HFOGKh5zI/1U6F3zT4cdjIaBmcCKkbyKhs3WRAPXxxCkuCwr6lWmBeQt4iEvSdXiHQbD0iCG7Q==",
+      "requires": {
+        "tslib": "1.8.1"
+      }
+    },
+    "@ngtools/json-schema": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/@ngtools/json-schema/-/json-schema-1.1.0.tgz",
+      "integrity": "sha1-w6DFRNYjkqzCgTpCyKDcb1j4aSI=",
+      "dev": true
+    },
+    "@ngtools/webpack": {
+      "version": "1.9.1",
+      "resolved": "https://registry.npmjs.org/@ngtools/webpack/-/webpack-1.9.1.tgz",
+      "integrity": "sha512-+Fa/cpwiVnQuOMUUVmBH7+/xkhZHG/uEL60FBt6Pv8yFdLoXkgYPllJKsph5uWGpv40/zlnDYQv7k7yngeCX2A==",
+      "dev": true,
+      "requires": {
+        "chalk": "2.2.2",
+        "enhanced-resolve": "3.4.1",
+        "loader-utils": "1.1.0",
+        "magic-string": "0.22.4",
+        "semver": "5.4.1",
+        "source-map": "0.5.7",
+        "tree-kill": "1.2.0"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "3.2.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.0.tgz",
+          "integrity": "sha512-NnSOmMEYtVR2JVMIGTzynRkkaxtiq1xnFBcdQD/DnNCYPoEPsVJhM98BDyaoNOQIi7p4okdi3E27eN7GQbsUug==",
+          "dev": true,
+          "requires": {
+            "color-convert": "1.9.1"
+          }
+        },
+        "chalk": {
+          "version": "2.2.2",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.2.2.tgz",
+          "integrity": "sha512-LvixLAQ4MYhbf7hgL4o5PeK32gJKvVzDRiSNIApDofQvyhl8adgG2lJVXn4+ekQoK7HL9RF8lqxwerpe0x2pCw==",
+          "dev": true,
+          "requires": {
+            "ansi-styles": "3.2.0",
+            "escape-string-regexp": "1.0.5",
+            "supports-color": "4.5.0"
+          }
+        },
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        },
+        "supports-color": {
+          "version": "4.5.0",
+          "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-4.5.0.tgz",
+          "integrity": "sha1-vnoN5ITexcXN34s9WRJQRJEvY1s=",
+          "dev": true,
+          "requires": {
+            "has-flag": "2.0.0"
+          }
+        }
+      }
+    },
+    "@schematics/angular": {
+      "version": "0.1.11",
+      "resolved": "https://registry.npmjs.org/@schematics/angular/-/angular-0.1.11.tgz",
+      "integrity": "sha512-jYTantZjdYeDjxh9ZLYvGbDI0VeUxgSrcBjHvnHqMNe+YGJenY988ifWCwzjmOowj57maLrQQGrdoO7oUeNdyw==",
+      "dev": true,
+      "requires": {
+        "@angular-devkit/core": "0.0.22"
+      }
+    },
+    "@schematics/schematics": {
+      "version": "0.0.11",
+      "resolved": "https://registry.npmjs.org/@schematics/schematics/-/schematics-0.0.11.tgz",
+      "integrity": "sha512-HAXgAIuuAGjiIKohGlRUkmUTWYtNmclR12KHlQQxT9pHFdEb2OrpHjUp2YoV32jiU6jIZm4pf3ODwlPA0VbwnA==",
+      "dev": true
+    },
+    "@swimlane/ngx-datatable": {
+      "version": "11.1.7",
+      "resolved": "https://registry.npmjs.org/@swimlane/ngx-datatable/-/ngx-datatable-11.1.7.tgz",
+      "integrity": "sha512-TMDN26Q4J+Sh+OPqAx8BK5Q/3hAAmcTAUQ9wvC9nboSOAmYUaHGz8t21yGdeUtRtunIfMpjTnwLa+X2Pfoq42w=="
+    },
+    "@types/form-data": {
+      "version": "2.2.1",
+      "resolved": "https://registry.npmjs.org/@types/form-data/-/form-data-2.2.1.tgz",
+      "integrity": "sha512-JAMFhOaHIciYVh8fb5/83nmuO/AHwmto+Hq7a9y8FzLDcC1KCU344XDOMEmahnrTFlHjgh4L0WJFczNIX2GxnQ==",
+      "dev": true,
+      "requires": {
+        "@types/node": "6.0.94"
+      }
+    },
+    "@types/hammerjs": {
+      "version": "2.0.35",
+      "resolved": "https://registry.npmjs.org/@types/hammerjs/-/hammerjs-2.0.35.tgz",
+      "integrity": "sha512-4mUIMSZ2U4UOWq1b+iV7XUTE4w+Kr3x+Zb/Qz5ROO6BTZLw2c8/ftjq0aRgluguLs4KRuBnrOy/s389HVn1/zA==",
+      "dev": true
+    },
+    "@types/jasmine": {
+      "version": "2.5.38",
+      "resolved": "https://registry.npmjs.org/@types/jasmine/-/jasmine-2.5.38.tgz",
+      "integrity": "sha1-pDeRJMSSHU4h3lTsdGacnps1Zxc=",
+      "dev": true
+    },
+    "@types/lodash": {
+      "version": "4.14.90",
+      "resolved": "https://registry.npmjs.org/@types/lodash/-/lodash-4.14.90.tgz",
+      "integrity": "sha512-f59r3IBSWZNmFQV8k9sTLslvkNrO0/DnuxFbM0xfdr3R7a97Bs6kmuSEIiwWWAFA+1KABGaRzFoUVt/wR/B2qQ==",
+      "dev": true
+    },
+    "@types/node": {
+      "version": "6.0.94",
+      "resolved": "https://registry.npmjs.org/@types/node/-/node-6.0.94.tgz",
+      "integrity": "sha512-CwopBfOTONzc1bDDTh8/KzW+zssiIPw+nSf27Y1cuGIkZJ7zuhkig6xO5p9pBW/RY99DznOMCIj+FXx8EIy+qw==",
+      "dev": true
+    },
+    "@types/q": {
+      "version": "0.0.32",
+      "resolved": "https://registry.npmjs.org/@types/q/-/q-0.0.32.tgz",
+      "integrity": "sha1-vShOV8hPEyXacCur/IKlMoGQwMU=",
+      "dev": true
+    },
+    "@types/request": {
+      "version": "2.0.9",
+      "resolved": "https://registry.npmjs.org/@types/request/-/request-2.0.9.tgz",
+      "integrity": "sha512-IWq4Uhm7dShxPdnfbGtIv9ImhGFJ3HugoIfugUd+jt40Oxe6ZfWIEaHFvp4QmRNZVDj0G6dZfa+u0U0PF3bYpg==",
+      "dev": true,
+      "requires": {
+        "@types/form-data": "2.2.1",
+        "@types/node": "6.0.94"
+      }
+    },
+    "@types/selenium-webdriver": {
+      "version": "2.53.43",
+      "resolved": "https://registry.npmjs.org/@types/selenium-webdriver/-/selenium-webdriver-2.53.43.tgz",
+      "integrity": "sha512-UBYHWph6P3tutkbXpW6XYg9ZPbTKjw/YC2hGG1/GEvWwTbvezBUv3h+mmUFw79T3RFPnmedpiXdOBbXX+4l0jg==",
+      "dev": true
+    },
+    "abbrev": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz",
+      "integrity": "sha512-nne9/IiQ/hzIhY6pdDnbBtz7DjPTKrY00P/zvPSm5pOFkl6xuGrGnXn/VtTNNfNtAfZ9/1RtehkszU9qcTii0Q=="
+    },
+    "accepts": {
+      "version": "1.3.4",
+      "resolved": "https://registry.npmjs.org/accepts/-/accepts-1.3.4.tgz",
+      "integrity": "sha1-hiRnWMfdbSGmR0/whKR0DsBesh8=",
+      "requires": {
+        "mime-types": "2.1.17",
+        "negotiator": "0.6.1"
+      }
+    },
+    "acorn": {
+      "version": "5.2.1",
+      "resolved": "https://registry.npmjs.org/acorn/-/acorn-5.2.1.tgz",
+      "integrity": "sha512-jG0u7c4Ly+3QkkW18V+NRDN+4bWHdln30NL1ZL2AvFZZmQe/BfopYCtghCKKVBUSetZ4QKcyA0pY6/4Gw8Pv8w==",
+      "dev": true
+    },
+    "acorn-dynamic-import": {
+      "version": "2.0.2",
+      "resolved": "https://registry.npmjs.org/acorn-dynamic-import/-/acorn-dynamic-import-2.0.2.tgz",
+      "integrity": "sha1-x1K9IQvvZ5UBtsbLf8hPj0cVjMQ=",
+      "dev": true,
+      "requires": {
+        "acorn": "4.0.13"
+      },
+      "dependencies": {
+        "acorn": {
+          "version": "4.0.13",
+          "resolved": "https://registry.npmjs.org/acorn/-/acorn-4.0.13.tgz",
+          "integrity": "sha1-EFSVrlNh1pe9GVyCUZLhrX8lN4c=",
+          "dev": true
+        }
+      }
+    },
+    "adm-zip": {
+      "version": "0.4.7",
+      "resolved": "https://registry.npmjs.org/adm-zip/-/adm-zip-0.4.7.tgz",
+      "integrity": "sha1-hgbCy/HEJs6MjsABdER/1Jtur8E=",
+      "dev": true
+    },
+    "after": {
+      "version": "0.8.2",
+      "resolved": "https://registry.npmjs.org/after/-/after-0.8.2.tgz",
+      "integrity": "sha1-/ts5T58OAqqXaOcCvaI7UF+ufh8=",
+      "dev": true
+    },
+    "agent-base": {
+      "version": "2.1.1",
+      "resolved": "https://registry.npmjs.org/agent-base/-/agent-base-2.1.1.tgz",
+      "integrity": "sha1-1t4Q1a9hMtW9aSQn1G/FOFOQlMc=",
+      "dev": true,
+      "requires": {
+        "extend": "3.0.1",
+        "semver": "5.0.3"
+      },
+      "dependencies": {
+        "semver": {
+          "version": "5.0.3",
+          "resolved": "https://registry.npmjs.org/semver/-/semver-5.0.3.tgz",
+          "integrity": "sha1-d0Zt5YnNXTyV8TiqeLxWmjy10no=",
+          "dev": true
+        }
+      }
+    },
+    "ajv": {
+      "version": "5.5.1",
+      "resolved": "https://registry.npmjs.org/ajv/-/ajv-5.5.1.tgz",
+      "integrity": "sha1-s4u4h22ehr7plJVqBOch6IskjrI=",
+      "requires": {
+        "co": "4.6.0",
+        "fast-deep-equal": "1.0.0",
+        "fast-json-stable-stringify": "2.0.0",
+        "json-schema-traverse": "0.3.1"
+      }
+    },
+    "ajv-keywords": {
+      "version": "2.1.1",
+      "resolved": "https://registry.npmjs.org/ajv-keywords/-/ajv-keywords-2.1.1.tgz",
+      "integrity": "sha1-YXmX/F9gV2iUxDX5QNgZ4TW4B2I=",
+      "dev": true
+    },
+    "align-text": {
+      "version": "0.1.4",
+      "resolved": "https://registry.npmjs.org/align-text/-/align-text-0.1.4.tgz",
+      "integrity": "sha1-DNkKVhCT810KmSVsIrcGlDP60Rc=",
+      "dev": true,
+      "requires": {
+        "kind-of": "3.2.2",
+        "longest": "1.0.1",
+        "repeat-string": "1.6.1"
+      }
+    },
+    "alphanum-sort": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/alphanum-sort/-/alphanum-sort-1.0.2.tgz",
+      "integrity": "sha1-l6ERlkmyEa0zaR2fn0hqjsn74KM=",
+      "dev": true
+    },
+    "amdefine": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/amdefine/-/amdefine-1.0.1.tgz",
+      "integrity": "sha1-SlKCrBZHKek2Gbz9OtFR+BfOkfU="
+    },
+    "angulartics2": {
+      "version": "2.5.0",
+      "resolved": "https://registry.npmjs.org/angulartics2/-/angulartics2-2.5.0.tgz",
+      "integrity": "sha1-5cUAIRmgaKZ1lDjVnGxpLUjxaPY="
+    },
+    "ansi-html": {
+      "version": "0.0.7",
+      "resolved": "https://registry.npmjs.org/ansi-html/-/ansi-html-0.0.7.tgz",
+      "integrity": "sha1-gTWEAhliqenm/QOflA0S9WynhZ4=",
+      "dev": true
+    },
+    "ansi-regex": {
+      "version": "2.1.1",
+      "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz",
+      "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8="
+    },
+    "ansi-styles": {
+      "version": "2.2.1",
+      "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-2.2.1.tgz",
+      "integrity": "sha1-tDLdM1i2NM914eRmQ2gkBTPB3b4="
+    },
+    "any-promise": {
+      "version": "1.3.0",
+      "resolved": "https://registry.npmjs.org/any-promise/-/any-promise-1.3.0.tgz",
+      "integrity": "sha1-q8av7tzqUugJzcA3au0845Y10X8=",
+      "dev": true
+    },
+    "anymatch": {
+      "version": "1.3.2",
+      "resolved": "https://registry.npmjs.org/anymatch/-/anymatch-1.3.2.tgz",
+      "integrity": "sha512-0XNayC8lTHQ2OI8aljNCN3sSx6hsr/1+rlcDAotXJR7C1oZZHCNsfpbKwMjRA3Uqb5tF1Rae2oloTr4xpq+WjA==",
+      "dev": true,
+      "requires": {
+        "micromatch": "2.3.11",
+        "normalize-path": "2.1.1"
+      }
+    },
+    "app-root-path": {
+      "version": "2.0.1",
+      "resolved": "https://registry.npmjs.org/app-root-path/-/app-root-path-2.0.1.tgz",
+      "integrity": "sha1-zWLc+OT9WkF+/GZNLlsQZTxlG0Y=",
+      "dev": true
+    },
+    "append-transform": {
+      "version": "0.4.0",
+      "resolved": "https://registry.npmjs.org/append-transform/-/append-transform-0.4.0.tgz",
+      "integrity": "sha1-126/jKlNJ24keja61EpLdKthGZE=",
+      "dev": true,
+      "requires": {
+        "default-require-extensions": "1.0.0"
+      }
+    },
+    "aproba": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/aproba/-/aproba-1.2.0.tgz",
+      "integrity": "sha512-Y9J6ZjXtoYh8RnXVCMOU/ttDmk1aBjunq9vO0ta5x85WDQiQfUF9sIPBITdbiiIVcBo03Hi3jMxigBtsddlXRw=="
+    },
+    "are-we-there-yet": {
+      "version": "1.1.4",
+      "resolved": "https://registry.npmjs.org/are-we-there-yet/-/are-we-there-yet-1.1.4.tgz",
+      "integrity": "sha1-u13KOCu5TwXhUZQ3PRb9O6HKEQ0=",
+      "requires": {
+        "delegates": "1.0.0",
+        "readable-stream": "2.3.3"
+      }
+    },
+    "argparse": {
+      "version": "1.0.9",
+      "resolved": "https://registry.npmjs.org/argparse/-/argparse-1.0.9.tgz",
+      "integrity": "sha1-c9g7wmP4bpf4zE9rrhsOkKfSLIY=",
+      "dev": true,
+      "requires": {
+        "sprintf-js": "1.0.3"
+      }
+    },
+    "arr-diff": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/arr-diff/-/arr-diff-2.0.0.tgz",
+      "integrity": "sha1-jzuCf5Vai9ZpaX5KQlasPOrjVs8=",
+      "dev": true,
+      "requires": {
+        "arr-flatten": "1.1.0"
+      }
+    },
+    "arr-flatten": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/arr-flatten/-/arr-flatten-1.1.0.tgz",
+      "integrity": "sha512-L3hKV5R/p5o81R7O02IGnwpDmkp6E982XhtbuwSe3O4qOtMMMtodicASA1Cny2U+aCXcNpml+m4dPsvsJ3jatg==",
+      "dev": true
+    },
+    "array-find-index": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/array-find-index/-/array-find-index-1.0.2.tgz",
+      "integrity": "sha1-3wEKoSh+Fku9pvlyOwqWoexBh6E="
+    },
+    "array-flatten": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-1.1.1.tgz",
+      "integrity": "sha1-ml9pkFGx5wczKPKgCJaLZOopVdI="
+    },
+    "array-includes": {
+      "version": "3.0.3",
+      "resolved": "https://registry.npmjs.org/array-includes/-/array-includes-3.0.3.tgz",
+      "integrity": "sha1-GEtI9i2S10UrsxsyMWXH+L0CJm0=",
+      "dev": true,
+      "requires": {
+        "define-properties": "1.1.2",
+        "es-abstract": "1.10.0"
+      }
+    },
+    "array-slice": {
+      "version": "0.2.3",
+      "resolved": "https://registry.npmjs.org/array-slice/-/array-slice-0.2.3.tgz",
+      "integrity": "sha1-3Tz7gO15c6dRF82sabC5nshhhvU=",
+      "dev": true
+    },
+    "array-union": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/array-union/-/array-union-1.0.2.tgz",
+      "integrity": "sha1-mjRBDk9OPaI96jdb5b5w8kd47Dk=",
+      "dev": true,
+      "requires": {
+        "array-uniq": "1.0.3"
+      }
+    },
+    "array-uniq": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/array-uniq/-/array-uniq-1.0.3.tgz",
+      "integrity": "sha1-r2rId6Jcx/dOBYiUdThY39sk/bY=",
+      "dev": true
+    },
+    "array-unique": {
+      "version": "0.2.1",
+      "resolved": "https://registry.npmjs.org/array-unique/-/array-unique-0.2.1.tgz",
+      "integrity": "sha1-odl8yvy8JiXMcPrc6zalDFiwGlM=",
+      "dev": true
+    },
+    "arraybuffer.slice": {
+      "version": "0.0.6",
+      "resolved": "https://registry.npmjs.org/arraybuffer.slice/-/arraybuffer.slice-0.0.6.tgz",
+      "integrity": "sha1-8zshWfBTKj8xB6JywMz70a0peco=",
+      "dev": true
+    },
+    "arrify": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/arrify/-/arrify-1.0.1.tgz",
+      "integrity": "sha1-iYUI2iIm84DfkEcoRWhJwVAaSw0=",
+      "dev": true
+    },
+    "asap": {
+      "version": "2.0.6",
+      "resolved": "https://registry.npmjs.org/asap/-/asap-2.0.6.tgz",
+      "integrity": "sha1-5QNHYR1+aQlDIIu9r+vLwvuGbUY=",
+      "dev": true,
+      "optional": true
+    },
+    "asn1": {
+      "version": "0.2.3",
+      "resolved": "https://registry.npmjs.org/asn1/-/asn1-0.2.3.tgz",
+      "integrity": "sha1-2sh4dxPJlmhJ/IGAd36+nB3fO4Y="
+    },
+    "asn1.js": {
+      "version": "4.9.2",
+      "resolved": "https://registry.npmjs.org/asn1.js/-/asn1.js-4.9.2.tgz",
+      "integrity": "sha512-b/OsSjvWEo8Pi8H0zsDd2P6Uqo2TK2pH8gNLSJtNLM2Db0v2QaAZ0pBQJXVjAn4gBuugeVDr7s63ZogpUIwWDg==",
+      "dev": true,
+      "requires": {
+        "bn.js": "4.11.8",
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.0"
+      }
+    },
+    "assert": {
+      "version": "1.4.1",
+      "resolved": "https://registry.npmjs.org/assert/-/assert-1.4.1.tgz",
+      "integrity": "sha1-mZEtWRg2tab1s0XA8H7vwI/GXZE=",
+      "dev": true,
+      "requires": {
+        "util": "0.10.3"
+      }
+    },
+    "assert-plus": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/assert-plus/-/assert-plus-1.0.0.tgz",
+      "integrity": "sha1-8S4PPF13sLHN2RRpQuTpbB5N1SU="
+    },
+    "async": {
+      "version": "2.6.0",
+      "resolved": "https://registry.npmjs.org/async/-/async-2.6.0.tgz",
+      "integrity": "sha512-xAfGg1/NTLBBKlHFmnd7PlmUW9KhVQIUuSrYem9xzFUZy13ScvtyGGejaae9iAVRiRq9+Cx7DPFaAAhCpyxyPw==",
+      "dev": true,
+      "requires": {
+        "lodash": "4.17.4"
+      }
+    },
+    "async-each": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/async-each/-/async-each-1.0.1.tgz",
+      "integrity": "sha1-GdOGodntxufByF04iu28xW0zYC0=",
+      "dev": true
+    },
+    "async-foreach": {
+      "version": "0.1.3",
+      "resolved": "https://registry.npmjs.org/async-foreach/-/async-foreach-0.1.3.tgz",
+      "integrity": "sha1-NhIfhFwFeBct5Bmpfb6x0W7DRUI="
+    },
+    "asynckit": {
+      "version": "0.4.0",
+      "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz",
+      "integrity": "sha1-x57Zf380y48robyXkLzDZkdLS3k="
+    },
+    "autoprefixer": {
+      "version": "6.7.7",
+      "resolved": "https://registry.npmjs.org/autoprefixer/-/autoprefixer-6.7.7.tgz",
+      "integrity": "sha1-Hb0cg1ZY41zj+ZhAmdsAWFx4IBQ=",
+      "dev": true,
+      "requires": {
+        "browserslist": "1.7.7",
+        "caniuse-db": "1.0.30000783",
+        "normalize-range": "0.1.2",
+        "num2fraction": "1.2.2",
+        "postcss": "5.2.18",
+        "postcss-value-parser": "3.3.0"
+      }
+    },
+    "aws-sign2": {
+      "version": "0.7.0",
+      "resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz",
+      "integrity": "sha1-tG6JCTSpWR8tL2+G1+ap8bP+dqg="
+    },
+    "aws4": {
+      "version": "1.6.0",
+      "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.6.0.tgz",
+      "integrity": "sha1-g+9cqGCysy5KDe7e6MdxudtXRx4="
+    },
+    "babel-code-frame": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-code-frame/-/babel-code-frame-6.26.0.tgz",
+      "integrity": "sha1-Y/1D99weO7fONZR9uP42mj9Yx0s=",
+      "dev": true,
+      "requires": {
+        "chalk": "1.1.3",
+        "esutils": "2.0.2",
+        "js-tokens": "3.0.2"
+      }
+    },
+    "babel-generator": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-generator/-/babel-generator-6.26.0.tgz",
+      "integrity": "sha1-rBriAHC3n248odMmlhMFN3TyDcU=",
+      "dev": true,
+      "requires": {
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "detect-indent": "4.0.0",
+        "jsesc": "1.3.0",
+        "lodash": "4.17.4",
+        "source-map": "0.5.7",
+        "trim-right": "1.0.1"
+      },
+      "dependencies": {
+        "jsesc": {
+          "version": "1.3.0",
+          "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-1.3.0.tgz",
+          "integrity": "sha1-RsP+yMGJKxKwgz25vHYiF226s0s=",
+          "dev": true
+        },
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        }
+      }
+    },
+    "babel-messages": {
+      "version": "6.23.0",
+      "resolved": "https://registry.npmjs.org/babel-messages/-/babel-messages-6.23.0.tgz",
+      "integrity": "sha1-8830cDhYA1sqKVHG7F7fbGLyYw4=",
+      "dev": true,
+      "requires": {
+        "babel-runtime": "6.26.0"
+      }
+    },
+    "babel-runtime": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-runtime/-/babel-runtime-6.26.0.tgz",
+      "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=",
+      "dev": true,
+      "requires": {
+        "core-js": "2.5.3",
+        "regenerator-runtime": "0.11.1"
+      }
+    },
+    "babel-template": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-template/-/babel-template-6.26.0.tgz",
+      "integrity": "sha1-3gPi0WOWsGn0bdn/+FIfsaDjXgI=",
+      "dev": true,
+      "requires": {
+        "babel-runtime": "6.26.0",
+        "babel-traverse": "6.26.0",
+        "babel-types": "6.26.0",
+        "babylon": "6.18.0",
+        "lodash": "4.17.4"
+      }
+    },
+    "babel-traverse": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-traverse/-/babel-traverse-6.26.0.tgz",
+      "integrity": "sha1-RqnL1+3MYsjlwGTi0tjQ9ANXZu4=",
+      "dev": true,
+      "requires": {
+        "babel-code-frame": "6.26.0",
+        "babel-messages": "6.23.0",
+        "babel-runtime": "6.26.0",
+        "babel-types": "6.26.0",
+        "babylon": "6.18.0",
+        "debug": "2.6.9",
+        "globals": "9.18.0",
+        "invariant": "2.2.2",
+        "lodash": "4.17.4"
+      }
+    },
+    "babel-types": {
+      "version": "6.26.0",
+      "resolved": "https://registry.npmjs.org/babel-types/-/babel-types-6.26.0.tgz",
+      "integrity": "sha1-o7Bz+Uq0nrb6Vc1lInozQ4BjJJc=",
+      "dev": true,
+      "requires": {
+        "babel-runtime": "6.26.0",
+        "esutils": "2.0.2",
+        "lodash": "4.17.4",
+        "to-fast-properties": "1.0.3"
+      }
+    },
+    "babylon": {
+      "version": "6.18.0",
+      "resolved": "https://registry.npmjs.org/babylon/-/babylon-6.18.0.tgz",
+      "integrity": "sha512-q/UEjfGJ2Cm3oKV71DJz9d25TPnq5rhBVL2Q4fA5wcC3jcrdn7+SssEybFIxwAvvP+YCsCYNKughoF33GxgycQ==",
+      "dev": true
+    },
+    "backo2": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/backo2/-/backo2-1.0.2.tgz",
+      "integrity": "sha1-MasayLEpNjRj41s+u2n038+6eUc=",
+      "dev": true
+    },
+    "balanced-match": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.0.tgz",
+      "integrity": "sha1-ibTRmasr7kneFk6gK4nORi1xt2c="
+    },
+    "base64-arraybuffer": {
+      "version": "0.1.5",
+      "resolved": "https://registry.npmjs.org/base64-arraybuffer/-/base64-arraybuffer-0.1.5.tgz",
+      "integrity": "sha1-c5JncZI7Whl0etZmqlzUv5xunOg=",
+      "dev": true
+    },
+    "base64-js": {
+      "version": "1.2.1",
+      "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.2.1.tgz",
+      "integrity": "sha512-dwVUVIXsBZXwTuwnXI9RK8sBmgq09NDHzyR9SAph9eqk76gKK2JSQmZARC2zRC81JC2QTtxD0ARU5qTS25gIGw==",
+      "dev": true
+    },
+    "base64id": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/base64id/-/base64id-1.0.0.tgz",
+      "integrity": "sha1-R2iMuZu2gE8OBtPnY7HDLlfY5rY=",
+      "dev": true
+    },
+    "basic-auth": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/basic-auth/-/basic-auth-2.0.0.tgz",
+      "integrity": "sha1-AV2z81PgLlY3d1X5YnQuiYHnu7o=",
+      "requires": {
+        "safe-buffer": "5.1.1"
+      }
+    },
+    "batch": {
+      "version": "0.6.1",
+      "resolved": "https://registry.npmjs.org/batch/-/batch-0.6.1.tgz",
+      "integrity": "sha1-3DQxT05nkxgJP8dgJyUl+UvyXBY=",
+      "dev": true
+    },
+    "bcrypt-pbkdf": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.1.tgz",
+      "integrity": "sha1-Y7xdy2EzG5K8Bf1SiVPDNGKgb40=",
+      "optional": true,
+      "requires": {
+        "tweetnacl": "0.14.5"
+      }
+    },
+    "better-assert": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/better-assert/-/better-assert-1.0.2.tgz",
+      "integrity": "sha1-QIZrnhueC1W0gYlDEeaPr/rrxSI=",
+      "dev": true,
+      "requires": {
+        "callsite": "1.0.0"
+      }
+    },
+    "big.js": {
+      "version": "3.2.0",
+      "resolved": "https://registry.npmjs.org/big.js/-/big.js-3.2.0.tgz",
+      "integrity": "sha512-+hN/Zh2D08Mx65pZ/4g5bsmNiZUuChDiQfTUQ7qJr4/kuopCr88xZsAXv6mBoZEsUI4OuGHlX59qE94K2mMW8Q==",
+      "dev": true
+    },
+    "binary-extensions": {
+      "version": "1.11.0",
+      "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-1.11.0.tgz",
+      "integrity": "sha1-RqoXUftqL5PuXmibsQh9SxTGwgU=",
+      "dev": true
+    },
+    "blob": {
+      "version": "0.0.4",
+      "resolved": "https://registry.npmjs.org/blob/-/blob-0.0.4.tgz",
+      "integrity": "sha1-vPEwUspURj8w+fx+lbmkdjCpSSE=",
+      "dev": true
+    },
+    "block-stream": {
+      "version": "0.0.9",
+      "resolved": "https://registry.npmjs.org/block-stream/-/block-stream-0.0.9.tgz",
+      "integrity": "sha1-E+v+d4oDIFz+A3UUgeu0szAMEmo=",
+      "requires": {
+        "inherits": "2.0.3"
+      }
+    },
+    "blocking-proxy": {
+      "version": "0.0.5",
+      "resolved": "https://registry.npmjs.org/blocking-proxy/-/blocking-proxy-0.0.5.tgz",
+      "integrity": "sha1-RikF4Nz76pcPQao3Ij3anAexkSs=",
+      "dev": true,
+      "requires": {
+        "minimist": "1.2.0"
+      }
+    },
+    "bluebird": {
+      "version": "3.5.1",
+      "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.5.1.tgz",
+      "integrity": "sha512-MKiLiV+I1AA596t9w1sQJ8jkiSr5+ZKi0WKrYGUn6d1Fx+Ij4tIj+m2WMQSGczs5jZVxV339chE8iwk6F64wjA==",
+      "dev": true
+    },
+    "bn.js": {
+      "version": "4.11.8",
+      "resolved": "https://registry.npmjs.org/bn.js/-/bn.js-4.11.8.tgz",
+      "integrity": "sha512-ItfYfPLkWHUjckQCk8xC+LwxgK8NYcXywGigJgSwOP8Y2iyWT4f2vsZnoOXTTbo+o5yXmIUJ4gn5538SO5S3gA==",
+      "dev": true
+    },
+    "body-parser": {
+      "version": "1.18.2",
+      "resolved": "https://registry.npmjs.org/body-parser/-/body-parser-1.18.2.tgz",
+      "integrity": "sha1-h2eKGdhLR9hZuDGZvVm84iKxBFQ=",
+      "requires": {
+        "bytes": "3.0.0",
+        "content-type": "1.0.4",
+        "debug": "2.6.9",
+        "depd": "1.1.1",
+        "http-errors": "1.6.2",
+        "iconv-lite": "0.4.19",
+        "on-finished": "2.3.0",
+        "qs": "6.5.1",
+        "raw-body": "2.3.2",
+        "type-is": "1.6.15"
+      }
+    },
+    "bonjour": {
+      "version": "3.5.0",
+      "resolved": "https://registry.npmjs.org/bonjour/-/bonjour-3.5.0.tgz",
+      "integrity": "sha1-jokKGD2O6aI5OzhExpGkK897yfU=",
+      "dev": true,
+      "requires": {
+        "array-flatten": "2.1.1",
+        "deep-equal": "1.0.1",
+        "dns-equal": "1.0.0",
+        "dns-txt": "2.0.2",
+        "multicast-dns": "6.2.1",
+        "multicast-dns-service-types": "1.1.0"
+      },
+      "dependencies": {
+        "array-flatten": {
+          "version": "2.1.1",
+          "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-2.1.1.tgz",
+          "integrity": "sha1-Qmu52oQJDBg42BLIFQryCoMx4pY=",
+          "dev": true
+        }
+      }
+    },
+    "boolbase": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/boolbase/-/boolbase-1.0.0.tgz",
+      "integrity": "sha1-aN/1++YMUes3cl6p4+0xDcwed24=",
+      "dev": true
+    },
+    "boom": {
+      "version": "4.3.1",
+      "resolved": "https://registry.npmjs.org/boom/-/boom-4.3.1.tgz",
+      "integrity": "sha1-T4owBctKfjiJ90kDD9JbluAdLjE=",
+      "requires": {
+        "hoek": "4.2.0"
+      }
+    },
+    "brace-expansion": {
+      "version": "1.1.8",
+      "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.8.tgz",
+      "integrity": "sha1-wHshHHyVLsH479Uad+8NHTmQopI=",
+      "requires": {
+        "balanced-match": "1.0.0",
+        "concat-map": "0.0.1"
+      }
+    },
+    "braces": {
+      "version": "1.8.5",
+      "resolved": "https://registry.npmjs.org/braces/-/braces-1.8.5.tgz",
+      "integrity": "sha1-uneWLhLf+WnWt2cR6RS3N4V79qc=",
+      "dev": true,
+      "requires": {
+        "expand-range": "1.8.2",
+        "preserve": "0.2.0",
+        "repeat-element": "1.1.2"
+      }
+    },
+    "brorand": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/brorand/-/brorand-1.1.0.tgz",
+      "integrity": "sha1-EsJe/kCkXjwyPrhnWgoM5XsiNx8=",
+      "dev": true
+    },
+    "browserify-aes": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/browserify-aes/-/browserify-aes-1.1.1.tgz",
+      "integrity": "sha512-UGnTYAnB2a3YuYKIRy1/4FB2HdM866E0qC46JXvVTYKlBlZlnvfpSfY6OKfXZAkv70eJ2a1SqzpAo5CRhZGDFg==",
+      "dev": true,
+      "requires": {
+        "buffer-xor": "1.0.3",
+        "cipher-base": "1.0.4",
+        "create-hash": "1.1.3",
+        "evp_bytestokey": "1.0.3",
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.1"
+      }
+    },
+    "browserify-cipher": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/browserify-cipher/-/browserify-cipher-1.0.0.tgz",
+      "integrity": "sha1-mYgkSHS/XtTijalWZtzWasj8Njo=",
+      "dev": true,
+      "requires": {
+        "browserify-aes": "1.1.1",
+        "browserify-des": "1.0.0",
+        "evp_bytestokey": "1.0.3"
+      }
+    },
+    "browserify-des": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/browserify-des/-/browserify-des-1.0.0.tgz",
+      "integrity": "sha1-2qJ3cXRwki7S/hhZQRihdUOXId0=",
+      "dev": true,
+      "requires": {
+        "cipher-base": "1.0.4",
+        "des.js": "1.0.0",
+        "inherits": "2.0.3"
+      }
+    },
+    "browserify-rsa": {
+      "version": "4.0.1",
+      "resolved": "https://registry.npmjs.org/browserify-rsa/-/browserify-rsa-4.0.1.tgz",
+      "integrity": "sha1-IeCr+vbyApzy+vsTNWenAdQTVSQ=",
+      "dev": true,
+      "requires": {
+        "bn.js": "4.11.8",
+        "randombytes": "2.0.5"
+      }
+    },
+    "browserify-sign": {
+      "version": "4.0.4",
+      "resolved": "https://registry.npmjs.org/browserify-sign/-/browserify-sign-4.0.4.tgz",
+      "integrity": "sha1-qk62jl17ZYuqa/alfmMMvXqT0pg=",
+      "dev": true,
+      "requires": {
+        "bn.js": "4.11.8",
+        "browserify-rsa": "4.0.1",
+        "create-hash": "1.1.3",
+        "create-hmac": "1.1.6",
+        "elliptic": "6.4.0",
+        "inherits": "2.0.3",
+        "parse-asn1": "5.1.0"
+      }
+    },
+    "browserify-zlib": {
+      "version": "0.2.0",
+      "resolved": "https://registry.npmjs.org/browserify-zlib/-/browserify-zlib-0.2.0.tgz",
+      "integrity": "sha512-Z942RysHXmJrhqk88FmKBVq/v5tqmSkDz7p54G/MGyjMnCFFnC79XWNbg+Vta8W6Wb2qtSZTSxIGkJrRpCFEiA==",
+      "dev": true,
+      "requires": {
+        "pako": "1.0.6"
+      }
+    },
+    "browserslist": {
+      "version": "1.7.7",
+      "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-1.7.7.tgz",
+      "integrity": "sha1-C9dnBCWL6CmyOYu1Dkti0aFmsLk=",
+      "dev": true,
+      "requires": {
+        "caniuse-db": "1.0.30000783",
+        "electron-to-chromium": "1.3.28"
+      }
+    },
+    "buffer": {
+      "version": "4.9.1",
+      "resolved": "https://registry.npmjs.org/buffer/-/buffer-4.9.1.tgz",
+      "integrity": "sha1-bRu2AbB6TvztlwlBMgkwJ8lbwpg=",
+      "dev": true,
+      "requires": {
+        "base64-js": "1.2.1",
+        "ieee754": "1.1.8",
+        "isarray": "1.0.0"
+      }
+    },
+    "buffer-indexof": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/buffer-indexof/-/buffer-indexof-1.1.1.tgz",
+      "integrity": "sha512-4/rOEg86jivtPTeOUUT61jJO1Ya1TrR/OkqCSZDyq84WJh3LuuiphBYJN+fm5xufIk4XAFcEwte/8WzC8If/1g==",
+      "dev": true
+    },
+    "buffer-xor": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/buffer-xor/-/buffer-xor-1.0.3.tgz",
+      "integrity": "sha1-JuYe0UIvtw3ULm42cp7VHYVf6Nk=",
+      "dev": true
+    },
+    "builtin-modules": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/builtin-modules/-/builtin-modules-1.1.1.tgz",
+      "integrity": "sha1-Jw8HbFpywC9bZaR9+Uxf46J4iS8="
+    },
+    "builtin-status-codes": {
+      "version": "3.0.0",
+      "resolved": "https://registry.npmjs.org/builtin-status-codes/-/builtin-status-codes-3.0.0.tgz",
+      "integrity": "sha1-hZgoeOIbmOHGZCXgPQF0eI9Wnug=",
+      "dev": true
+    },
+    "bytes": {
+      "version": "3.0.0",
+      "resolved": "https://registry.npmjs.org/bytes/-/bytes-3.0.0.tgz",
+      "integrity": "sha1-0ygVQE1olpn4Wk6k+odV3ROpYEg="
+    },
+    "cacache": {
+      "version": "10.0.1",
+      "resolved": "https://registry.npmjs.org/cacache/-/cacache-10.0.1.tgz",
+      "integrity": "sha512-dRHYcs9LvG9cHgdPzjiI+/eS7e1xRhULrcyOx04RZQsszNJXU2SL9CyG60yLnge282Qq5nwTv+ieK2fH+WPZmA==",
+      "dev": true,
+      "requires": {
+        "bluebird": "3.5.1",
+        "chownr": "1.0.1",
+        "glob": "7.1.2",
+        "graceful-fs": "4.1.11",
+        "lru-cache": "4.1.1",
+        "mississippi": "1.3.0",
+        "mkdirp": "0.5.1",
+        "move-concurrently": "1.0.1",
+        "promise-inflight": "1.0.1",
+        "rimraf": "2.6.2",
+        "ssri": "5.0.0",
+        "unique-filename": "1.1.0",
+        "y18n": "3.2.1"
+      }
+    },
+    "callsite": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/callsite/-/callsite-1.0.0.tgz",
+      "integrity": "sha1-KAOY5dZkvXQDi28JBRU+borxvCA=",
+      "dev": true
+    },
+    "camel-case": {
+      "version": "3.0.0",
+      "resolved": "https://registry.npmjs.org/camel-case/-/camel-case-3.0.0.tgz",
+      "integrity": "sha1-yjw2iKTpzzpM2nd9xNy8cTJJz3M=",
+      "dev": true,
+      "requires": {
+        "no-case": "2.3.2",
+        "upper-case": "1.1.3"
+      }
+    },
+    "camelcase": {
+      "version": "2.1.1",
+      "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-2.1.1.tgz",
+      "integrity": "sha1-fB0W1nmhu+WcoCys7PsBHiAfWh8="
+    },
+    "camelcase-keys": {
+      "version": "2.1.0",
+      "resolved": "https://registry.npmjs.org/camelcase-keys/-/camelcase-keys-2.1.0.tgz",
+      "integrity": "sha1-MIvur/3ygRkFHvodkyITyRuPkuc=",
+      "requires": {
+        "camelcase": "2.1.1",
+        "map-obj": "1.0.1"
+      }
+    },
+    "caniuse-api": {
+      "version": "1.6.1",
+      "resolved": "https://registry.npmjs.org/caniuse-api/-/caniuse-api-1.6.1.tgz",
+      "integrity": "sha1-tTTnxzTE+B7F++isoq0kNUuWLGw=",
+      "dev": true,
+      "requires": {
+        "browserslist": "1.7.7",
+        "caniuse-db": "1.0.30000783",
+        "lodash.memoize": "4.1.2",
+        "lodash.uniq": "4.5.0"
+      }
+    },
+    "caniuse-db": {
+      "version": "1.0.30000783",
+      "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000783.tgz",
+      "integrity": "sha1-FrMNRyZqT1FcxprgMWtnDJYDzb4=",
+      "dev": true
+    },
+    "caseless": {
+      "version": "0.12.0",
+      "resolved": "https://registry.npmjs.org/caseless/-/caseless-0.12.0.tgz",
+      "integrity": "sha1-G2gcIf+EAzyCZUMJBolCDRhxUdw="
+    },
+    "center-align": {
+      "version": "0.1.3",
+      "resolved": "https://registry.npmjs.org/center-align/-/center-align-0.1.3.tgz",
+      "integrity": "sha1-qg0yYptu6XIgBBHL1EYckHvCt60=",
+      "dev": true,
+      "requires": {
+        "align-text": "0.1.4",
+        "lazy-cache": "1.0.4"
+      },
+      "dependencies": {
+        "lazy-cache": {
+          "version": "1.0.4",
+          "resolved": "https://registry.npmjs.org/lazy-cache/-/lazy-cache-1.0.4.tgz",
+          "integrity": "sha1-odePw6UEdMuAhF07O24dpJpEbo4=",
+          "dev": true
+        }
+      }
+    },
+    "chalk": {
+      "version": "1.1.3",
+      "resolved": "https://registry.npmjs.org/chalk/-/chalk-1.1.3.tgz",
+      "integrity": "sha1-qBFcVeSnAv5NFQq9OHKCKn4J/Jg=",
+      "requires": {
+        "ansi-styles": "2.2.1",
+        "escape-string-regexp": "1.0.5",
+        "has-ansi": "2.0.0",
+        "strip-ansi": "3.0.1",
+        "supports-color": "2.0.0"
+      }
+    },
+    "chokidar": {
+      "version": "1.7.0",
+      "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-1.7.0.tgz",
+      "integrity": "sha1-eY5ol3gVHIB2tLNg5e3SjNortGg=",
+      "dev": true,
+      "requires": {
+        "anymatch": "1.3.2",
+        "async-each": "1.0.1",
+        "fsevents": "1.1.3",
+        "glob-parent": "2.0.0",
+        "inherits": "2.0.3",
+        "is-binary-path": "1.0.1",
+        "is-glob": "2.0.1",
+        "path-is-absolute": "1.0.1",
+        "readdirp": "2.1.0"
+      },
+      "dependencies": {
+        "is-extglob": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/is-extglob/-/is-extglob-1.0.0.tgz",
+          "integrity": "sha1-rEaBd8SUNAWgkvyPKXYMb/xiBsA=",
+          "dev": true
+        },
+        "is-glob": {
+          "version": "2.0.1",
+          "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-2.0.1.tgz",
+          "integrity": "sha1-0Jb5JqPe1WAPP9/ZEZjLCIjC2GM=",
+          "dev": true,
+          "requires": {
+            "is-extglob": "1.0.0"
+          }
+        }
+      }
+    },
+    "chownr": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/chownr/-/chownr-1.0.1.tgz",
+      "integrity": "sha1-4qdQQqlVGQi+vSW4Uj1fl2nXkYE=",
+      "dev": true
+    },
+    "cipher-base": {
+      "version": "1.0.4",
+      "resolved": "https://registry.npmjs.org/cipher-base/-/cipher-base-1.0.4.tgz",
+      "integrity": "sha512-Kkht5ye6ZGmwv40uUDZztayT2ThLQGfnj/T71N/XzeZeo3nf8foyW7zGTsPYkEya3m5f3cAypH+qe7YOrM1U2Q==",
+      "dev": true,
+      "requires": {
+        "inherits": "2.0.3",
+        "safe-buffer": "5.1.1"
+      }
+    },
+    "circular-dependency-plugin": {
+      "version": "4.3.0",
+      "resolved": "https://registry.npmjs.org/circular-dependency-plugin/-/circular-dependency-plugin-4.3.0.tgz",
+      "integrity": "sha512-L3W9L1S0wC64rq+QSaZzmWnJW7cVBgimxI2lNEFEX5biwlRG8EHRM68JFi+CX5ZkCGUWJHIpnhdVs181Zlq3wA==",
+      "dev": true
+    },
+    "clap": {
+      "version": "1.2.3",
+      "resolved": "https://registry.npmjs.org/clap/-/clap-1.2.3.tgz",
+      "integrity": "sha512-4CoL/A3hf90V3VIEjeuhSvlGFEHKzOz+Wfc2IVZc+FaUgU0ZQafJTP49fvnULipOPcAfqhyI2duwQyns6xqjYA==",
+      "dev": true,
+      "requires": {
+        "chalk": "1.1.3"
+      }
+    },
+    "clean-css": {
+      "version": "4.1.9",
+      "resolved": "https://registry.npmjs.org/clean-css/-/clean-css-4.1.9.tgz",
+      "integrity": "sha1-Nc7ornaHpJuYA09w3gDE7dOCYwE=",
+      "dev": true,
+      "requires": {
+        "source-map": "0.5.7"
+      },
+      "dependencies": {
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        }
+      }
+    },
+    "cliui": {
+      "version": "3.2.0",
+      "resolved": "https://registry.npmjs.org/cliui/-/cliui-3.2.0.tgz",
+      "integrity": "sha1-EgYBU3qRbSmUD5NNo7SNWFo5IT0=",
+      "requires": {
+        "string-width": "1.0.2",
+        "strip-ansi": "3.0.1",
+        "wrap-ansi": "2.1.0"
+      }
+    },
+    "clone": {
+      "version": "1.0.3",
+      "resolved": "https://registry.npmjs.org/clone/-/clone-1.0.3.tgz",
+      "integrity": "sha1-KY1+IjFmD0DAA8LtMUDezz9TCF8=",
+      "dev": true
+    },
+    "clone-deep": {
+      "version": "0.3.0",
+      "resolved": "https://registry.npmjs.org/clone-deep/-/clone-deep-0.3.0.tgz",
+      "integrity": "sha1-NIxhrpzb4O3+BT2R/0zFIdeQ7eg=",
+      "dev": true,
+      "requires": {
+        "for-own": "1.0.0",
+        "is-plain-object": "2.0.4",
+        "kind-of": "3.2.2",
+        "shallow-clone": "0.1.2"
+      }
+    },
+    "co": {
+      "version": "4.6.0",
+      "resolved": "https://registry.npmjs.org/co/-/co-4.6.0.tgz",
+      "integrity": "sha1-bqa989hTrlTMuOR7+gvz+QMfsYQ="
+    },
+    "coa": {
+      "version": "1.0.4",
+      "resolved": "https://registry.npmjs.org/coa/-/coa-1.0.4.tgz",
+      "integrity": "sha1-qe8VNmDWqGqL3sAomlxoTSF0Mv0=",
+      "dev": true,
+      "requires": {
+        "q": "1.5.1"
+      }
+    },
+    "code-point-at": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/code-point-at/-/code-point-at-1.1.0.tgz",
+      "integrity": "sha1-DQcLTQQ6W+ozovGkDi7bPZpMz3c="
+    },
+    "codelyzer": {
+      "version": "4.0.2",
+      "resolved": "https://registry.npmjs.org/codelyzer/-/codelyzer-4.0.2.tgz",
+      "integrity": "sha512-nYwOr49+IV09e7C4aXkVALRz0+XpHqZiUUcxHuDZH4xP1FBcHINyr3qvVhv5Gfm7XRmoLx32tsIhrQhW/gBcog==",
+      "dev": true,
+      "requires": {
+        "app-root-path": "2.0.1",
+        "css-selector-tokenizer": "0.7.0",
+        "cssauron": "1.4.0",
+        "semver-dsl": "1.0.1",
+        "source-map": "0.5.7",
+        "sprintf-js": "1.0.3"
+      },
+      "dependencies": {
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        }
+      }
+    },
+    "color": {
+      "version": "0.11.4",
+      "resolved": "https://registry.npmjs.org/color/-/color-0.11.4.tgz",
+      "integrity": "sha1-bXtcdPtl6EHNSHkq0e1eB7kE12Q=",
+      "dev": true,
+      "requires": {
+        "clone": "1.0.3",
+        "color-convert": "1.9.1",
+        "color-string": "0.3.0"
+      }
+    },
+    "color-convert": {
+      "version": "1.9.1",
+      "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-1.9.1.tgz",
+      "integrity": "sha512-mjGanIiwQJskCC18rPR6OmrZ6fm2Lc7PeGFYwCmy5J34wC6F1PzdGL6xeMfmgicfYcNLGuVFA3WzXtIDCQSZxQ==",
+      "dev": true,
+      "requires": {
+        "color-name": "1.1.3"
+      }
+    },
+    "color-name": {
+      "version": "1.1.3",
+      "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.3.tgz",
+      "integrity": "sha1-p9BVi9icQveV3UIyj3QIMcpTvCU=",
+      "dev": true
+    },
+    "color-string": {
+      "version": "0.3.0",
+      "resolved": "https://registry.npmjs.org/color-string/-/color-string-0.3.0.tgz",
+      "integrity": "sha1-J9RvtnAlxcL6JZk7+/V55HhBuZE=",
+      "dev": true,
+      "requires": {
+        "color-name": "1.1.3"
+      }
+    },
+    "colormin": {
+      "version": "1.1.2",
+      "resolved": "https://registry.npmjs.org/colormin/-/colormin-1.1.2.tgz",
+      "integrity": "sha1-6i90IKcrlogaOKrlnsEkpvcpgTM=",
+      "dev": true,
+      "requires": {
+        "color": "0.11.4",
+        "css-color-names": "0.0.4",
+        "has": "1.0.1"
+      }
+    },
+    "colors": {
+      "version": "1.1.2",
+      "resolved": "https://registry.npmjs.org/colors/-/colors-1.1.2.tgz",
+      "integrity": "sha1-FopHAXVran9RoSzgyXv6KMCE7WM=",
+      "dev": true
+    },
+    "combine-lists": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/combine-lists/-/combine-lists-1.0.1.tgz",
+      "integrity": "sha1-RYwH4J4NkA/Ci3Cj/sLazR0st/Y=",
+      "dev": true,
+      "requires": {
+        "lodash": "4.17.4"
+      }
+    },
+    "combined-stream": {
+      "version": "1.0.5",
+      "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.5.tgz",
+      "integrity": "sha1-k4NwpXtKUd6ix3wV1cX9+JUWQAk=",
+      "requires": {
+        "delayed-stream": "1.0.0"
+      }
+    },
+    "commander": {
+      "version": "2.12.2",
+      "resolved": "https://registry.npmjs.org/commander/-/commander-2.12.2.tgz",
+      "integrity": "sha512-BFnaq5ZOGcDN7FlrtBT4xxkgIToalIIxwjxLWVJ8bGTpe1LroqMiqQXdA7ygc7CRvaYS+9zfPGFnJqFSayx+AA==",
+      "dev": true
+    },
+    "common-tags": {
+      "version": "1.5.1",
+      "resolved": "https://registry.npmjs.org/common-tags/-/common-tags-1.5.1.tgz",
+      "integrity": "sha512-NrUYGY5TApAk9KB+IZXkR3GR4tA3g26HDsoiGt4kCMHZ727gOGkC+UNfq0Z22jE15bLkc/6RV5Jw1RBW6Usg6A==",
+      "dev": true,
+      "requires": {
+        "babel-runtime": "6.26.0"
+      }
+    },
+    "commondir": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/commondir/-/commondir-1.0.1.tgz",
+      "integrity": "sha1-3dgA2gxmEnOTzKWVDqloo6rxJTs=",
+      "dev": true
+    },
+    "component-bind": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/component-bind/-/component-bind-1.0.0.tgz",
+      "integrity": "sha1-AMYIq33Nk4l8AAllGx06jh5zu9E=",
+      "dev": true
+    },
+    "component-emitter": {
+      "version": "1.1.2",
+      "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.1.2.tgz",
+      "integrity": "sha1-KWWU8nU9qmOZbSrwjRWpURbJrsM=",
+      "dev": true
+    },
+    "component-inherit": {
+      "version": "0.0.3",
+      "resolved": "https://registry.npmjs.org/component-inherit/-/component-inherit-0.0.3.tgz",
+      "integrity": "sha1-ZF/ErfWLcrZJ1crmUTVhnbJv8UM=",
+      "dev": true
+    },
+    "compressible": {
+      "version": "2.0.12",
+      "resolved": "https://registry.npmjs.org/compressible/-/compressible-2.0.12.tgz",
+      "integrity": "sha1-xZpcmdt2dn6YdlAOJx72OzSTvWY=",
+      "dev": true,
+      "requires": {
+        "mime-db": "1.30.0"
+      }
+    },
+    "compression": {
+      "version": "1.7.1",
+      "resolved": "https://registry.npmjs.org/compression/-/compression-1.7.1.tgz",
+      "integrity": "sha1-7/JgPvwuIs+G810uuTWJ+YdTc9s=",
+      "dev": true,
+      "requires": {
+        "accepts": "1.3.4",
+        "bytes": "3.0.0",
+        "compressible": "2.0.12",
+        "debug": "2.6.9",
+        "on-headers": "1.0.1",
+        "safe-buffer": "5.1.1",
+        "vary": "1.1.2"
+      }
+    },
+    "concat-map": {
+      "version": "0.0.1",
+      "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz",
+      "integrity": "sha1-2Klr13/Wjfd5OnMDajug1UBdR3s="
+    },
+    "concat-stream": {
+      "version": "1.6.0",
+      "resolved": "https://registry.npmjs.org/concat-stream/-/concat-stream-1.6.0.tgz",
+      "integrity": "sha1-CqxmL9Ur54lk1VMvaUeE5wEQrPc=",
+      "dev": true,
+      "requires": {
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.3",
+        "typedarray": "0.0.6"
+      }
+    },
+    "concurrently": {
+      "version": "3.5.1",
+      "resolved": "https://registry.npmjs.org/concurrently/-/concurrently-3.5.1.tgz",
+      "integrity": "sha512-689HrwGw8Rbk1xtV9C4dY6TPJAvIYZbRbnKSAtfJ7tHqICFGoZ0PCWYjxfmerRyxBG0o3sbG3pe7N8vqPwIHuQ==",
+      "dev": true,
+      "requires": {
+        "chalk": "0.5.1",
+        "commander": "2.6.0",
+        "date-fns": "1.29.0",
+        "lodash": "4.17.4",
+        "rx": "2.3.24",
+        "spawn-command": "0.0.2-1",
+        "supports-color": "3.2.3",
+        "tree-kill": "1.2.0"
+      },
+      "dependencies": {
+        "ansi-regex": {
+          "version": "0.2.1",
+          "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-0.2.1.tgz",
+          "integrity": "sha1-DY6UaWej2BQ/k+JOKYUl/BsiNfk=",
+          "dev": true
+        },
+        "ansi-styles": {
+          "version": "1.1.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-1.1.0.tgz",
+          "integrity": "sha1-6uy/Zs1waIJ2Cy9GkVgrj1XXp94=",
+          "dev": true
+        },
+        "chalk": {
+          "version": "0.5.1",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-0.5.1.tgz",
+          "integrity": "sha1-Zjs6ZItotV0EaQ1JFnqoN4WPIXQ=",
+          "dev": true,
+          "requires": {
+            "ansi-styles": "1.1.0",
+            "escape-string-regexp": "1.0.5",
+            "has-ansi": "0.1.0",
+            "strip-ansi": "0.3.0",
+            "supports-color": "0.2.0"
+          },
+          "dependencies": {
+            "supports-color": {
+              "version": "0.2.0",
+              "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-0.2.0.tgz",
+              "integrity": "sha1-2S3iaU6z9nMjlz1649i1W0wiGQo=",
+              "dev": true
+            }
+          }
+        },
+        "commander": {
+          "version": "2.6.0",
+          "resolved": "https://registry.npmjs.org/commander/-/commander-2.6.0.tgz",
+          "integrity": "sha1-nfflL7Kgyw+4kFjugMMQQiXzfh0=",
+          "dev": true
+        },
+        "has-ansi": {
+          "version": "0.1.0",
+          "resolved": "https://registry.npmjs.org/has-ansi/-/has-ansi-0.1.0.tgz",
+          "integrity": "sha1-hPJlqujA5qiKEtcCKJS3VoiUxi4=",
+          "dev": true,
+          "requires": {
+            "ansi-regex": "0.2.1"
+          }
+        },
+        "has-flag": {
+          "version": "1.0.0",
+          "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-1.0.0.tgz",
+          "integrity": "sha1-nZ55MWXOAXoA8AQYxD+UKnsdEfo=",
+          "dev": true
+        },
+        "strip-ansi": {
+          "version": "0.3.0",
+          "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-0.3.0.tgz",
+          "integrity": "sha1-JfSOoiynkYfzF0pNuHWTR7sSYiA=",
+          "dev": true,
+          "requires": {
+            "ansi-regex": "0.2.1"
+          }
+        },
+        "supports-color": {
+          "version": "3.2.3",
+          "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-3.2.3.tgz",
+          "integrity": "sha1-ZawFBLOVQXHYpklGsq48u4pfVPY=",
+          "dev": true,
+          "requires": {
+            "has-flag": "1.0.0"
+          }
+        }
+      }
+    },
+    "configstore": {
+      "version": "1.4.0",
+      "resolved": "https://registry.npmjs.org/configstore/-/configstore-1.4.0.tgz",
+      "integrity": "sha1-w1eB0FAdJowlxUuLF/YkDopPsCE=",
+      "dev": true,
+      "requires": {
+        "graceful-fs": "4.1.11",
+        "mkdirp": "0.5.1",
+        "object-assign": "4.1.1",
+        "os-tmpdir": "1.0.2",
+        "osenv": "0.1.4",
+        "uuid": "2.0.3",
+        "write-file-atomic": "1.3.4",
+        "xdg-basedir": "2.0.0"
+      },
+      "dependencies": {
+        "uuid": {
+          "version": "2.0.3",
+          "resolved": "https://registry.npmjs.org/uuid/-/uuid-2.0.3.tgz",
+          "integrity": "sha1-Z+LoY3lyFVMN/zGOW/nc6/1Hsho=",
+          "dev": true
+        }
+      }
+    },
+    "connect": {
+      "version": "3.6.5",
+      "resolved": "https://registry.npmjs.org/connect/-/connect-3.6.5.tgz",
+      "integrity": "sha1-+43ee6B2OHfQ7J352sC0tA5yx9o=",
+      "dev": true,
+      "requires": {
+        "debug": "2.6.9",
+        "finalhandler": "1.0.6",
+        "parseurl": "1.3.2",
+        "utils-merge": "1.0.1"
+      },
+      "dependencies": {
+        "finalhandler": {
+          "version": "1.0.6",
+          "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.0.6.tgz",
+          "integrity": "sha1-AHrqM9Gk0+QgF/YkhIrVjSEvgU8=",
+          "dev": true,
+          "requires": {
+            "debug": "2.6.9",
+            "encodeurl": "1.0.1",
+            "escape-html": "1.0.3",
+            "on-finished": "2.3.0",
+            "parseurl": "1.3.2",
+            "statuses": "1.3.1",
+            "unpipe": "1.0.0"
+          }
+        },
+        "statuses": {
+          "version": "1.3.1",
+          "resolved": "https://registry.npmjs.org/statuses/-/statuses-1.3.1.tgz",
+          "integrity": "sha1-+vUbnrdKrvOzrPStX2Gr8ky3uT4=",
+          "dev": true
+        }
+      }
+    },
+    "connect-history-api-fallback": {
+      "version": "1.5.0",
+      "resolved": "https://registry.npmjs.org/connect-history-api-fallback/-/connect-history-api-fallback-1.5.0.tgz",
+      "integrity": "sha1-sGhzk0vF40T+9hGhlqb6rgruAVo=",
+      "dev": true
+    },
+    "console-browserify": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/console-browserify/-/console-browserify-1.1.0.tgz",
+      "integrity": "sha1-8CQcRXMKn8YyOyBtvzjtx0HQuxA=",
+      "dev": true,
+      "requires": {
+        "date-now": "0.1.4"
+      }
+    },
+    "console-control-strings": {
+      "version": "1.1.0",
+      "resolved": "https://registry.npmjs.org/console-control-strings/-/console-control-strings-1.1.0.tgz",
+      "integrity": "sha1-PXz0Rk22RG6mRL9LOVB/mFEAjo4="
+    },
+    "constants-browserify": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/constants-browserify/-/constants-browserify-1.0.0.tgz",
+      "integrity": "sha1-wguW2MYXdIqvHBYCF2DNJ/y4y3U=",
+      "dev": true
+    },
+    "content-disposition": {
+      "version": "0.5.2",
+      "resolved": "https://registry.npmjs.org/content-disposition/-/content-disposition-0.5.2.tgz",
+      "integrity": "sha1-DPaLud318r55YcOoUXjLhdunjLQ="
+    },
+    "content-type": {
+      "version": "1.0.4",
+      "resolved": "https://registry.npmjs.org/content-type/-/content-type-1.0.4.tgz",
+      "integrity": "sha512-hIP3EEPs8tB9AT1L+NUqtwOAps4mk2Zob89MWXMHjHWg9milF/j4osnnQLXBCBFBk/tvIG/tUc9mOUJiPBhPXA=="
+    },
+    "convert-source-map": {
+      "version": "1.5.1",
+      "resolved": "https://registry.npmjs.org/convert-source-map/-/convert-source-map-1.5.1.tgz",
+      "integrity": "sha1-uCeAl7m8IpNl3lxiz1/K7YtVmeU=",
+      "dev": true
+    },
+    "cookie": {
+      "version": "0.3.1",
+      "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.3.1.tgz",
+      "integrity": "sha1-5+Ch+e9DtMi6klxcWpboBtFoc7s="
+    },
+    "cookie-signature": {
+      "version": "1.0.6",
+      "resolved": "https://registry.npmjs.org/cookie-signature/-/cookie-signature-1.0.6.tgz",
+      "integrity": "sha1-4wOogrNCzD7oylE6eZmXNNqzriw="
+    },
+    "copy-concurrently": {
+      "version": "1.0.5",
+      "resolved": "https://registry.npmjs.org/copy-concurrently/-/copy-concurrently-1.0.5.tgz",
+      "integrity": "sha512-f2domd9fsVDFtaFcbaRZuYXwtdmnzqbADSwhSWYxYB/Q8zsdUUFMXVRwXGDMWmbEzAn1kdRrtI1T/KTFOL4X2A==",
+      "dev": true,
+      "requires": {
+        "aproba": "1.2.0",
+        "fs-write-stream-atomic": "1.0.10",
+        "iferr": "0.1.5",
+        "mkdirp": "0.5.1",
+        "rimraf": "2.6.2",
+        "run-queue": "1.0.3"
+      }
+    },
+    "copy-webpack-plugin": {
+      "version": "4.3.0",
+      "resolved": "https://registry.npmjs.org/copy-webpack-plugin/-/copy-webpack-plugin-4.3.0.tgz",
+      "integrity": "sha512-5o1/xyWm8OYDmLFKAWMuPU3A/jZ4Z6kZSZGh36KD2XmtxnRa8lQyLx7bCNQm08BPaR/oqUdtJOr9jWfnYINp9g==",
+      "dev": true,
+      "requires": {
+        "cacache": "10.0.1",
+        "find-cache-dir": "1.0.0",
+        "globby": "7.1.1",
+        "is-glob": "4.0.0",
+        "loader-utils": "0.2.17",
+        "lodash": "4.17.4",
+        "minimatch": "3.0.4",
+        "p-limit": "1.1.0",
+        "pify": "3.0.0",
+        "serialize-javascript": "1.4.0"
+      },
+      "dependencies": {
+        "loader-utils": {
+          "version": "0.2.17",
+          "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-0.2.17.tgz",
+          "integrity": "sha1-+G5jdNQyBabmxg6RlvF8Apm/s0g=",
+          "dev": true,
+          "requires": {
+            "big.js": "3.2.0",
+            "emojis-list": "2.1.0",
+            "json5": "0.5.1",
+            "object-assign": "4.1.1"
+          }
+        },
+        "pify": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/pify/-/pify-3.0.0.tgz",
+          "integrity": "sha1-5aSs0sEB/fPZpNB/DbxNtJ3SgXY=",
+          "dev": true
+        }
+      }
+    },
+    "core-js": {
+      "version": "2.5.3",
+      "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.5.3.tgz",
+      "integrity": "sha1-isw4NFgk8W2DZbfJtCWRaOjtYD4="
+    },
+    "core-object": {
+      "version": "3.1.5",
+      "resolved": "https://registry.npmjs.org/core-object/-/core-object-3.1.5.tgz",
+      "integrity": "sha512-sA2/4+/PZ/KV6CKgjrVrrUVBKCkdDO02CUlQ0YKTQoYUwPYNOtOAcWlbYhd5v/1JqYaA6oZ4sDlOU4ppVw6Wbg==",
+      "dev": true,
+      "requires": {
+        "chalk": "2.3.0"
+      },
+      "dependencies": {
+        "ansi-styles": {
+          "version": "3.2.0",
+          "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-3.2.0.tgz",
+          "integrity": "sha512-NnSOmMEYtVR2JVMIGTzynRkkaxtiq1xnFBcdQD/DnNCYPoEPsVJhM98BDyaoNOQIi7p4okdi3E27eN7GQbsUug==",
+          "dev": true,
+          "requires": {
+            "color-convert": "1.9.1"
+          }
+        },
+        "chalk": {
+          "version": "2.3.0",
+          "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.0.tgz",
+          "integrity": "sha512-Az5zJR2CBujap2rqXGaJKaPHyJ0IrUimvYNX+ncCy8PJP4ltOGTrHUIo097ZaL2zMeKYpiCdqDvS6zdrTFok3Q==",
+          "dev": true,
+          "requires": {
+            "ansi-styles": "3.2.0",
+            "escape-string-regexp": "1.0.5",
+            "supports-color": "4.5.0"
+          }
+        },
+        "supports-color": {
+          "version": "4.5.0",
+          "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-4.5.0.tgz",
+          "integrity": "sha1-vnoN5ITexcXN34s9WRJQRJEvY1s=",
+          "dev": true,
+          "requires": {
+            "has-flag": "2.0.0"
+          }
+        }
+      }
+    },
+    "core-util-is": {
+      "version": "1.0.2",
+      "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.2.tgz",
+      "integrity": "sha1-tf1UIgqivFq1eqtxQMlAdUUDwac="
+    },
+    "cosmiconfig": {
+      "version": "2.2.2",
+      "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-2.2.2.tgz",
+      "integrity": "sha512-GiNXLwAFPYHy25XmTPpafYvn3CLAkJ8FLsscq78MQd1Kh0OU6Yzhn4eV2MVF4G9WEQZoWEGltatdR+ntGPMl5A==",
+      "dev": true,
+      "requires": {
+        "is-directory": "0.3.1",
+        "js-yaml": "3.7.0",
+        "minimist": "1.2.0",
+        "object-assign": "4.1.1",
+        "os-homedir": "1.0.2",
+        "parse-json": "2.2.0",
+        "require-from-string": "1.2.1"
+      }
+    },
+    "crc": {
+      "version": "3.4.4",
+      "resolved": "https://registry.npmjs.org/crc/-/crc-3.4.4.tgz",
+      "integrity": "sha1-naHpgOO9RPxck79as9ozeNheRms="
+    },
+    "create-ecdh": {
+      "version": "4.0.0",
+      "resolved": "https://registry.npmjs.org/create-ecdh/-/create-ecdh-4.0.0.tgz",
+      "integrity": "sha1-iIxyNZbN92EvZJgjPuvXo1MBc30=",
+      "dev": true,
+      "requires": {
+        "bn.js": "4.11.8",
+        "elliptic": "6.4.0"
+      }
+    },
+    "create-hash": {
+      "version": "1.1.3",
+      "resolved": "https://registry.npmjs.org/create-hash/-/create-hash-1.1.3.tgz",
+      "integrity": "sha1-YGBCrIuSYnUPSDyt2rD1gZFy2P0=",
+      "dev": true,
+      "requires": {
+        "cipher-base": "1.0.4",
+        "inherits": "2.0.3",
+        "ripemd160": "2.0.1",
+        "sha.js": "2.4.9"
+      }
+    },
+    "create-hmac": {
+      "version": "1.1.6",
+      "resolved": "https://registry.npmjs.org/create-hmac/-/create-hmac-1.1.6.tgz",
+      "integrity": "sha1-rLniIaThe9sHbpBlfEK5PjcmzwY=",
+      "dev": true,
+      "requires": {
+        "cipher-base": "1.0.4",
+        "create-hash": "1.1.3",
+        "inherits": "2.0.3",
+        "ripemd160": "2.0.1",
+        "safe-buffer": "5.1.1",
+        "sha.js": "2.4.9"
+      }
+    },
+    "cross-spawn": {
+      "version": "3.0.1",
+      "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-3.0.1.tgz",
+      "integrity": "sha1-ElYDfsufDF9549bvE14wdwGEuYI=",
+      "requires": {
+        "lru-cache": "4.1.1",
+        "which": "1.3.0"
+      }
+    },
+    "cryptiles": {
+      "version": "3.1.2",
+      "resolved": "https://registry.npmjs.org/cryptiles/-/cryptiles-3.1.2.tgz",
+      "integrity": "sha1-qJ+7Ig9c4l7FboxKqKT9e1sNKf4=",
+      "requires": {
+        "boom": "5.2.0"
+      },
+      "dependencies": {
+        "boom": {
+          "version": "5.2.0",
+          "resolved": "https://registry.npmjs.org/boom/-/boom-5.2.0.tgz",
+          "integrity": "sha512-Z5BTk6ZRe4tXXQlkqftmsAUANpXmuwlsF5Oov8ThoMbQRzdGTA1ngYRW160GexgOgjsFOKJz0LYhoNi+2AMBUw==",
+          "requires": {
+            "hoek": "4.2.0"
+          }
+        }
+      }
+    },
+    "crypto-browserify": {
+      "version": "3.12.0",
+      "resolved": "https://registry.npmjs.org/crypto-browserify/-/crypto-browserify-3.12.0.tgz",
+      "integrity": "sha512-fz4spIh+znjO2VjL+IdhEpRJ3YN6sMzITSBijk6FK2UvTqruSQW+/cCZTSNsMiZNvUeq0CqurF+dAbyiGOY6Wg==",
+      "dev": true,
+      "requires": {
+        "browserify-cipher": "1.0.0",
+        "browserify-sign": "4.0.4",
+        "create-ecdh": "4.0.0",
+        "create-hash": "1.1.3",
+        "create-hmac": "1.1.6",
+        "diffie-hellman": "5.0.2",
+        "inherits": "2.0.3",
+        "pbkdf2": "3.0.14",
+        "public-encrypt": "4.0.0",
+        "randombytes": "2.0.5",
+        "randomfill": "1.0.3"
+      }
+    },
+    "css-color-names": {
+      "version": "0.0.4",
+      "resolved": "https://registry.npmjs.org/css-color-names/-/css-color-names-0.0.4.tgz",
+      "integrity": "sha1-gIrcLnnPhHOAabZGyyDsJ762KeA=",
+      "dev": true
+    },
+    "css-loader": {
+      "version": "0.28.7",
+      "resolved": "https://registry.npmjs.org/css-loader/-/css-loader-0.28.7.tgz",
+      "integrity": "sha512-GxMpax8a/VgcfRrVy0gXD6yLd5ePYbXX/5zGgTVYp4wXtJklS8Z2VaUArJgc//f6/Dzil7BaJObdSv8eKKCPgg==",
+      "dev": true,
+      "requires": {
+        "babel-code-frame": "6.26.0",
+        "css-selector-tokenizer": "0.7.0",
+        "cssnano": "3.10.0",
+        "icss-utils": "2.1.0",
+        "loader-utils": "1.1.0",
+        "lodash.camelcase": "4.3.0",
+        "object-assign": "4.1.1",
+        "postcss": "5.2.18",
+        "postcss-modules-extract-imports": "1.1.0",
+        "postcss-modules-local-by-default": "1.2.0",
+        "postcss-modules-scope": "1.1.0",
+        "postcss-modules-values": "1.3.0",
+        "postcss-value-parser": "3.3.0",
+        "source-list-map": "2.0.0"
+      }
+    },
+    "css-parse": {
+      "version": "1.7.0",
+      "resolved": "https://registry.npmjs.org/css-parse/-/css-parse-1.7.0.tgz",
+      "integrity": "sha1-Mh9s9zeCpv91ERE5D8BeLGV9jJs=",
+      "dev": true
+    },
+    "css-select": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/css-select/-/css-select-1.2.0.tgz",
+      "integrity": "sha1-KzoRBTnFNV8c2NMUYj6HCxIeyFg=",
+      "dev": true,
+      "requires": {
+        "boolbase": "1.0.0",
+        "css-what": "2.1.0",
+        "domutils": "1.5.1",
+        "nth-check": "1.0.1"
+      }
+    },
+    "css-selector-tokenizer": {
+      "version": "0.7.0",
+      "resolved": "https://registry.npmjs.org/css-selector-tokenizer/-/css-selector-tokenizer-0.7.0.tgz",
+      "integrity": "sha1-5piEdK6MlTR3v15+/s/OzNnPTIY=",
+      "dev": true,
+      "requires": {
+        "cssesc": "0.1.0",
+        "fastparse": "1.1.1",
+        "regexpu-core": "1.0.0"
+      }
+    },
+    "css-what": {
+      "version": "2.1.0",
+      "resolved": "https://registry.npmjs.org/css-what/-/css-what-2.1.0.tgz",
+      "integrity": "sha1-lGfQMsOM+u+58teVASUwYvh/ob0=",
+      "dev": true
+    },
+    "cssauron": {
+      "version": "1.4.0",
+      "resolved": "https://registry.npmjs.org/cssauron/-/cssauron-1.4.0.tgz",
+      "integrity": "sha1-pmAt/34EqDBtwNuaVR6S6LVmKtg=",
+      "dev": true,
+      "requires": {
+        "through": "2.3.8"
+      }
+    },
+    "cssesc": {
+      "version": "0.1.0",
+      "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-0.1.0.tgz",
+      "integrity": "sha1-yBSQPkViM3GgR3tAEJqq++6t27Q=",
+      "dev": true
+    },
+    "cssnano": {
+      "version": "3.10.0",
+      "resolved": "https://registry.npmjs.org/cssnano/-/cssnano-3.10.0.tgz",
+      "integrity": "sha1-Tzj2zqK5sX+gFJDyPx3GjqZcHDg=",
+      "dev": true,
+      "requires": {
+        "autoprefixer": "6.7.7",
+        "decamelize": "1.2.0",
+        "defined": "1.0.0",
+        "has": "1.0.1",
+        "object-assign": "4.1.1",
+        "postcss": "5.2.18",
+        "postcss-calc": "5.3.1",
+        "postcss-colormin": "2.2.2",
+        "postcss-convert-values": "2.6.1",
+        "postcss-discard-comments": "2.0.4",
+        "postcss-discard-duplicates": "2.1.0",
+        "postcss-discard-empty": "2.1.0",
+        "postcss-discard-overridden": "0.1.1",
+        "postcss-discard-unused": "2.2.3",
+        "postcss-filter-plugins": "2.0.2",
+        "postcss-merge-idents": "2.1.7",
+        "postcss-merge-longhand": "2.0.2",
+        "postcss-merge-rules": "2.1.2",
+        "postcss-minify-font-values": "1.0.5",
+        "postcss-minify-gradients": "1.0.5",
+        "postcss-minify-params": "1.2.2",
+        "postcss-minify-selectors": "2.1.1",
+        "postcss-normalize-charset": "1.1.1",
+        "postcss-normalize-url": "3.0.8",
+        "postcss-ordered-values": "2.2.3",
+        "postcss-reduce-idents": "2.4.0",
+        "postcss-reduce-initial": "1.0.1",
+        "postcss-reduce-transforms": "1.0.4",
+        "postcss-svgo": "2.1.6",
+        "postcss-unique-selectors": "2.0.2",
+        "postcss-value-parser": "3.3.0",
+        "postcss-zindex": "2.2.0"
+      }
+    },
+    "csso": {
+      "version": "2.3.2",
+      "resolved": "https://registry.npmjs.org/csso/-/csso-2.3.2.tgz",
+      "integrity": "sha1-3dUsWHAz9J6Utx/FVWnyUuj/X4U=",
+      "dev": true,
+      "requires": {
+        "clap": "1.2.3",
+        "source-map": "0.5.7"
+      },
+      "dependencies": {
+        "source-map": {
+          "version": "0.5.7",
+          "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz",
+          "integrity": "sha1-igOdLRAh0i0eoUyA2OpGi6LvP8w=",
+          "dev": true
+        }
+      }
+    },
+    "cuint": {
+      "version": "0.2.2",
+      "resolved": "https://registry.npmjs.org/cuint/-/cuint-0.2.2.tgz",
+      "integrity": "sha1-QICG1AlVDCYxFVYZ6fp7ytw7mRs=",
+      "dev": true
+    },
+    "currently-unhandled": {
+      "version": "0.4.1",
+      "resolved": "https://registry.npmjs.org/currently-unhandled/-/currently-unhandled-0.4.1.tgz",
+      "integrity": "sha1-mI3zP+qxke95mmE2nddsF635V+o=",
+      "requires": {
+        "array-find-index": "1.0.2"
+      }
+    },
+    "custom-event": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/custom-event/-/custom-event-1.0.1.tgz",
+      "integrity": "sha1-XQKkaFCt8bSjF5RqOSj8y1v9BCU=",
+      "dev": true
+    },
+    "cyclist": {
+      "version": "0.2.2",
+      "resolved": "https://registry.npmjs.org/cyclist/-/cyclist-0.2.2.tgz",
+      "integrity": "sha1-GzN5LhHpFKL9bW7WRHRkRE5fpkA=",
+      "dev": true
+    },
+    "d": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/d/-/d-1.0.0.tgz",
+      "integrity": "sha1-dUu1v+VUUdpppYuU1F9MWwRi1Y8=",
+      "dev": true,
+      "requires": {
+        "es5-ext": "0.10.37"
+      }
+    },
+    "dashdash": {
+      "version": "1.14.1",
+      "resolved": "https://registry.npmjs.org/dashdash/-/dashdash-1.14.1.tgz",
+      "integrity": "sha1-hTz6D3y+L+1d4gMmuN1YEDX24vA=",
+      "requires": {
+        "assert-plus": "1.0.0"
+      }
+    },
+    "date-fns": {
+      "version": "1.29.0",
+      "resolved": "https://registry.npmjs.org/date-fns/-/date-fns-1.29.0.tgz",
+      "integrity": "sha512-lbTXWZ6M20cWH8N9S6afb0SBm6tMk+uUg6z3MqHPKE9atmsY3kJkTm8vKe93izJ2B2+q5MV990sM2CHgtAZaOw==",
+      "dev": true
+    },
+    "date-now": {
+      "version": "0.1.4",
+      "resolved": "https://registry.npmjs.org/date-now/-/date-now-0.1.4.tgz",
+      "integrity": "sha1-6vQ5/U1ISK105cx9vvIAZyueNFs=",
+      "dev": true
+    },
+    "debug": {
+      "version": "2.6.9",
+      "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz",
+      "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==",
+      "requires": {
+        "ms": "2.0.0"
+      }
+    },
+    "decamelize": {
+      "version": "1.2.0",
+      "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-1.2.0.tgz",
+      "integrity": "sha1-9lNNFRSCabIDUue+4m9QH5oZEpA="
+    },
+    "deep-equal": {
+      "version": "1.0.1",
+      "resolved": "https://registry.npmjs.org/deep-equal/-/deep-equal-1.0.1.tgz",
+      "integrity": "sha1-9dJgKStmDghO/0zbyfCK0yR0SLU=",
+      "dev": true
+    },
+    "deep-extend": {
+      "version": "0.4.2",
+      "resolved": "https://registry.npmjs.org/deep-extend/-/deep-extend-0.4.2.tgz",
+      "integrity": "sha1-SLaZwn4zS/ifEIkr5DL25MfTSn8=",
+      "dev": true
+    },
+    "default-require-extensions": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/default-require-extensions/-/default-require-extensions-1.0.0.tgz",
+      "integrity": "sha1-836hXT4T/9m0N9M+GnW1+5eHTLg=",
+      "dev": true,
+      "requires": {
+        "strip-bom": "2.0.0"
+      }
+    },
+    "define-properties": {
+      "version": "1.1.2",
+      "resolved": "https://registry.npmjs.org/define-properties/-/define-properties-1.1.2.tgz",
+      "integrity": "sha1-g6c/L+pWmJj7c3GTyPhzyvbUXJQ=",
+      "dev": true,
+      "requires": {
+        "foreach": "2.0.5",
+        "object-keys": "1.0.11"
+      }
+    },
+    "defined": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/defined/-/defined-1.0.0.tgz",
+      "integrity": "sha1-yY2bzvdWdBiOEQlpFRGZ45sfppM=",
+      "dev": true
+    },
+    "del": {
+      "version": "3.0.0",
+      "resolved": "https://registry.npmjs.org/del/-/del-3.0.0.tgz",
+      "integrity": "sha1-U+z2mf/LyzljdpGrE7rxYIGXZuU=",
+      "dev": true,
+      "requires": {
+        "globby": "6.1.0",
+        "is-path-cwd": "1.0.0",
+        "is-path-in-cwd": "1.0.0",
+        "p-map": "1.2.0",
+        "pify": "3.0.0",
+        "rimraf": "2.6.2"
+      },
+      "dependencies": {
+        "globby": {
+          "version": "6.1.0",
+          "resolved": "https://registry.npmjs.org/globby/-/globby-6.1.0.tgz",
+          "integrity": "sha1-9abXDoOV4hyFj7BInWTfAkJNUGw=",
+          "dev": true,
+          "requires": {
+            "array-union": "1.0.2",
+            "glob": "7.1.2",
+            "object-assign": "4.1.1",
+            "pify": "2.3.0",
+            "pinkie-promise": "2.0.1"
+          },
+          "dependencies": {
+            "pify": {
+              "version": "2.3.0",
+              "resolved": "https://registry.npmjs.org/pify/-/pify-2.3.0.tgz",
+              "integrity": "sha1-7RQaasBDqEnqWISY59yosVMw6Qw=",
+              "dev": true
+            }
+          }
+        },
+        "pify": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/pify/-/pify-3.0.0.tgz",
+          "integrity": "sha1-5aSs0sEB/fPZpNB/DbxNtJ3SgXY=",
+          "dev": true
+        }
+      }
+    },
+    "delayed-stream": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz",
+      "integrity": "sha1-3zrhmayt+31ECqrgsp4icrJOxhk="
+    },
+    "delegates": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/delegates/-/delegates-1.0.0.tgz",
+      "integrity": "sha1-hMbhWbgZBP3KWaDvRM2HDTElD5o="
+    },
+    "denodeify": {
+      "version": "1.2.1",
+      "resolved": "https://registry.npmjs.org/denodeify/-/denodeify-1.2.1.tgz",
+      "integrity": "sha1-OjYof1A05pnnV3kBBSwubJQlFjE=",
+      "dev": true
+    },
+    "depd": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/depd/-/depd-1.1.1.tgz",
+      "integrity": "sha1-V4O04cRZ8G+lyif5kfPQbnoxA1k="
+    },
+    "des.js": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/des.js/-/des.js-1.0.0.tgz",
+      "integrity": "sha1-wHTS4qpqipoH29YfmhXCzYPsjsw=",
+      "dev": true,
+      "requires": {
+        "inherits": "2.0.3",
+        "minimalistic-assert": "1.0.0"
+      }
+    },
+    "destroy": {
+      "version": "1.0.4",
+      "resolved": "https://registry.npmjs.org/destroy/-/destroy-1.0.4.tgz",
+      "integrity": "sha1-l4hXRCxEdJ5CBmE+N5RiBYJqvYA="
+    },
+    "detect-indent": {
+      "version": "4.0.0",
+      "resolved": "https://registry.npmjs.org/detect-indent/-/detect-indent-4.0.0.tgz",
+      "integrity": "sha1-920GQ1LN9Docts5hnE7jqUdd4gg=",
+      "dev": true,
+      "requires": {
+        "repeating": "2.0.1"
+      }
+    },
+    "detect-node": {
+      "version": "2.0.3",
+      "resolved": "https://registry.npmjs.org/detect-node/-/detect-node-2.0.3.tgz",
+      "integrity": "sha1-ogM8CcyOFY03dI+951B4Mr1s4Sc=",
+      "dev": true
+    },
+    "di": {
+      "version": "0.0.1",
+      "resolved": "https://registry.npmjs.org/di/-/di-0.0.1.tgz",
+      "integrity": "sha1-gGZJMmzqp8qjMG112YXqJ0i6kTw=",
+      "dev": true
+    },
+    "diff": {
+      "version": "3.4.0",
+      "resolved": "https://registry.npmjs.org/diff/-/diff-3.4.0.tgz",
+      "integrity": "sha512-QpVuMTEoJMF7cKzi6bvWhRulU1fZqZnvyVQgNhPaxxuTYwyjn/j1v9falseQ/uXWwPnO56RBfwtg4h/EQXmucA==",
+      "dev": true
+    },
+    "diffie-hellman": {
+      "version": "5.0.2",
+      "resolved": "https://registry.npmjs.org/diffie-hellman/-/diffie-hellman-5.0.2.tgz",
+      "integrity": "sha1-tYNXOScM/ias9jIJn97SoH8gnl4=",
+      "dev": true,
+      "requires": {
+        "bn.js": "4.11.8",
+        "miller-rabin": "4.0.1",
+        "randombytes": "2.0.5"
+      }
+    },
+    "dir-glob": {
+      "version": "2.0.0",
+      "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-2.0.0.tgz",
+      "integrity": "sha512-37qirFDz8cA5fimp9feo43fSuRo2gHwaIn6dXL8Ber1dGwUosDrGZeCCXq57WnIqE4aQ+u3eQZzsk1yOzhdwag==",
+      "dev": true,
+      "requires": {
+        "arrify": "1.0.1",
+        "path-type": "3.0.0"
+      },
+      "dependencies": {
+        "path-type": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/path-type/-/path-type-3.0.0.tgz",
+          "integrity": "sha512-T2ZUsdZFHgA3u4e5PfPbjd7HDDpxPnQb5jN0SrDsjNSuVXHJqtwTnWqG0B1jZrgmJ/7lj1EmVIByWt1gxGkWvg==",
+          "dev": true,
+          "requires": {
+            "pify": "3.0.0"
+          }
+        },
+        "pify": {
+          "version": "3.0.0",
+          "resolved": "https://registry.npmjs.org/pify/-/pify-3.0.0.tgz",
+          "integrity": "sha1-5aSs0sEB/fPZpNB/DbxNtJ3SgXY=",
+          "dev": true
+        }
+      }
+    },
+    "dns-equal": {
+      "version": "1.0.0",
+      "resolved": "https://registry.npmjs.org/dns-equal/-/dns-equal-1.0.0.tgz",
+      "integrity": "sha1-s55/HabrCnW6nBcySzR1PEfgZU0=",
+      "dev": true
+    },
+    "dns-packet": {
+      "version": "1.2.2",
+      "resolved": "https://registry.npmjs.org/dns-packet/-/dns-packet-1.2.2.tgz",
+      "integrity": "sha512-kN+DjfGF7dJGUL7nWRktL9Z18t1rWP3aQlyZdY8XlpvU3Nc6GeFTQApftcjtWKxAZfiggZSGrCEoszNgvnpwDg==",
+      "dev": true,
+      "requires": {
+        "ip": "1.1.5",
+        "safe-buffer": "5.1.1"
+      }
+    },
+    "dns-txt": {
+      "version": "2.0.2",
+      "resolved": "https://registry.npmjs.org/dns-txt/-/dns-txt-2.0.2.tgz",
+      "integrity": "sha1-uR2Ab10nGI5Ks+fRB9iBocxGQrY=",
+      "dev": true,
+      "requires": {
+        "buffer-indexof": "1.1.1"
+      }
+    },
+    "dom-converter": {
+      "version": "0.1.4",
+      "resolved": "https://registry.npmjs.org/dom-converter/-/dom-converter-0.1.4.tgz",
+      "integrity": "sha1-pF71cnuJDJv/5tfIduexnLDhfzs=",
+      "dev": true,
+      "requires": {
+        "utila": "0.3.3"
+      },
+      "dependencies": {
+        "utila": {
+          "version": "0.3.3",
+          "resolved": "https://registry.npmjs.org/utila/-/utila-0.3.3.tgz",
+          "integrity": "sha1-1+jn1+MJEHCSsF+NloiCTWM6QiY=",
+          "dev": true
+        }
+      }
+    },
+    "dom-serialize": {
+      "version": "2.2.1",
+      "resolved": "https://registry.npmjs.org/dom-serialize/-/dom-serialize-2.2.1.tgz",
+      "integrity": "sha1-ViromZ9Evl6jB29UGdzVnrQ6yVs=",
+      "dev": true,
+      "requires": {
+        "custom-event": "1.0.1",
+        "ent": "2.2.0",
+        "extend": "3.0.1",
+        "void-elements": "2.0.1"
+      }
+    },
+    "dom-serializer": {
+      "version": "0.1.0",
+      "resolved": "https://registry.npmjs.org/dom-serializer/-/dom-serializer-0.1.0.tgz",
+      "integrity": "sha1-BzxpdUbOB4DOI75KKOKT5AvDDII=",
+      "dev": true,
+      "requires": {
+        "domelementtype": "1.1.3",
+        "entities": "1.1.1"
+      },
+      "dependencies": {
+        "domelementtype": {
+          "version": "1.1.3",
+          "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.1.3.tgz",
+          "integrity": "sha1-vSh3PiZCiBrsUVRJJCmcXNgiGFs=",
+          "dev": true
+        }
+      }
+    },
+    "domain-browser": {
+      "version": "1.1.7",
+      "resolved": "https://registry.npmjs.org/domain-browser/-/domain-browser-1.1.7.tgz",
+      "integrity": "sha1-hnqksJP6oF8d4IwG9NeyH9+GmLw=",
+      "dev": true
+    },
+    "domelementtype": {
+      "version": "1.3.0",
+      "resolved": "https://registry.npmjs.org/domelementtype/-/domelementtype-1.3.0.tgz",
+      "integrity": "sha1-sXrtguirWeUt2cGbF1bg/BhyBMI=",
+      "dev": true
+    },
+    "domhandler": {
+      "version": "2.1.0",
+      "resolved": "https://registry.npmjs.org/domhandler/-/domhandler-2.1.0.tgz",
+      "integrity": "sha1-0mRvXlf2w7qxHPbLBdPArPdBJZQ=",
+      "dev": true,
+      "requires": {
+        "domelementtype": "1.3.0"
+      }
+    },
+    "domutils": {
+      "version": "1.5.1",
+      "resolved": "https://registry.npmjs.org/domutils/-/domutils-1.5.1.tgz",
+      "integrity": "sha1-3NhIiib1Y9YQeeSMn3t+Mjc2gs8=",
+      "dev": true,
+      "requires": {
+        "dom-serializer": "0.1.0",
+        "domelementtype": "1.3.0"
+      }
+    },
+    "dotenv": {
+      "version": "4.0.0",
+      "resolved": "https://registry.npmjs.org/dotenv/-/dotenv-4.0.0.tgz",
+      "integrity": "sha1-hk7xN5rO1Vzm+V3r7NzhefegzR0="
+    },
+    "duplexer": {
+      "version": "0.1.1",
+      "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.1.tgz",
+      "integrity": "sha1-rOb/gIwc5mtX0ev5eXessCM0z8E=",
+      "dev": true
+    },
+    "duplexify": {
+      "version": "3.5.1",
+      "resolved": "https://registry.npmjs.org/duplexify/-/duplexify-3.5.1.tgz",
+      "integrity": "sha512-j5goxHTwVED1Fpe5hh3q9R93Kip0Bg2KVAt4f8CEYM3UEwYcPSvWbXaUQOzdX/HtiNomipv+gU7ASQPDbV7pGQ==",
+      "dev": true,
+      "requires": {
+        "end-of-stream": "1.4.0",
+        "inherits": "2.0.3",
+        "readable-stream": "2.3.3",
+        "stream-shift": "1.0.0"
+      }
+    },
+    "ecc-jsbn": {
+      "version": "0.1.1",
+      "resolved": "https://registry.npmjs.org/ecc-jsbn/-/ecc-jsbn-0.1.1.tgz",
+      "integrity": "sha1-D8c6ntXw1Tw4GTOYUj735UN3dQU=",
+      "optional": true,
+      "requires": {
+        "jsbn": "0.1.1"
+      }
+    },
+    "ee-first": {
+      "version": "1.1.1",
+      "resolved": "https://registry.npmjs.org/ee-first/-/ee-first-1.1.1.tgz",
+      "integrity": "sha1-WQxhFWsK4vTwJVcyoViyZrxWsh0="
+    },
+    "ejs": {
+      "version": "2.5.7",
+      "resolved": "https://registry.npmjs.org/ejs/-/ejs-2.5.7.tgz",
+      "integrity": "sha1-zIcsFoiArjxxiXYv1f/ACJbJUYo=",
+      "dev": true
+    },
+    "electron-to-chromium": {
+      "version": "1.3.28",
+ 

<TRUNCATED>