You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2014/02/19 19:20:53 UTC

git commit: [HELIX-360] Remove code duplication for list of required paths

Repository: helix
Updated Branches:
  refs/heads/master 560097fbe -> 0e310fa10


[HELIX-360] Remove code duplication for list of required paths


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

Branch: refs/heads/master
Commit: 0e310fa10971e9d6a14c450829fb162731bd7833
Parents: 560097f
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Mon Feb 3 11:21:55 2014 -0800
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Wed Feb 19 10:20:38 2014 -0800

----------------------------------------------------------------------
 .../api/accessor/AtomicParticipantAccessor.java |  4 +-
 .../api/accessor/AtomicResourceAccessor.java    |  4 +-
 .../helix/api/accessor/ClusterAccessor.java     | 34 ++-------
 .../helix/api/accessor/ParticipantAccessor.java | 34 +++------
 .../helix/api/accessor/ResourceAccessor.java    |  7 +-
 .../stages/PersistAssignmentStage.java          |  4 +-
 .../org/apache/helix/manager/zk/ZKUtil.java     | 79 +++++++++++++-------
 .../helix/manager/zk/ZkHelixConnection.java     | 11 +--
 .../org/apache/helix/tools/NewClusterSetup.java |  6 +-
 .../api/accessor/TestAccessorRecreate.java      |  2 +-
 10 files changed, 89 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java
index 1c734e3..90f58ea 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicParticipantAccessor.java
@@ -64,11 +64,11 @@ public class AtomicParticipantAccessor extends ParticipantAccessor {
    */
   public AtomicParticipantAccessor(ClusterId clusterId, HelixDataAccessor accessor,
       HelixLockable lockProvider) {
-    super(accessor);
+    super(clusterId, accessor);
     _clusterId = clusterId;
     _accessor = accessor;
     _lockProvider = lockProvider;
-    _participantAccessor = new ParticipantAccessor(accessor);
+    _participantAccessor = new ParticipantAccessor(clusterId, accessor);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
index cda83d8..48457b2 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
@@ -60,11 +60,11 @@ public class AtomicResourceAccessor extends ResourceAccessor {
    */
   public AtomicResourceAccessor(ClusterId clusterId, HelixDataAccessor accessor,
       HelixLockable lockProvider) {
-    super(accessor);
+    super(clusterId, accessor);
     _clusterId = clusterId;
     _accessor = accessor;
     _lockProvider = lockProvider;
-    _resourceAccessor = new ResourceAccessor(accessor);
+    _resourceAccessor = new ResourceAccessor(clusterId, accessor);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
index 36c7aaa..abb3e49 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
@@ -57,6 +57,7 @@ import org.apache.helix.controller.context.ControllerContextHolder;
 import org.apache.helix.controller.rebalancer.config.PartitionedRebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfigHolder;
+import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConfiguration;
 import org.apache.helix.model.ClusterConstraints;
@@ -75,7 +76,6 @@ import org.apache.helix.model.ResourceConfiguration;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
@@ -720,7 +720,7 @@ public class ClusterAccessor {
    * @return true if valid or false otherwise
    */
   public boolean isClusterStructureValid() {
-    List<String> paths = getRequiredPaths(_keyBuilder);
+    List<String> paths = ZKUtil.getRequiredPathsForCluster(_clusterId.toString());
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
     if (baseAccessor != null) {
       boolean[] existsResults = baseAccessor.exists(paths, 0);
@@ -738,7 +738,7 @@ public class ClusterAccessor {
    */
   public void initClusterStructure() {
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
-    List<String> paths = getRequiredPaths(_keyBuilder);
+    List<String> paths = ZKUtil.getRequiredPathsForCluster(_clusterId.toString());
     for (String path : paths) {
       boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT);
       if (!status && LOG.isDebugEnabled()) {
@@ -752,33 +752,11 @@ public class ClusterAccessor {
    */
   private void clearClusterStructure() {
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
-    List<String> paths = getRequiredPaths(_keyBuilder);
+    List<String> paths = ZKUtil.getRequiredPathsForCluster(_clusterId.toString());
     baseAccessor.remove(paths, 0);
   }
 
   /**
-   * Get all property paths that must be set for a cluster structure to be valid
-   * @param keyBuilder a PropertyKey.Builder for the cluster
-   * @return list of paths as strings
-   */
-  private static List<String> getRequiredPaths(PropertyKey.Builder keyBuilder) {
-    List<String> paths = Lists.newArrayList();
-    paths.add(keyBuilder.clusterConfigs().getPath());
-    paths.add(keyBuilder.instanceConfigs().getPath());
-    paths.add(keyBuilder.propertyStore().getPath());
-    paths.add(keyBuilder.liveInstances().getPath());
-    paths.add(keyBuilder.instances().getPath());
-    paths.add(keyBuilder.externalViews().getPath());
-    paths.add(keyBuilder.controller().getPath());
-    paths.add(keyBuilder.stateModelDefs().getPath());
-    paths.add(keyBuilder.controllerMessages().getPath());
-    paths.add(keyBuilder.controllerTaskErrors().getPath());
-    paths.add(keyBuilder.controllerTaskStatuses().getPath());
-    paths.add(keyBuilder.controllerLeaderHistory().getPath());
-    return paths;
-  }
-
-  /**
    * add a participant to cluster
    * @param participant
    * @return true if participant added, false otherwise
@@ -793,7 +771,7 @@ public class ClusterAccessor {
       return false;
     }
 
-    ParticipantAccessor participantAccessor = new ParticipantAccessor(_accessor);
+    ParticipantAccessor participantAccessor = new ParticipantAccessor(_clusterId, _accessor);
     ParticipantId participantId = participant.getId();
     InstanceConfig existConfig =
         _accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify()));
@@ -832,7 +810,7 @@ public class ClusterAccessor {
    * @return true if participant dropped, false if there was an error
    */
   public boolean dropParticipantFromCluster(ParticipantId participantId) {
-    ParticipantAccessor accessor = new ParticipantAccessor(_accessor);
+    ParticipantAccessor accessor = new ParticipantAccessor(_clusterId, _accessor);
     return accessor.dropParticipant(participantId);
   }
 

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
index c3deafe..3a34ca2 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
@@ -45,6 +45,7 @@ import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
 import org.apache.helix.api.config.ParticipantConfig;
 import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.MessageId;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
@@ -53,6 +54,7 @@ import org.apache.helix.api.id.SessionId;
 import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.rebalancer.config.PartitionedRebalancerConfig;
 import org.apache.helix.controller.rebalancer.config.RebalancerConfig;
+import org.apache.helix.manager.zk.ZKUtil;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -75,9 +77,11 @@ public class ParticipantAccessor {
   private static final Logger LOG = Logger.getLogger(ParticipantAccessor.class);
 
   private final HelixDataAccessor _accessor;
+  private final ClusterId _clusterId;
   private final PropertyKey.Builder _keyBuilder;
 
-  public ParticipantAccessor(HelixDataAccessor accessor) {
+  public ParticipantAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
+    _clusterId = clusterId;
     _accessor = accessor;
     _keyBuilder = accessor.keyBuilder();
   }
@@ -709,7 +713,8 @@ public class ParticipantAccessor {
    * Create empty persistent properties to ensure that there is a valid participant structure
    */
   public void initParticipantStructure(ParticipantId participantId) {
-    List<String> paths = getRequiredPaths(_keyBuilder, participantId);
+    List<String> paths =
+        ZKUtil.getRequiredPathsForInstance(_clusterId.toString(), participantId.toString());
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
     for (String path : paths) {
       boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT);
@@ -723,7 +728,8 @@ public class ParticipantAccessor {
    * Clear properties for the participant
    */
   void clearParticipantStructure(ParticipantId participantId) {
-    List<String> paths = getRequiredPaths(_keyBuilder, participantId);
+    List<String> paths =
+        ZKUtil.getRequiredPathsForInstance(_clusterId.toString(), participantId.toString());
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
     baseAccessor.remove(paths, 0);
   }
@@ -733,7 +739,8 @@ public class ParticipantAccessor {
    * @return true if valid or false otherwise
    */
   public boolean isParticipantStructureValid(ParticipantId participantId) {
-    List<String> paths = getRequiredPaths(_keyBuilder, participantId);
+    List<String> paths =
+        ZKUtil.getRequiredPathsForInstance(_clusterId.toString(), participantId.toString());
     BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
     if (baseAccessor != null) {
       boolean[] existsResults = baseAccessor.exists(paths, 0);
@@ -747,27 +754,10 @@ public class ParticipantAccessor {
   }
 
   /**
-   * Get the paths that should be created if the participant exists
-   * @param keyBuilder PropertyKey.Builder for the cluster
-   * @param participantId the participant for which to generate paths
-   * @return list of required paths as strings
-   */
-  private static List<String> getRequiredPaths(PropertyKey.Builder keyBuilder,
-      ParticipantId participantId) {
-    List<String> paths = Lists.newArrayList();
-    paths.add(keyBuilder.instanceConfig(participantId.stringify()).getPath());
-    paths.add(keyBuilder.messages(participantId.stringify()).getPath());
-    paths.add(keyBuilder.currentStates(participantId.stringify()).getPath());
-    paths.add(keyBuilder.participantErrors(participantId.stringify()).getPath());
-    paths.add(keyBuilder.statusUpdates(participantId.stringify()).getPath());
-    return paths;
-  }
-
-  /**
    * Get a ResourceAccessor instance
    * @return ResourceAccessor
    */
   protected ResourceAccessor resourceAccessor() {
-    return new ResourceAccessor(_accessor);
+    return new ResourceAccessor(_clusterId, _accessor);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
index 80c5b16..73d43b0 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
@@ -35,6 +35,7 @@ import org.apache.helix.api.State;
 import org.apache.helix.api.config.ResourceConfig;
 import org.apache.helix.api.config.ResourceConfig.ResourceType;
 import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.api.id.ResourceId;
@@ -58,10 +59,12 @@ import com.google.common.collect.Sets;
 
 public class ResourceAccessor {
   private static final Logger LOG = Logger.getLogger(ResourceAccessor.class);
+  private final ClusterId _clusterId;
   private final HelixDataAccessor _accessor;
   private final PropertyKey.Builder _keyBuilder;
 
-  public ResourceAccessor(HelixDataAccessor accessor) {
+  public ResourceAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
+    _clusterId = clusterId;
     _accessor = accessor;
     _keyBuilder = accessor.keyBuilder();
   }
@@ -481,6 +484,6 @@ public class ResourceAccessor {
    * @return ParticipantAccessor
    */
   protected ParticipantAccessor participantAccessor() {
-    return new ParticipantAccessor(_accessor);
+    return new ParticipantAccessor(_clusterId, _accessor);
   }
 }

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java
index 45fc355..b3252a8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistAssignmentStage.java
@@ -21,6 +21,7 @@ package org.apache.helix.controller.stages;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
 import org.apache.helix.api.accessor.ResourceAccessor;
 import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
@@ -33,8 +34,9 @@ public class PersistAssignmentStage extends AbstractBaseStage {
   @Override
   public void process(ClusterEvent event) throws Exception {
     HelixManager helixManager = event.getAttribute("helixmanager");
+    Cluster cluster = event.getAttribute("ClusterDataCache");
     HelixDataAccessor accessor = helixManager.getHelixDataAccessor();
-    ResourceAccessor resourceAccessor = new ResourceAccessor(accessor);
+    ResourceAccessor resourceAccessor = new ResourceAccessor(cluster.getId(), accessor);
     BestPossibleStateOutput assignments =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     for (ResourceId resourceId : assignments.getAssignedResources()) {

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/manager/zk/ZKUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKUtil.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKUtil.java
index 9018416..30ee16c 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKUtil.java
@@ -44,25 +44,8 @@ public final class ZKUtil {
     if (clusterName == null || zkClient == null) {
       return false;
     }
-    ArrayList<String> requiredPaths = new ArrayList<String>();
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.IDEALSTATES, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
-        ConfigScopeProperty.CLUSTER.toString(), clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
-        ConfigScopeProperty.PARTICIPANT.toString()));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
-        ConfigScopeProperty.RESOURCE.toString()));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.LIVEINSTANCES, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.INSTANCES, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONTROLLER, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.STATEMODELDEFS, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.MESSAGES_CONTROLLER, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.ERRORS_CONTROLLER, clusterName));
-    requiredPaths.add(PropertyPathConfig
-        .getPath(PropertyType.STATUSUPDATES_CONTROLLER, clusterName));
-    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.HISTORY, clusterName));
+
+    List<String> requiredPaths = getRequiredPathsForCluster(clusterName);
     boolean isValid = true;
 
     for (String path : requiredPaths) {
@@ -77,16 +60,7 @@ public final class ZKUtil {
   public static boolean isInstanceSetup(ZkClient zkclient, String clusterName, String instanceName,
       InstanceType type) {
     if (type == InstanceType.PARTICIPANT || type == InstanceType.CONTROLLER_PARTICIPANT) {
-      ArrayList<String> requiredPaths = new ArrayList<String>();
-      requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
-          ConfigScopeProperty.PARTICIPANT.toString(), instanceName));
-      requiredPaths.add(PropertyPathConfig
-          .getPath(PropertyType.MESSAGES, clusterName, instanceName));
-      requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, clusterName,
-          instanceName));
-      requiredPaths.add(PropertyPathConfig.getPath(PropertyType.STATUSUPDATES, clusterName,
-          instanceName));
-      requiredPaths.add(PropertyPathConfig.getPath(PropertyType.ERRORS, clusterName, instanceName));
+      List<String> requiredPaths = getRequiredPathsForInstance(clusterName, instanceName);
       boolean isValid = true;
 
       for (String path : requiredPaths) {
@@ -101,6 +75,53 @@ public final class ZKUtil {
     return true;
   }
 
+  /**
+   * Get the required ZK paths for a valid cluster
+   * @param clusterName the cluster to check
+   * @return List of paths as strings
+   */
+  public static List<String> getRequiredPathsForCluster(String clusterName) {
+    List<String> requiredPaths = new ArrayList<String>();
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.IDEALSTATES, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
+        ConfigScopeProperty.CLUSTER.toString(), clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
+        ConfigScopeProperty.PARTICIPANT.toString()));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
+        ConfigScopeProperty.RESOURCE.toString()));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.LIVEINSTANCES, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.INSTANCES, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.EXTERNALVIEW, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONTROLLER, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.STATEMODELDEFS, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.MESSAGES_CONTROLLER, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.ERRORS_CONTROLLER, clusterName));
+    requiredPaths.add(PropertyPathConfig
+        .getPath(PropertyType.STATUSUPDATES_CONTROLLER, clusterName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.HISTORY, clusterName));
+    return requiredPaths;
+  }
+
+  /**
+   * Get the required ZK paths for a valid instance
+   * @param clusterName the cluster that owns the instance
+   * @param instanceName the instance to check
+   * @return List of paths as strings
+   */
+  public static List<String> getRequiredPathsForInstance(String clusterName, String instanceName) {
+    List<String> requiredPaths = new ArrayList<String>();
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CONFIGS, clusterName,
+        ConfigScopeProperty.PARTICIPANT.toString(), instanceName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.MESSAGES, clusterName, instanceName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.CURRENTSTATES, clusterName,
+        instanceName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.STATUSUPDATES, clusterName,
+        instanceName));
+    requiredPaths.add(PropertyPathConfig.getPath(PropertyType.ERRORS, clusterName, instanceName));
+    return requiredPaths;
+  }
+
   public static void createChildren(ZkClient client, String parentPath, List<ZNRecord> list) {
     client.createPersistent(parentPath, true);
     if (list != null) {

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
index 1bdc54c..d59fad7 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkHelixConnection.java
@@ -24,7 +24,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
@@ -42,9 +41,9 @@ import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixAutoController;
-import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixConnection;
 import org.apache.helix.HelixConnectionStateListener;
+import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixController;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
@@ -65,7 +64,6 @@ import org.apache.helix.api.accessor.ResourceAccessor;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ControllerId;
 import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
@@ -232,12 +230,12 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
 
   @Override
   public ResourceAccessor createResourceAccessor(ClusterId clusterId) {
-    return new ResourceAccessor(createDataAccessor(clusterId));
+    return new ResourceAccessor(clusterId, createDataAccessor(clusterId));
   }
 
   @Override
   public ParticipantAccessor createParticipantAccessor(ClusterId clusterId) {
-    return new ParticipantAccessor(createDataAccessor(clusterId));
+    return new ParticipantAccessor(clusterId, createDataAccessor(clusterId));
   }
 
   @Override
@@ -380,8 +378,7 @@ public class ZkHelixConnection implements HelixConnection, IZkStateListener {
       ClusterId clusterId, ParticipantId participantId) {
     addListener(role, listener,
         new PropertyKey.Builder(clusterId.stringify()).healthReports(participantId.stringify()),
-        ChangeType.HEALTH,
-        new EventType[] {
+        ChangeType.HEALTH, new EventType[] {
             EventType.NodeChildrenChanged, EventType.NodeDeleted, EventType.NodeCreated
         });
   }

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
index ba8958d..ea28c76 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
@@ -256,11 +256,13 @@ public class NewClusterSetup {
   }
 
   ParticipantAccessor participantAccessor(String clusterName) {
-    return new ParticipantAccessor(new ZKHelixDataAccessor(clusterName, _baseAccessor));
+    return new ParticipantAccessor(ClusterId.from(clusterName), new ZKHelixDataAccessor(
+        clusterName, _baseAccessor));
   }
 
   ResourceAccessor resourceAccessor(String clusterName) {
-    return new ResourceAccessor(new ZKHelixDataAccessor(clusterName, _baseAccessor));
+    return new ResourceAccessor(ClusterId.from(clusterName), new ZKHelixDataAccessor(clusterName,
+        _baseAccessor));
   }
 
   void addCluster(String[] optValues) {

http://git-wip-us.apache.org/repos/asf/helix/blob/0e310fa1/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java
index 4eebbc6..ee51834 100644
--- a/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java
+++ b/helix-core/src/test/java/org/apache/helix/api/accessor/TestAccessorRecreate.java
@@ -118,7 +118,7 @@ public class TestAccessorRecreate extends ZkUnitTestBase {
     Assert.assertTrue(created);
 
     // read the participant
-    ParticipantAccessor participantAccessor = new ParticipantAccessor(helixAccessor);
+    ParticipantAccessor participantAccessor = new ParticipantAccessor(clusterId, helixAccessor);
     Participant participantSnapshot = participantAccessor.readParticipant(participantId);
     Assert.assertEquals(participantSnapshot.getUserConfig().getIntField(MODIFIER, -1), 1);