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 2013/09/27 01:05:10 UTC

[1/6] [HELIX-238] Refactor, add update to accessors, test update logic

Updated Branches:
  refs/heads/helix-logical-model 41b6e77de -> c070a7651


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java
index 3c1ee13..652ac73 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestAutoRebalanceStrategy.java
@@ -34,19 +34,33 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 
 import org.apache.helix.HelixDefinedState;
-import org.apache.helix.Mocks.MockAccessor;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.HelixVersion;
+import org.apache.helix.api.Participant;
+import org.apache.helix.api.RunningInstance;
+import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
-import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.api.config.ClusterConfig;
+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;
+import org.apache.helix.api.id.ProcId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
-import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.Maps;
+
 public class TestAutoRebalanceStrategy {
   private static Logger logger = Logger.getLogger(TestAutoRebalanceStrategy.class);
 
@@ -210,23 +224,36 @@ public class TestAutoRebalanceStrategy {
 
     private Map<String, Map<String, String>> getMapping(final Map<String, List<String>> listResult) {
       final Map<String, Map<String, String>> mapResult = new HashMap<String, Map<String, String>>();
-      ClusterDataCache cache = new ClusterDataCache();
-      MockAccessor accessor = new MockAccessor();
-      Builder keyBuilder = accessor.keyBuilder();
-      for (String node : _liveNodes) {
-        LiveInstance liveInstance = new LiveInstance(node);
-        liveInstance.setSessionId("testSession");
-        accessor.setProperty(keyBuilder.liveInstance(node), liveInstance);
-      }
-      cache.refresh(accessor);
       for (String partition : _partitions) {
-        List<String> preferenceList = listResult.get(partition);
-        Map<String, String> currentStateMap = _currentMapping.get(partition);
-        Set<String> disabled = Collections.emptySet();
-        Map<String, String> assignment =
-            ConstraintBasedAssignment.computeAutoBestStateForPartition(cache, _stateModelDef,
-                preferenceList, currentStateMap, disabled);
-        mapResult.put(partition, assignment);
+        Map<String, String> rawCurStateMap = _currentMapping.get(partition);
+        ClusterConfig cluster =
+            new ClusterConfig.Builder(ClusterId.from("cluster")).addStateModelDefinition(
+                _stateModelDef).build();
+        Map<ParticipantId, Participant> liveParticipantMap = Maps.newHashMap();
+        for (String node : _liveNodes) {
+          Set<String> tags = Collections.emptySet();
+          Map<MessageId, Message> messageMap = Collections.emptyMap();
+          Set<PartitionId> disabledPartitionIdSet = Collections.emptySet();
+          Map<ResourceId, CurrentState> currentStateMap = Maps.newHashMap();
+          RunningInstance runningInstance =
+              new RunningInstance(SessionId.from("testSession"), HelixVersion.from("1.2.3.4"),
+                  ProcId.from("1234"));
+          Participant participant =
+              new Participant(ParticipantId.from(node), node, 0, true, disabledPartitionIdSet,
+                  tags, runningInstance, currentStateMap, messageMap, new UserConfig(
+                      Scope.participant(ParticipantId.from(node))));
+          liveParticipantMap.put(participant.getId(), participant);
+        }
+        List<ParticipantId> preferenceList =
+            IdealState.preferenceListFromStringList(listResult.get(partition));
+        Set<ParticipantId> disabledParticipantsForPartition = Collections.emptySet();
+        Map<ParticipantId, State> currentStateMap =
+            IdealState.participantStateMapFromStringMap(rawCurStateMap);
+        Map<ParticipantId, State> assignment =
+            NewConstraintBasedAssignment.computeAutoBestStateForPartition(cluster,
+                ResourceId.from(RESOURCE_NAME), liveParticipantMap, _stateModelDef, preferenceList,
+                currentStateMap, disabledParticipantsForPartition);
+        mapResult.put(partition, IdealState.stringMapFromParticipantStateMap(assignment));
       }
       return mapResult;
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
index 7eb811b..e07e7d7 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
@@ -35,17 +35,17 @@ import java.util.TreeSet;
 
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ClusterConfig;
-import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.MessageId;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.UserConfig;
+import org.apache.helix.api.config.ClusterConfig;
+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;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
 import org.apache.helix.model.CurrentState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
index 5325e30..d6c6d01 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.controller.ClusterController;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
index a5001ed..09a0b1c 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
@@ -29,9 +29,9 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
 import org.apache.helix.controller.rebalancer.context.Rebalancer;
 import org.apache.helix.controller.rebalancer.context.RebalancerConfig;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
index a3dea87..ade7ca1 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
@@ -24,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
index 7f4335f..dbe183d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
@@ -28,7 +28,7 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockParticipant;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
index e09b7e8..3613cb8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
@@ -26,12 +26,12 @@ import java.util.UUID;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
index d33fea6..a947445 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle2.java
@@ -41,7 +41,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
index 7809fcb..a3d6237 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
@@ -26,8 +26,8 @@ import org.apache.helix.Criteria;
 import org.apache.helix.Criteria.DataSource;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
index aded9b3..07a2fc0 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
@@ -24,10 +24,10 @@ import java.util.UUID;
 import org.apache.helix.Criteria;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory;
 import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError;
 import org.apache.helix.model.ExternalView;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
index a0330d4..cba5459 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestRenamePartition.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
index 90d317c..292eca3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMessage.java
@@ -40,8 +40,8 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
index 690ade3..aa91589 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgContraints.java
@@ -16,8 +16,8 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
index fe0cb48..d922843 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSchedulerMsgUsingQueue.java
@@ -14,8 +14,8 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
index 99c20b4..1b69572 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestSessionExpiryInTransition.java
@@ -27,7 +27,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockTransition;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
index 4fff890..451972a 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStateTransitionTimeout.java
@@ -31,9 +31,9 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 2be136d..5ce6664 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
@@ -31,11 +31,11 @@ import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.participant.statemachine.StateModel;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
index 8ebcd44..a818fd3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
@@ -31,7 +31,7 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ControllerManager;
 import org.apache.helix.manager.zk.ParticipantManager;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
index 3c8308f..4cef5a0 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestDefaultControllerMsgHandlerFactory.java
@@ -25,7 +25,7 @@ import java.util.Map;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.apache.helix.manager.zk.DefaultControllerMessageHandlerFactory.DefaultControllerMessageHandler;
 import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java
index cd65a42..d83bd46 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZNRecordSizeLimit.java
@@ -27,7 +27,7 @@ import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java
index 187a077..a81079a 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkHelixAdmin.java
@@ -33,7 +33,7 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.ConstraintId;
+import org.apache.helix.api.id.ConstraintId;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java
index b12a750..52d43ce 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallback.java
@@ -24,7 +24,7 @@ import java.util.Date;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.apache.helix.model.Message;
 import org.testng.AssertJUnit;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java
index 3be41f8..8183e2c 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/TestAsyncCallbackSvc.java
@@ -30,8 +30,8 @@ import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.Mocks;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.handling.AsyncCallbackService;
 import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java b/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java
index cc4eaab..11bdeb5 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/TestDefaultMessagingService.java
@@ -33,7 +33,7 @@ import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
 import org.apache.helix.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java
index 8e33e6d..3e6ff47 100644
--- a/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java
+++ b/helix-core/src/test/java/org/apache/helix/messaging/handling/TestHelixTaskExecutor.java
@@ -29,8 +29,8 @@ import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.Mocks;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
index 44ef775..61e727c 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/controller/MockController.java
@@ -30,10 +30,10 @@ import java.util.TreeMap;
 
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java b/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
index 04e56da..193abd3 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/controller/MockControllerProcess.java
@@ -22,7 +22,7 @@ package org.apache.helix.mock.controller;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java b/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java
index 95ecfdb..9880605 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/participant/DummyProcess.java
@@ -32,7 +32,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.participant.statemachine.StateModel;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java b/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java
index 51fd399..0a744c2 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/participant/ErrTransition.java
@@ -25,8 +25,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 
 // simulate error transition

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java
index 731f3b8..6c7ac48 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/participant/StoreAccessOneNodeTransition.java
@@ -25,7 +25,7 @@ import org.apache.helix.AccessOption;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
index b1fc1cd..691492e 100644
--- a/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/model/TestConstraint.java
@@ -29,7 +29,7 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java b/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java
index 170578a..e00f245 100644
--- a/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/model/TestIdealState.java
@@ -29,9 +29,9 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.TestHelper;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.IdealState.IdealStateModeProperty;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
index 5ac9829..32b9ec9 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/TestDistControllerStateModel.java
@@ -22,8 +22,8 @@ package org.apache.helix.participant;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
 import org.testng.annotations.BeforeMethod;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
index a9679de..223ae53 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestHelixAdminCli.java
@@ -29,8 +29,8 @@ import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
index 001f599..f0922f3 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/BootstrapHandler.java
@@ -26,7 +26,7 @@ import org.apache.helix.Criteria;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.MessageId;
+import org.apache.helix.api.id.MessageId;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java b/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java
index 5b590f0..c6ab3a4 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/DummyParticipant.java
@@ -23,7 +23,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.Message;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.participant.statemachine.StateModel;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java b/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
index 72ea9f8..cb5906f 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/NewModelExample.java
@@ -6,19 +6,19 @@ import java.util.concurrent.TimeUnit;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ClusterAccessor;
-import org.apache.helix.api.ClusterConfig;
-import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.ParticipantConfig;
-import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.UserConfig;
+import org.apache.helix.api.accessor.ClusterAccessor;
+import org.apache.helix.api.config.ClusterConfig;
+import org.apache.helix.api.config.ParticipantConfig;
+import org.apache.helix.api.config.ResourceConfig;
+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;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.rebalancer.context.FullAutoRebalancerContext;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
index f157aac..11d4fa4 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/Quickstart.java
@@ -33,7 +33,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.model.ExternalView;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/TaskStateModel.java
----------------------------------------------------------------------
diff --git a/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/TaskStateModel.java b/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/TaskStateModel.java
index 713e0ea..3c1cab4 100644
--- a/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/TaskStateModel.java
+++ b/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/TaskStateModel.java
@@ -24,7 +24,7 @@ import java.util.Set;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.Message;


[2/6] [HELIX-238] Refactor, add update to accessors, test update logic

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java
index aaeeb34..7720143 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java
@@ -3,7 +3,7 @@ package org.apache.helix.controller.stages;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.ResourceAssignment;
 
 import com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java
index 7478609..ea1a507 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java
@@ -26,7 +26,7 @@ import org.apache.helix.HelixManagerProperties;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.HelixVersion;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
+import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java
index 1a92919..f1c9323 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java
@@ -23,16 +23,16 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.MessageId;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.CurrentState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
index 477247e..8ff52bd 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
@@ -35,13 +35,13 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.ZNRecordDelta;
 import org.apache.helix.ZNRecordDelta.MergeOperation;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SchedulerTaskConfig;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.config.SchedulerTaskConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.context.RebalancerConfig;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
index ad0ad95..fb6dfe8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
@@ -27,16 +27,16 @@ import java.util.UUID;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SchedulerTaskConfig;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.config.SchedulerTaskConfig;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.context.RebalancerContext;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java
index 21f5188..89231c2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java
@@ -25,8 +25,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.Message;
 
 public class NewMessageOutput {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java
index 04d6af8..4adfbcb 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java
@@ -28,14 +28,14 @@ import java.util.TreeMap;
 
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.Resource;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.context.RebalancerConfig;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java
index 8c3c847..dfea7fc 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java
@@ -28,10 +28,10 @@ import java.util.Set;
 
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.ClusterConstraints;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
index d10e155..26050f8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
@@ -22,8 +22,8 @@ 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.ClusterAccessor;
-import org.apache.helix.api.ClusterId;
+import org.apache.helix.api.accessor.ClusterAccessor;
+import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
index 38da0ac..e839a98 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
@@ -25,11 +25,11 @@ import java.util.Map;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.Resource;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
index 74b29c7..d8fa81c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
@@ -32,10 +32,10 @@ import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 f982847..be0b7f0 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
@@ -2,8 +2,8 @@ package org.apache.helix.controller.stages;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.ResourceAccessor;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.accessor.ResourceAccessor;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.model.ResourceAssignment;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
index e3e8f94..3dd3b81 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceCurrentState.java
@@ -24,11 +24,11 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.CurrentState;
 
 public class ResourceCurrentState {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 e0f3361..a50f76b 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
@@ -30,7 +30,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerProperties;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.LiveInstance;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java
index 6a7bb59..06bcbed 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/EspressoRelayStrategy.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.helix.HelixException;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.IdealState;
 
 public class EspressoRelayStrategy {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
index 482d3cf..bd98632 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
@@ -21,9 +21,9 @@ package org.apache.helix.manager.zk;
 
 import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.CurrentState;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 9d61914..38a067b 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
@@ -36,11 +36,11 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 14c63f4..3c5502d 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
@@ -53,14 +53,14 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertsHolder;
 import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.api.ConstraintId;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.ConstraintId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
 import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConstraints;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 f5afc59..a207b0c 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
@@ -33,10 +33,10 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.messaging.handling.AsyncCallbackService;
 import org.apache.helix.messaging.handling.HelixTaskExecutor;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 ab8fe8a..5bb67fd 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
@@ -35,8 +35,8 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.Attributes;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java
index 5d67d5c..81b5155 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/GroupMessageHandler.java
@@ -26,7 +26,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.helix.PropertyKey;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.Attributes;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 6c6cf8e..c6eaa65 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
@@ -39,10 +39,10 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecordBucketizer;
 import org.apache.helix.ZNRecordDelta;
 import org.apache.helix.ZNRecordDelta.MergeOperation;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.Attributes;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 af58c98..8cf1aa7 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
@@ -45,8 +45,8 @@ import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
index 8104bce..99981b4 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
@@ -2,8 +2,8 @@ package org.apache.helix.model;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.UserConfig;
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.ClusterId;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java
index 2b014ae..8e37b18 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConstraints.java
@@ -27,7 +27,7 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ConstraintId;
+import org.apache.helix.api.id.ConstraintId;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 a828117..2fe37ce 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
@@ -26,11 +26,11 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
index f4304f4..77df3cf 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
@@ -25,10 +25,10 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/IdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/IdealState.java b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
index ba62391..342acf2 100644
--- a/helix-core/src/main/java/org/apache/helix/model/IdealState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
@@ -32,12 +32,12 @@ import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.rebalancer.context.RebalancerRef;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 89b043b..d2e1187 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
@@ -27,8 +27,8 @@ import java.util.Set;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
index 70822ac..fda144a 100644
--- a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
+++ b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
@@ -22,9 +22,9 @@ package org.apache.helix.model;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.HelixVersion;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.ProcId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.ProcId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 b372ac2..71e9696 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
@@ -34,13 +34,13 @@ 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.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.controller.stages.ClusterEvent;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
index 45a78c0..c2741c8 100644
--- a/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
@@ -2,8 +2,8 @@ package org.apache.helix.model;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.UserConfig;
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.PartitionId;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
index 4ab21ec..c478e13 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
@@ -26,10 +26,10 @@ import java.util.Map;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
index c753f22..230a78b 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
@@ -2,8 +2,8 @@ package org.apache.helix.model;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ResourceConfig.ResourceType;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.config.ResourceConfig.ResourceType;
+import org.apache.helix.api.id.ResourceId;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 41aa929..32ee2c7 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
@@ -31,7 +31,7 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.builder.StateTransitionTableBuilder;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java
index 7a0e7a0..f329daa 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/ClusterConstraintsBuilder.java
@@ -22,7 +22,7 @@ package org.apache.helix.model.builder;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.helix.api.ConstraintId;
+import org.apache.helix.api.id.ConstraintId;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java
index 26238f2..0519979 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/CurrentStateBuilder.java
@@ -4,11 +4,11 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.CurrentState.CurrentStateProperty;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java
index 3eb1afb..a6707e9 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/IdealStateBuilder.java
@@ -22,9 +22,9 @@ package org.apache.helix.model.builder;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.model.IdealState;
 
 public abstract class IdealStateBuilder {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java
index a44315b..da463e1 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/MessageConstraintItemBuilder.java
@@ -1,7 +1,7 @@
 package org.apache.helix.model.builder;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ConstraintItem;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java
index 9b0f18a..8dfb0e5 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/ResourceAssignmentBuilder.java
@@ -3,10 +3,10 @@ package org.apache.helix.model.builder;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.ResourceAssignment;
 
 /*

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java
index d9af9f5..9a2678a 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/StateConstraintItemBuilder.java
@@ -1,7 +1,7 @@
 package org.apache.helix.model.builder;
 
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
 import org.apache.helix.model.ConstraintItem;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 7dcbfda..83b93d8 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
@@ -22,9 +22,9 @@ package org.apache.helix.monitoring.mbeans;
 import java.util.Map;
 
 import org.apache.helix.HelixDefinedState;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java b/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java
index cd238e1..c4eceb2 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/HelixCustomCodeRunner.java
@@ -29,8 +29,8 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 5812b1d..2258b95 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
@@ -31,12 +31,12 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.messaging.handling.BatchMessageHandler;
 import org.apache.helix.messaging.handling.BatchMessageWrapper;
 import org.apache.helix.messaging.handling.HelixStateTransitionHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
index 3370c99..ea03d58 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
@@ -44,12 +44,12 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ClusterAccessor;
-import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.accessor.ClusterAccessor;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.Stage;
 import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.stages.AttributeName;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java b/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java
index 86d2eb8..f3ed88e 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/MessagePoster.java
@@ -22,10 +22,10 @@ package org.apache.helix.tools;
 import java.util.UUID;
 
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.LiveInstance.LiveInstanceProperty;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java b/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java
index 4b014de..e46ad13 100644
--- a/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java
+++ b/helix-core/src/main/java/org/apache/helix/util/RebalanceUtil.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.helix.HelixException;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.StateModelDefinition;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 d371d1a..6e7db19 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
@@ -38,7 +38,7 @@ import org.apache.helix.HelixProperty;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.Error;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java b/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
index 89c7a04..74d1d53 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
@@ -23,12 +23,12 @@ import org.apache.helix.Mocks.MockHelixTaskExecutor;
 import org.apache.helix.Mocks.MockManager;
 import org.apache.helix.Mocks.MockStateModel;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.messaging.handling.AsyncCallbackService;
 import org.apache.helix.messaging.handling.HelixStateTransitionHandler;
 import org.apache.helix.messaging.handling.HelixTask;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java b/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
index 34bb9da..07de562 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
@@ -25,12 +25,12 @@ import org.apache.helix.Mocks.MockManager;
 import org.apache.helix.Mocks.MockStateModel;
 import org.apache.helix.Mocks.MockStateModelAnnotated;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.messaging.handling.HelixStateTransitionHandler;
 import org.apache.helix.messaging.handling.HelixTask;
 import org.apache.helix.messaging.handling.HelixTaskExecutor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/TestHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestHelper.java b/helix-core/src/test/java/org/apache/helix/TestHelper.java
index 681666b..1b307c9 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelper.java
@@ -45,11 +45,11 @@ import org.I0Itec.zkclient.ZkServer;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.apache.commons.io.FileUtils;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.integration.manager.ZkTestManager;
 import org.apache.helix.manager.zk.CallbackHandler;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
index 40a189d..1aac7e6 100644
--- a/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
+++ b/helix-core/src/test/java/org/apache/helix/TestZKCallback.java
@@ -25,12 +25,12 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.CurrentState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
index 7eb53fe..e5b4452 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
@@ -28,10 +28,10 @@ import org.I0Itec.zkclient.IZkStateListener;
 import org.I0Itec.zkclient.ZkConnection;
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.Pipeline;
 import org.apache.helix.controller.pipeline.Stage;
 import org.apache.helix.controller.pipeline.StageContext;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/api/TestId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/TestId.java b/helix-core/src/test/java/org/apache/helix/api/TestId.java
index aa870f3..24b6b3b 100644
--- a/helix-core/src/test/java/org/apache/helix/api/TestId.java
+++ b/helix-core/src/test/java/org/apache/helix/api/TestId.java
@@ -1,5 +1,14 @@
 package org.apache.helix.api;
 
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ProcId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java b/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java
index 29370cb..5a14287 100644
--- a/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java
+++ b/helix-core/src/test/java/org/apache/helix/api/TestNamespacedConfig.java
@@ -3,6 +3,8 @@ package org.apache.helix.api;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.InstanceConfig.InstanceConfigProperty;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
index 5775f9e..f5e22d9 100644
--- a/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
+++ b/helix-core/src/test/java/org/apache/helix/api/TestNewStages.java
@@ -27,6 +27,12 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
+import org.apache.helix.api.accessor.ClusterAccessor;
+import org.apache.helix.api.config.ResourceConfig;
+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;
 import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
 import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.ClusterEvent;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java b/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java
new file mode 100644
index 0000000..7999ab4
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java
@@ -0,0 +1,157 @@
+package org.apache.helix.api;
+
+import org.apache.helix.api.config.ClusterConfig;
+import org.apache.helix.api.config.ParticipantConfig;
+import org.apache.helix.api.config.ResourceConfig;
+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;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.controller.rebalancer.context.FullAutoRebalancerContext;
+import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/*
+ * 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.
+ */
+
+/**
+ * Testing the deltas of the various config classes. They should be able to update corresponding
+ * configs correctly
+ */
+public class TestUpdateConfig {
+  @Test
+  public void testParticipantConfigUpdate() {
+    final String ORIG_HOSTNAME = "host1";
+    final String NEW_HOSTNAME = "host2";
+    final int PORT = 1234;
+    final String TAG1 = "tag1";
+    final String TAG2 = "tag2";
+    final String TAG3 = "tag3";
+    final PartitionId partition1 = PartitionId.from("resource_1");
+    final PartitionId partition2 = PartitionId.from("resource_2");
+    final PartitionId partition3 = PartitionId.from("resource_3");
+    final ParticipantId participantId = ParticipantId.from("participant");
+
+    // start: add a user config, set host & port, add 2 tags and 2 disabled partition, start
+    // disabled
+    UserConfig userConfig = new UserConfig(Scope.participant(participantId));
+    userConfig.setSimpleField("key1", "value1");
+    ParticipantConfig config =
+        new ParticipantConfig.Builder(participantId).hostName(ORIG_HOSTNAME).port(PORT)
+            .enabled(false).addTag(TAG1).addTag(TAG2).addDisabledPartition(partition1)
+            .addDisabledPartition(partition2).userConfig(userConfig).build();
+    UserConfig newUserConfig = new UserConfig(Scope.participant(participantId));
+    newUserConfig.setSimpleField("key2", "value2");
+
+    // update: change host, remove a tag, add a tag, remove a disabled partition, add a disabled
+    // partition, change user config
+    ParticipantConfig updated =
+        new ParticipantConfig.Delta(participantId).setHostName(NEW_HOSTNAME).removeTag(TAG1)
+            .addTag(TAG3).removeDisabledPartition(partition1).addDisabledPartition(partition3)
+            .setUserConfig(newUserConfig).mergeInto(config);
+    Assert.assertEquals(updated.getHostName(), NEW_HOSTNAME);
+    Assert.assertEquals(updated.getPort(), PORT);
+    Assert.assertFalse(updated.hasTag(TAG1));
+    Assert.assertTrue(updated.hasTag(TAG2));
+    Assert.assertTrue(updated.hasTag(TAG3));
+    Assert.assertFalse(updated.getDisabledPartitions().contains(partition1));
+    Assert.assertTrue(updated.getDisabledPartitions().contains(partition2));
+    Assert.assertTrue(updated.getDisabledPartitions().contains(partition3));
+    Assert.assertNull(updated.getUserConfig().getSimpleField("key1"));
+    Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2");
+    Assert.assertFalse(updated.isEnabled());
+  }
+
+  @Test
+  public void testResourceConfigUpdate() {
+    final int OLD_BUCKET_SIZE = 0;
+    final int NEW_BUCKET_SIZE = 1;
+    final ResourceId resourceId = ResourceId.from("resource");
+
+    // start: add a user config, a semi auto rebalancer context, and set bucket size and batch
+    // message mode
+    UserConfig userConfig = new UserConfig(Scope.resource(resourceId));
+    userConfig.setSimpleField("key1", "value1");
+    SemiAutoRebalancerContext rebalancerContext =
+        new SemiAutoRebalancerContext.Builder(resourceId).build();
+    ResourceConfig config =
+        new ResourceConfig.Builder(resourceId).userConfig(userConfig)
+            .rebalancerContext(rebalancerContext).bucketSize(OLD_BUCKET_SIZE)
+            .batchMessageMode(true).build();
+
+    // update: overwrite user config, change to full auto rebalancer context, and change the bucket
+    // size
+    UserConfig newUserConfig = new UserConfig(Scope.resource(resourceId));
+    newUserConfig.setSimpleField("key2", "value2");
+    FullAutoRebalancerContext newRebalancerContext =
+        new FullAutoRebalancerContext.Builder(resourceId).build();
+    ResourceConfig updated =
+        new ResourceConfig.Delta(resourceId).setBucketSize(NEW_BUCKET_SIZE)
+            .setUserConfig(newUserConfig).setRebalancerContext(newRebalancerContext)
+            .mergeInto(config);
+    Assert.assertEquals(updated.getBucketSize(), NEW_BUCKET_SIZE);
+    Assert.assertTrue(updated.getBatchMessageMode());
+    Assert.assertNull(updated.getRebalancerConfig().getRebalancerContext(
+        SemiAutoRebalancerContext.class));
+    Assert.assertNotNull(updated.getRebalancerConfig().getRebalancerContext(
+        FullAutoRebalancerContext.class));
+    Assert.assertNull(updated.getUserConfig().getSimpleField("key1"));
+    Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2");
+  }
+
+  @Test
+  public void testClusterConfigUpdate() {
+    final ClusterId clusterId = ClusterId.from("cluster");
+    final StateModelDefId masterSlave = StateModelDefId.from("MasterSlave");
+    final State master = State.from("MASTER");
+    final State slave = State.from("SLAVE");
+    final State offline = State.from("OFFLINE");
+
+    // start: add a user config, add master and slave constraints
+    UserConfig userConfig = new UserConfig(Scope.cluster(clusterId));
+    userConfig.setSimpleField("key1", "value1");
+    ClusterConfig config =
+        new ClusterConfig.Builder(clusterId)
+            .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master, 2)
+            .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave, 3)
+            .userConfig(userConfig).build();
+
+    // update: overwrite user config, change master constraint, remove slave constraint, add offline
+    // constraint
+    UserConfig newUserConfig = new UserConfig(Scope.cluster(clusterId));
+    newUserConfig.setSimpleField("key2", "value2");
+    ClusterConfig updated =
+        new ClusterConfig.Delta(clusterId)
+            .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master, 1)
+            .removeStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave)
+            .addStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, offline, "R")
+            .setUserConfig(newUserConfig).mergeInto(config);
+    Assert.assertEquals(
+        updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, master), "1");
+    Assert.assertEquals(
+        updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, slave), "-1");
+    Assert.assertEquals(
+        updated.getStateUpperBoundConstraint(Scope.cluster(clusterId), masterSlave, offline), "R");
+    Assert.assertNull(updated.getUserConfig().getSimpleField("key1"));
+    Assert.assertEquals(updated.getUserConfig().getSimpleField("key2"), "value2");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java
index ebdcaff..5bbe54f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/context/TestSerializeRebalancerContext.java
@@ -2,12 +2,12 @@ package org.apache.helix.controller.rebalancer.context;
 
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.ResourceConfiguration;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 141f6b7..ecb8151 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
@@ -31,14 +31,14 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.Mocks;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.Resource;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceConfig.ResourceType;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.Scope;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.UserConfig;
+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.ParticipantId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.Stage;
 import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java
index a8783c3..18e8f4d 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleCalcStageCompatibility.java
@@ -26,12 +26,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.IdealStateModeProperty;
 import org.testng.AssertJUnit;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java
index 4874a83..d3f348e 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestBestPossibleStateCalcStage.java
@@ -23,11 +23,11 @@ import java.util.Date;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.testng.AssertJUnit;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java
index 28b8a19..fb113b9 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCompatibilityCheckStage.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.helix.Mocks;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
 import org.apache.helix.model.IdealState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java
index 7176797..3412e0a 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestCurrentStateComputationStage.java
@@ -24,13 +24,13 @@ import java.util.Map;
 
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
index 9f31609..450d654 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
@@ -31,9 +31,9 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.Pipeline;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 ff2e292..113e1af 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
@@ -29,17 +29,17 @@ import java.util.Set;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.HelixVersion;
-import org.apache.helix.api.MessageId;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ProcId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.RunningInstance;
 import org.apache.helix.api.Scope;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.UserConfig;
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ProcId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.controller.stages.NewMessageSelectionStage.Bounds;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
index f7d2317..5a7c6ac 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
@@ -29,10 +29,10 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.pipeline.Pipeline;
 import org.apache.helix.manager.zk.ZKHelixAdmin;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 ec6c525..20fb0c9 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
@@ -27,11 +27,11 @@ import java.util.UUID;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.rebalancer.context.RebalancerContext;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;


[3/6] [HELIX-238] Refactor, add update to accessors, test update logic

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
new file mode 100644
index 0000000..c6f9d19
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
@@ -0,0 +1,228 @@
+package org.apache.helix.api.config;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.api.Scope;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+/*
+ * 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.
+ */
+
+/**
+ * Generic configuration of Helix components prefixed with a namespace
+ */
+public class NamespacedConfig extends ZNRecord {
+  private static final char PREFIX_CHAR = '!';
+  private final String _prefix;
+
+  /**
+   * Instantiate a NamespacedConfig. It is intended for use only by entities that can be identified
+   * @param scope scope object
+   */
+  public NamespacedConfig(Scope<?> scope, String prefix) {
+    super(scope.getScopedId().stringify());
+    _prefix = prefix + PREFIX_CHAR;
+  }
+
+  /**
+   * Instantiate a NamespacedConfig from an existing HelixProperty
+   * @param property property wrapping a configuration
+   */
+  public NamespacedConfig(HelixProperty property, String prefix) {
+    super(property.getRecord());
+    _prefix = prefix + PREFIX_CHAR;
+    filterNonPrefixedFields();
+  }
+
+  /**
+   * Instantiate a NamespacedConfig as a copy of another NamedspacedConfig
+   * @param config populated NamespacedConfig
+   */
+  public NamespacedConfig(NamespacedConfig config) {
+    super(config.getId());
+    _prefix = config.getPrefix() + PREFIX_CHAR;
+    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
+      setRawPayload(config.getRawPayload());
+      setPayloadSerializer(config.getPayloadSerializer());
+    }
+    super.setSimpleFields(config.getPrefixedSimpleFields());
+    super.setListFields(config.getPrefixedListFields());
+    super.setMapFields(config.getPrefixedMapFields());
+  }
+
+  @Override
+  public void setMapField(String k, Map<String, String> v) {
+    super.setMapField(_prefix + k, v);
+  }
+
+  @Override
+  public Map<String, String> getMapField(String k) {
+    return super.getMapField(_prefix + k);
+  }
+
+  @Override
+  public void setMapFields(Map<String, Map<String, String>> mapFields) {
+    for (String k : mapFields.keySet()) {
+      super.setMapField(_prefix + k, mapFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of map fields
+   */
+  @Override
+  public Map<String, Map<String, String>> getMapFields() {
+    return convertToPrefixlessMap(super.getMapFields(), _prefix);
+  }
+
+  @Override
+  public void setListField(String k, List<String> v) {
+    super.setListField(_prefix + k, v);
+  }
+
+  @Override
+  public List<String> getListField(String k) {
+    return super.getListField(_prefix + k);
+  }
+
+  @Override
+  public void setListFields(Map<String, List<String>> listFields) {
+    for (String k : listFields.keySet()) {
+      super.setListField(_prefix + k, listFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of list fields
+   */
+  @Override
+  public Map<String, List<String>> getListFields() {
+    return convertToPrefixlessMap(super.getListFields(), _prefix);
+  }
+
+  @Override
+  public void setSimpleField(String k, String v) {
+    super.setSimpleField(_prefix + k, v);
+  }
+
+  @Override
+  public String getSimpleField(String k) {
+    return super.getSimpleField(_prefix + k);
+  }
+
+  @Override
+  public void setSimpleFields(Map<String, String> simpleFields) {
+    for (String k : simpleFields.keySet()) {
+      super.setSimpleField(_prefix + k, simpleFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of simple fields
+   */
+  @Override
+  public Map<String, String> getSimpleFields() {
+    return convertToPrefixlessMap(super.getSimpleFields(), _prefix);
+  }
+
+  /**
+   * Get the prefix used to distinguish these config properties
+   * @return string prefix, not including the underscore
+   */
+  public String getPrefix() {
+    return _prefix.substring(0, _prefix.indexOf(PREFIX_CHAR));
+  }
+
+  /**
+   * Remove all fields from this config that are not prefixed
+   */
+  private void filterNonPrefixedFields() {
+    // filter out any configuration that isn't user-defined
+    Predicate<String> keyFilter = new Predicate<String>() {
+      @Override
+      public boolean apply(String key) {
+        return key.contains(_prefix);
+      }
+    };
+    super.setMapFields(Maps.filterKeys(super.getMapFields(), keyFilter));
+    super.setListFields(Maps.filterKeys(super.getListFields(), keyFilter));
+    super.setSimpleFields(Maps.filterKeys(super.getSimpleFields(), keyFilter));
+  }
+
+  /**
+   * Get all map fields with prefixed keys
+   * @return prefixed map fields
+   */
+  private Map<String, Map<String, String>> getPrefixedMapFields() {
+    return super.getMapFields();
+  }
+
+  /**
+   * Get all list fields with prefixed keys
+   * @return prefixed list fields
+   */
+  private Map<String, List<String>> getPrefixedListFields() {
+    return super.getListFields();
+  }
+
+  /**
+   * Get all simple fields with prefixed keys
+   * @return prefixed simple fields
+   */
+  private Map<String, String> getPrefixedSimpleFields() {
+    return super.getSimpleFields();
+  }
+
+  /**
+   * Add user configuration to an existing helix property.
+   * @param property the property to update
+   * @param config the user config
+   */
+  public static void addConfigToProperty(HelixProperty property, NamespacedConfig config) {
+    ZNRecord record = property.getRecord();
+    record.getMapFields().putAll(config.getPrefixedMapFields());
+    record.getListFields().putAll(config.getPrefixedListFields());
+    record.getSimpleFields().putAll(config.getPrefixedSimpleFields());
+    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
+      record.setPayloadSerializer(config.getPayloadSerializer());
+      record.setRawPayload(config.getRawPayload());
+    }
+  }
+
+  /**
+   * Get a copy of a map with the key prefix stripped. The resulting map is immutable
+   * @param rawMap map of key, value pairs where the key is prefixed
+   * @return map of key, value pairs where the key is not prefixed
+   */
+  private static <T> Map<String, T> convertToPrefixlessMap(Map<String, T> rawMap, String prefix) {
+    Map<String, T> convertedMap = new HashMap<String, T>();
+    for (String rawKey : rawMap.keySet()) {
+      String k = rawKey.substring(prefix.length());
+      convertedMap.put(k, rawMap.get(rawKey));
+    }
+    return ImmutableMap.copyOf(convertedMap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
new file mode 100644
index 0000000..1d7b23e
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
@@ -0,0 +1,382 @@
+package org.apache.helix.api.config;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+/*
+ * 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.
+ */
+
+/**
+ * Configuration properties of a Helix participant
+ */
+public class ParticipantConfig {
+  private final ParticipantId _id;
+  private final String _hostName;
+  private final int _port;
+  private final boolean _isEnabled;
+  private final Set<PartitionId> _disabledPartitions;
+  private final Set<String> _tags;
+  private final UserConfig _userConfig;
+
+  /**
+   * Initialize a participant configuration. Also see ParticipantConfig.Builder
+   * @param id participant id
+   * @param hostName host where participant can be reached
+   * @param port port to use to contact participant
+   * @param isEnabled true if enabled, false if disabled
+   * @param disabledPartitions set of partitions, if any to disable on this participant
+   * @param tags tags to set for the participant
+   */
+  public ParticipantConfig(ParticipantId id, String hostName, int port, boolean isEnabled,
+      Set<PartitionId> disabledPartitions, Set<String> tags, UserConfig userConfig) {
+    _id = id;
+    _hostName = hostName;
+    _port = port;
+    _isEnabled = isEnabled;
+    _disabledPartitions = ImmutableSet.copyOf(disabledPartitions);
+    _tags = ImmutableSet.copyOf(tags);
+    _userConfig = userConfig;
+  }
+
+  /**
+   * Get the host name of the participant
+   * @return host name, or null if not applicable
+   */
+  public String getHostName() {
+    return _hostName;
+  }
+
+  /**
+   * Get the port of the participant
+   * @return port number, or -1 if not applicable
+   */
+  public int getPort() {
+    return _port;
+  }
+
+  /**
+   * Get if the participant is enabled
+   * @return true if enabled or false otherwise
+   */
+  public boolean isEnabled() {
+    return _isEnabled;
+  }
+
+  /**
+   * Get disabled partition id's
+   * @return set of disabled partition id's, or empty set if none
+   */
+  public Set<PartitionId> getDisabledPartitions() {
+    return _disabledPartitions;
+  }
+
+  /**
+   * Get tags
+   * @return set of tags
+   */
+  public Set<String> getTags() {
+    return _tags;
+  }
+
+  /**
+   * Check if participant has a tag
+   * @param tag tag to check
+   * @return true if tagged, false otherwise
+   */
+  public boolean hasTag(String tag) {
+    return _tags.contains(tag);
+  }
+
+  /**
+   * Get user-specified configuration properties of this participant
+   * @return UserConfig properties
+   */
+  public UserConfig getUserConfig() {
+    return _userConfig;
+  }
+
+  /**
+   * Get the participant id
+   * @return ParticipantId
+   */
+  public ParticipantId getId() {
+    return _id;
+  }
+
+  /**
+   * Update context for a ParticipantConfig
+   */
+  public static class Delta {
+    private enum Fields {
+      HOST_NAME,
+      PORT,
+      ENABLED,
+      USER_CONFIG
+    }
+
+    private Set<Fields> _updateFields;
+    private Set<String> _removedTags;
+    private Set<PartitionId> _removedDisabledPartitions;
+    private Builder _builder;
+
+    /**
+     * Instantiate the delta for a participant config
+     * @param participantId the participant to update
+     */
+    public Delta(ParticipantId participantId) {
+      _updateFields = Sets.newHashSet();
+      _removedTags = Sets.newHashSet();
+      _removedDisabledPartitions = Sets.newHashSet();
+      _builder = new Builder(participantId);
+    }
+
+    /**
+     * Set the participant host name
+     * @param hostName reachable host when live
+     * @return Delta
+     */
+    public Delta setHostName(String hostName) {
+      _builder.hostName(hostName);
+      _updateFields.add(Fields.HOST_NAME);
+      return this;
+    }
+
+    /**
+     * Set the participant port
+     * @param port port number
+     * @return Delta
+     */
+    public Delta setPort(int port) {
+      _builder.port(port);
+      _updateFields.add(Fields.PORT);
+      return this;
+    }
+
+    /**
+     * <<<<<<< HEAD:helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
+     * Set the enabled status of the participant
+     * @param isEnabled true if enabled, false if disabled
+     *          =======
+     *          Set whether or not the participant is enabled
+     * @param isEnabled true if enabled, false otherwise
+     *          >>>>>>> helix-logical-model:helix-core/src/main/java/org/apache/helix/api/
+     *          ParticipantConfig.java
+     * @return Delta
+     */
+    public Delta setEnabled(boolean isEnabled) {
+      _builder.enabled(isEnabled);
+      _updateFields.add(Fields.ENABLED);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Delta
+     */
+    public Delta setUserConfig(UserConfig userConfig) {
+      _builder.userConfig(userConfig);
+      _updateFields.add(Fields.USER_CONFIG);
+      return this;
+    }
+
+    /**
+     * Add an new tag for this participant
+     * @param tag the tag to add
+     * @return Delta
+     */
+    public Delta addTag(String tag) {
+      _builder.addTag(tag);
+      return this;
+    }
+
+    /**
+     * Remove a tag for this participant
+     * @param tag the tag to remove
+     * @return Delta
+     */
+    public Delta removeTag(String tag) {
+      _removedTags.add(tag);
+      return this;
+    }
+
+    /**
+     * Add a partition to disable for this participant
+     * @param partitionId the partition to disable
+     * @return Delta
+     */
+    public Delta addDisabledPartition(PartitionId partitionId) {
+      _builder.addDisabledPartition(partitionId);
+      return this;
+    }
+
+    /**
+     * Remove a partition from the disabled set for this participant
+     * @param partitionId the partition to enable
+     * @return Delta
+     */
+    public Delta removeDisabledPartition(PartitionId partitionId) {
+      _removedDisabledPartitions.add(partitionId);
+      return this;
+    }
+
+    /**
+     * Create a ParticipantConfig that is the combination of an existing ParticipantConfig and this
+     * delta
+     * @param orig the original ParticipantConfig
+     * @return updated ParticipantConfig
+     */
+    public ParticipantConfig mergeInto(ParticipantConfig orig) {
+      ParticipantConfig deltaConfig = _builder.build();
+      Builder builder =
+          new Builder(orig.getId()).hostName(orig.getHostName()).port(orig.getPort())
+              .enabled(orig.isEnabled()).userConfig(orig.getUserConfig());
+      for (Fields field : _updateFields) {
+        switch (field) {
+        case HOST_NAME:
+          builder.hostName(deltaConfig.getHostName());
+          break;
+        case PORT:
+          builder.port(deltaConfig.getPort());
+          break;
+        case ENABLED:
+          builder.enabled(deltaConfig.isEnabled());
+          break;
+        case USER_CONFIG:
+          builder.userConfig(deltaConfig.getUserConfig());
+          break;
+        }
+      }
+      Set<String> tags = Sets.newHashSet(orig.getTags());
+      tags.addAll(deltaConfig.getTags());
+      tags.removeAll(_removedTags);
+      for (String tag : tags) {
+        builder.addTag(tag);
+      }
+      Set<PartitionId> disabledPartitions = Sets.newHashSet(orig.getDisabledPartitions());
+      disabledPartitions.addAll(deltaConfig.getDisabledPartitions());
+      disabledPartitions.removeAll(_removedDisabledPartitions);
+      for (PartitionId partitionId : disabledPartitions) {
+        builder.addDisabledPartition(partitionId);
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Assemble a participant
+   */
+  public static class Builder {
+    private final ParticipantId _id;
+    private String _hostName;
+    private int _port;
+    private boolean _isEnabled;
+    private final Set<PartitionId> _disabledPartitions;
+    private final Set<String> _tags;
+    private UserConfig _userConfig;
+
+    /**
+     * Build a participant with a given id
+     * @param id participant id
+     */
+    public Builder(ParticipantId id) {
+      _id = id;
+      _disabledPartitions = new HashSet<PartitionId>();
+      _tags = new HashSet<String>();
+      _isEnabled = true;
+      _userConfig = new UserConfig(Scope.participant(id));
+    }
+
+    /**
+     * Set the participant host name
+     * @param hostName reachable host when live
+     * @return Builder
+     */
+    public Builder hostName(String hostName) {
+      _hostName = hostName;
+      return this;
+    }
+
+    /**
+     * Set the participant port
+     * @param port port number
+     * @return Builder
+     */
+    public Builder port(int port) {
+      _port = port;
+      return this;
+    }
+
+    /**
+     * Set whether or not the participant is enabled
+     * @param isEnabled true if enabled, false otherwise
+     * @return Builder
+     */
+    public Builder enabled(boolean isEnabled) {
+      _isEnabled = isEnabled;
+      return this;
+    }
+
+    /**
+     * Add a partition to disable for this participant
+     * @param partitionId the partition to disable
+     * @return Builder
+     */
+    public Builder addDisabledPartition(PartitionId partitionId) {
+      _disabledPartitions.add(partitionId);
+      return this;
+    }
+
+    /**
+     * Add an arbitrary tag for this participant
+     * @param tag the tag to add
+     * @return Builder
+     */
+    public Builder addTag(String tag) {
+      _tags.add(tag);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Builder userConfig(UserConfig userConfig) {
+      _userConfig = userConfig;
+      return this;
+    }
+
+    /**
+     * Assemble the participant
+     * @return instantiated Participant
+     */
+    public ParticipantConfig build() {
+      return new ParticipantConfig(_id, _hostName, _port, _isEnabled, _disabledPartitions, _tags,
+          _userConfig);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
new file mode 100644
index 0000000..38d48ab
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
@@ -0,0 +1,373 @@
+package org.apache.helix.api.config;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.api.Partition;
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+
+import com.google.common.collect.Sets;
+
+/*
+ * 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.
+ */
+
+/**
+ * Full configuration of a Helix resource. Typically used to add or modify resources on a cluster
+ */
+public class ResourceConfig {
+  /**
+   * Type of a resource. A resource is any entity that can be managed by Helix.
+   */
+  public enum ResourceType {
+    /**
+     * A resource that is persistent, and potentially partitioned and replicated.
+     */
+    DATA
+  }
+
+  private final ResourceId _id;
+  private final RebalancerConfig _rebalancerConfig;
+  private final SchedulerTaskConfig _schedulerTaskConfig;
+  private final UserConfig _userConfig;
+  private final int _bucketSize;
+  private final boolean _batchMessageMode;
+  private final ResourceType _resourceType;
+
+  /**
+   * Instantiate a configuration. Consider using ResourceConfig.Builder
+   * @param id resource id
+   * @param partitionMap map of partition identifiers to partition objects
+   * @param schedulerTaskConfig configuration for scheduler tasks associated with the resource
+   * @param rebalancerConfig configuration for rebalancing the resource
+   * @param userConfig user-defined resource properties
+   * @param bucketSize bucket size for this resource
+   * @param batchMessageMode whether or not batch messaging is allowed
+   */
+  public ResourceConfig(ResourceId id, ResourceType resourceType,
+      SchedulerTaskConfig schedulerTaskConfig, RebalancerConfig rebalancerConfig,
+      UserConfig userConfig, int bucketSize, boolean batchMessageMode) {
+    _id = id;
+    _resourceType = resourceType;
+    _schedulerTaskConfig = schedulerTaskConfig;
+    _rebalancerConfig = rebalancerConfig;
+    _userConfig = userConfig;
+    _bucketSize = bucketSize;
+    _batchMessageMode = batchMessageMode;
+  }
+
+  /**
+   * Get the subunits of the resource
+   * @return map of subunit id to subunit or empty map if none
+   */
+  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap() {
+    return _rebalancerConfig.getRebalancerContext(RebalancerContext.class).getSubUnitMap();
+  }
+
+  /**
+   * Get a subunit that the resource contains
+   * @param subUnitId the subunit id to look up
+   * @return Partition or null if none is present with the given id
+   */
+  public Partition getSubUnit(PartitionId subUnitId) {
+    return getSubUnitMap().get(subUnitId);
+  }
+
+  /**
+   * Get the set of subunit ids that the resource contains
+   * @return subunit id set, or empty if none
+   */
+  public Set<? extends PartitionId> getSubUnitSet() {
+    return getSubUnitMap().keySet();
+  }
+
+  /**
+   * Get the resource properties configuring rebalancing
+   * @return RebalancerConfig properties
+   */
+  public RebalancerConfig getRebalancerConfig() {
+    return _rebalancerConfig;
+  }
+
+  /**
+   * Get the resource id
+   * @return ResourceId
+   */
+  public ResourceId getId() {
+    return _id;
+  }
+
+  /**
+   * Get the resource type
+   * @return ResourceType
+   */
+  public ResourceType getType() {
+    return _resourceType;
+  }
+
+  /**
+   * Get the properties configuring scheduler tasks
+   * @return SchedulerTaskConfig properties
+   */
+  public SchedulerTaskConfig getSchedulerTaskConfig() {
+    return _schedulerTaskConfig;
+  }
+
+  /**
+   * Get user-specified configuration properties of this resource
+   * @return UserConfig properties
+   */
+  public UserConfig getUserConfig() {
+    return _userConfig;
+  }
+
+  /**
+   * Get the bucket size for this resource
+   * @return bucket size
+   */
+  public int getBucketSize() {
+    return _bucketSize;
+  }
+
+  /**
+   * Get the batch message mode
+   * @return true if enabled, false if disabled
+   */
+  public boolean getBatchMessageMode() {
+    return _batchMessageMode;
+  }
+
+  @Override
+  public String toString() {
+    return getSubUnitMap().toString();
+  }
+
+  /**
+   * Update context for a ResourceConfig
+   */
+  public static class Delta {
+    private enum Fields {
+      TYPE,
+      REBALANCER_CONTEXT,
+      USER_CONFIG,
+      BUCKET_SIZE,
+      BATCH_MESSAGE_MODE
+    }
+
+    private Set<Fields> _updateFields;
+    private Builder _builder;
+
+    /**
+     * Instantiate the delta for a resource config
+     * @param resourceId the resource to update
+     */
+    public Delta(ResourceId resourceId) {
+      _builder = new Builder(resourceId);
+      _updateFields = Sets.newHashSet();
+    }
+
+    /**
+     * Set the type of this resource
+     * @param type ResourceType
+     * @return Delta
+     */
+    public Delta setType(ResourceType type) {
+      _builder.type(type);
+      _updateFields.add(Fields.TYPE);
+      return this;
+    }
+
+    /**
+     * Set the rebalancer configuration
+     * @param context properties of interest for rebalancing
+     * @return Delta
+     */
+    public Delta setRebalancerContext(RebalancerContext context) {
+      _builder.rebalancerContext(context);
+      _updateFields.add(Fields.REBALANCER_CONTEXT);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Delta
+     */
+    public Delta setUserConfig(UserConfig userConfig) {
+      _builder.userConfig(userConfig);
+      _updateFields.add(Fields.USER_CONFIG);
+      return this;
+    }
+
+    /**
+     * Set the bucket size
+     * @param bucketSize the size to use
+     * @return Delta
+     */
+    public Delta setBucketSize(int bucketSize) {
+      _builder.bucketSize(bucketSize);
+      _updateFields.add(Fields.BUCKET_SIZE);
+      return this;
+    }
+
+    /**
+     * Set the batch message mode
+     * @param batchMessageMode true to enable, false to disable
+     * @return Delta
+     */
+    public Delta setBatchMessageMode(boolean batchMessageMode) {
+      _builder.batchMessageMode(batchMessageMode);
+      _updateFields.add(Fields.BATCH_MESSAGE_MODE);
+      return this;
+    }
+
+    /**
+     * Create a ResourceConfig that is the combination of an existing ResourceConfig and this delta
+     * @param orig the original ResourceConfig
+     * @return updated ResourceConfig
+     */
+    public ResourceConfig mergeInto(ResourceConfig orig) {
+      ResourceConfig deltaConfig = _builder.build();
+      Builder builder =
+          new Builder(orig.getId())
+              .type(orig.getType())
+              .rebalancerContext(
+                  orig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class))
+              .schedulerTaskConfig(orig.getSchedulerTaskConfig()).userConfig(orig.getUserConfig())
+              .bucketSize(orig.getBucketSize()).batchMessageMode(orig.getBatchMessageMode());
+      for (Fields field : _updateFields) {
+        switch (field) {
+        case TYPE:
+          builder.type(deltaConfig.getType());
+          break;
+        case REBALANCER_CONTEXT:
+          builder.rebalancerContext(deltaConfig.getRebalancerConfig().getRebalancerContext(
+              RebalancerContext.class));
+          break;
+        case USER_CONFIG:
+          builder.userConfig(deltaConfig.getUserConfig());
+          break;
+        case BUCKET_SIZE:
+          builder.bucketSize(deltaConfig.getBucketSize());
+          break;
+        case BATCH_MESSAGE_MODE:
+          builder.batchMessageMode(deltaConfig.getBatchMessageMode());
+          break;
+        }
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Assembles a ResourceConfig
+   */
+  public static class Builder {
+    private final ResourceId _id;
+    private ResourceType _type;
+    private RebalancerConfig _rebalancerConfig;
+    private SchedulerTaskConfig _schedulerTaskConfig;
+    private UserConfig _userConfig;
+    private int _bucketSize;
+    private boolean _batchMessageMode;
+
+    /**
+     * Build a Resource with an id
+     * @param id resource id
+     */
+    public Builder(ResourceId id) {
+      _id = id;
+      _type = ResourceType.DATA;
+      _bucketSize = 0;
+      _batchMessageMode = false;
+      _userConfig = new UserConfig(Scope.resource(id));
+    }
+
+    /**
+     * Set the type of this resource
+     * @param type ResourceType
+     * @return Builder
+     */
+    public Builder type(ResourceType type) {
+      _type = type;
+      return this;
+    }
+
+    /**
+     * Set the rebalancer configuration
+     * @param rebalancerContext properties of interest for rebalancing
+     * @return Builder
+     */
+    public Builder rebalancerContext(RebalancerContext rebalancerContext) {
+      _rebalancerConfig = new RebalancerConfig(rebalancerContext);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Builder userConfig(UserConfig userConfig) {
+      _userConfig = userConfig;
+      return this;
+    }
+
+    /**
+     * @param schedulerTaskConfig
+     * @return
+     */
+    public Builder schedulerTaskConfig(SchedulerTaskConfig schedulerTaskConfig) {
+      _schedulerTaskConfig = schedulerTaskConfig;
+      return this;
+    }
+
+    /**
+     * Set the bucket size
+     * @param bucketSize the size to use
+     * @return Builder
+     */
+    public Builder bucketSize(int bucketSize) {
+      _bucketSize = bucketSize;
+      return this;
+    }
+
+    /**
+     * Set the batch message mode
+     * @param batchMessageMode true to enable, false to disable
+     * @return Builder
+     */
+    public Builder batchMessageMode(boolean batchMessageMode) {
+      _batchMessageMode = batchMessageMode;
+      return this;
+    }
+
+    /**
+     * Create a Resource object
+     * @return instantiated Resource
+     */
+    public ResourceConfig build() {
+      return new ResourceConfig(_id, _type, _schedulerTaskConfig, _rebalancerConfig, _userConfig,
+          _bucketSize, _batchMessageMode);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java
new file mode 100644
index 0000000..8096daa
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/SchedulerTaskConfig.java
@@ -0,0 +1,69 @@
+package org.apache.helix.api.config;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.model.Message;
+
+import com.google.common.collect.ImmutableMap;
+
+public class SchedulerTaskConfig {
+  // TODO refactor using Transition logical model
+  private final Map<String, Integer> _transitionTimeoutMap;
+
+  private final Map<PartitionId, Message> _innerMessageMap;
+
+  public SchedulerTaskConfig(Map<String, Integer> transitionTimeoutMap,
+      Map<PartitionId, Message> innerMsgMap) {
+    _transitionTimeoutMap = ImmutableMap.copyOf(transitionTimeoutMap);
+    _innerMessageMap = ImmutableMap.copyOf(innerMsgMap);
+  }
+
+  /**
+   * Get inner message for a partition
+   * @param partitionId
+   * @return inner message
+   */
+  public Message getInnerMessage(PartitionId partitionId) {
+    return _innerMessageMap.get(partitionId);
+  }
+
+  /**
+   * Get timeout for a transition
+   * @param transition
+   * @return timeout or -1 if not available
+   */
+  public int getTransitionTimeout(String transition) {
+    Integer timeout = _transitionTimeoutMap.get(transition);
+    if (timeout == null) {
+      return -1;
+    }
+
+    return timeout;
+  }
+
+  /**
+   * Get timeout for an inner message
+   * @param transition
+   * @param partitionId
+   * @return timeout or -1 if not available
+   */
+  public int getTimeout(String transition, PartitionId partitionId) {
+    Integer timeout = getTransitionTimeout(transition);
+    if (timeout == null) {
+      Message innerMessage = getInnerMessage(partitionId);
+      timeout = innerMessage.getTimeout();
+    }
+
+    return timeout;
+  }
+
+  /**
+   * Get partition-id set
+   * @return partition-id set
+   */
+  public Set<PartitionId> getPartitionSet() {
+    return _innerMessageMap.keySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
new file mode 100644
index 0000000..dbf70ea
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
@@ -0,0 +1,53 @@
+package org.apache.helix.api.config;
+
+import org.apache.helix.HelixProperty;
+import org.apache.helix.api.Scope;
+
+/*
+ * 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.
+ */
+
+/**
+ * Generic user-defined configuration of Helix components
+ */
+public class UserConfig extends NamespacedConfig {
+  /**
+   * Instantiate a UserConfig. It is intended for use only by entities that can be identified
+   * @param scope scope of the configuration, e.g. cluster, resource, partition, participant, etc
+   */
+  public UserConfig(Scope<?> scope) {
+    super(scope, UserConfig.class.getSimpleName());
+  }
+
+  /**
+   * Instantiate a UserConfig from an existing HelixProperty
+   * @param property property wrapping a configuration
+   */
+  private UserConfig(HelixProperty property) {
+    super(property, UserConfig.class.getSimpleName());
+  }
+
+  /**
+   * Get a UserConfig that filters out the user-specific configurations in a property
+   * @param property the property to extract from
+   * @return UserConfig
+   */
+  public static UserConfig from(HelixProperty property) {
+    return new UserConfig(property);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
new file mode 100644
index 0000000..f05bb5d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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.
+ */
+
+/**
+ * Identifies a cluster
+ */
+public class ClusterId extends Id {
+  @JsonProperty("id")
+  final private String _id;
+
+  /**
+   * Create a cluster id
+   * @param id string representation of the id
+   */
+  @JsonCreator
+  public ClusterId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete cluster id for a string name
+   * @param clusterId string cluster identifier
+   * @return ClusterId
+   */
+  public static ClusterId from(String clusterId) {
+    if (clusterId == null) {
+      return null;
+    }
+    return new ClusterId(clusterId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
new file mode 100644
index 0000000..26a7610
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
@@ -0,0 +1,80 @@
+package org.apache.helix.api.id;
+
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.State;
+import org.apache.helix.model.Transition;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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.
+ */
+
+/**
+ * Identifies a constraint item on the cluster
+ */
+public class ConstraintId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a constraint id
+   * @param constraintId string representing the constraint id
+   */
+  @JsonCreator
+  public ConstraintId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a constraint id from a string
+   * @param constraintId string representing the constraint id
+   * @return ConstraintId
+   */
+  public static ConstraintId from(String constraintId) {
+    return new ConstraintId(constraintId);
+  }
+
+  /**
+   * Get a state constraint id based on the state model definition and state
+   * @param scope the scope of the constraint
+   * @param stateModelDefId the state model
+   * @param state the constrained state
+   * @return ConstraintId
+   */
+  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId, State state) {
+    return new ConstraintId(scope + "|" + stateModelDefId + "|" + state);
+  }
+
+  /**
+   * Get a state constraint id based on the state model definition and transition
+   * @param scope the scope of the constraint
+   * @param stateModelDefId the state model
+   * @param transition the constrained transition
+   * @return ConstraintId
+   */
+  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId,
+      Transition transition) {
+    return new ConstraintId(scope + "|" + stateModelDefId + "|" + transition);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
new file mode 100644
index 0000000..1130afd
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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.
+ */
+
+/**
+ * Identifies Helix nodes that take on the CONTROLLER role
+ */
+public class ControllerId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a controller id
+   * @param id string representation of a controller id
+   */
+  @JsonCreator
+  public ControllerId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a ControllerId from a string
+   * @param controllerId string representing the id
+   * @return ControllerId
+   */
+  public static ControllerId from(String controllerId) {
+    return new ControllerId(controllerId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/Id.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/Id.java b/helix-core/src/main/java/org/apache/helix/api/id/Id.java
new file mode 100644
index 0000000..ea9c6cc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/Id.java
@@ -0,0 +1,55 @@
+package org.apache.helix.api.id;
+
+/*
+ * 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.
+ */
+
+/**
+ * Generic identifier for Helix constructs
+ */
+public abstract class Id implements Comparable<Id> {
+  public abstract String stringify();
+
+  @Override
+  public String toString() {
+    return stringify();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that instanceof Id) {
+      return this.stringify().equals(((Id) that).stringify());
+    } else if (that instanceof String) {
+      return this.stringify().equals(that);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.stringify().hashCode();
+  }
+
+  @Override
+  public int compareTo(Id that) {
+    if (that instanceof Id) {
+      return this.stringify().compareTo(that.stringify());
+    }
+    return -1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
new file mode 100644
index 0000000..a59976d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 MessageId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a message id
+   * @param id string representation of a message id
+   */
+  @JsonCreator
+  public MessageId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete message id
+   * @param messageId string message identifier
+   * @return MsgId
+   */
+  public static MessageId from(String messageId) {
+    if (messageId == null) {
+      return null;
+    }
+    return new MessageId(messageId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
new file mode 100644
index 0000000..5fe91dd
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 ParticipantId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Instantiate for a participant with a string name
+   * @param id string participant id
+   */
+  @JsonCreator
+  public ParticipantId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete participant id
+   * @param participantId string participant identifier
+   * @return ParticipantId
+   */
+  public static ParticipantId from(String participantId) {
+    if (participantId == null) {
+      return null;
+    }
+    return new ParticipantId(participantId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
new file mode 100644
index 0000000..dd1bc0d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
@@ -0,0 +1,112 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 PartitionId extends Id {
+  @JsonProperty("resourceId")
+  private final ResourceId _resourceId;
+  @JsonProperty("partitionName")
+  private final String _partitionName;
+
+  /**
+   * Instantiate for a resource and suffix
+   * @param resourceId resource that the partition belongs to
+   * @param partitionName name of the partition relative to the resource
+   */
+  @JsonCreator
+  public PartitionId(@JsonProperty("resourceId") ResourceId resourceId,
+      @JsonProperty("partitionName") String partitionName) {
+    _resourceId = resourceId;
+    _partitionName = partitionName;
+  }
+
+  /**
+   * Get the id of the resource containing this partition
+   * @return ResourceId
+   */
+  public ResourceId getResourceId() {
+    return _resourceId;
+  }
+
+  @Override
+  public String stringify() {
+    // check in case the partition name is instantiated incorrectly
+    if (_resourceId.stringify().equals(_partitionName)) {
+      return _partitionName;
+    }
+    return String.format("%s_%s", _resourceId, _partitionName);
+  }
+
+  /**
+   * @param partitionName
+   * @return
+   */
+  public static String stripResourceId(String partitionName) {
+    if (partitionName == null || !partitionName.contains("_")) {
+      return partitionName;
+    }
+    return partitionName.substring(partitionName.lastIndexOf("_") + 1);
+  }
+
+  /**
+   * @param partitionName
+   * @return
+   */
+  public static ResourceId extractResourceId(String partitionName) {
+    if (partitionName == null || !partitionName.contains("_")) {
+      return ResourceId.from(partitionName);
+    }
+    return ResourceId.from(partitionName.substring(0, partitionName.lastIndexOf("_")));
+  }
+
+  /**
+   * Get a concrete partition id
+   * @param partitionId string partition identifier
+   * @return PartitionId
+   */
+  public static PartitionId from(String partitionId) {
+    if (partitionId == null) {
+      return null;
+    }
+    return new PartitionId(extractResourceId(partitionId), stripResourceId(partitionId));
+  }
+
+  /**
+   * Same as {@link PartitionId#from(String)}.
+   * @param partitionId string partition identifier
+   * @return PartitionId
+   */
+  public static PartitionId valueOf(String partitionId) {
+    return from(partitionId);
+  }
+
+  /**
+   * Get a concrete partition id
+   * @param resourceId resource identifier
+   * @param partitionSuffix partition identifier relative to a resource
+   * @return PartitionId
+   */
+  public static PartitionId from(ResourceId resourceId, String partitionSuffix) {
+    return new PartitionId(resourceId, partitionSuffix);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
new file mode 100644
index 0000000..3b8c21d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 ProcId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a process id
+   * @param id string representation of a process id
+   */
+  @JsonCreator
+  public ProcId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete process id
+   * @param processId string process identifier (e.g. pid@host)
+   * @return ProcId
+   */
+  public static ProcId from(String processId) {
+    if (processId == null) {
+      return null;
+    }
+    return new ProcId(processId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
new file mode 100644
index 0000000..e70fff1
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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.
+ */
+
+/**
+ * Identifies a resource
+ */
+public class ResourceId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Create a resource id
+   * @param id string representation of a resource id
+   */
+  @JsonCreator
+  public ResourceId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  /**
+   * Get a concrete resource id for a string name
+   * @param resourceId string resource identifier
+   * @return ResourceId
+   */
+  public static ResourceId from(String resourceId) {
+    if (resourceId == null) {
+      return null;
+    }
+    return new ResourceId(resourceId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
new file mode 100644
index 0000000..17fb3a3
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 SessionId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a session id
+   * @param id string representing a session id
+   */
+  @JsonCreator
+  public SessionId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete session id
+   * @param sessionId string session identifier
+   * @return SessionId
+   */
+  public static SessionId from(String sessionId) {
+    if (sessionId == null) {
+      return null;
+    }
+    return new SessionId(sessionId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
new file mode 100644
index 0000000..d50390c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
@@ -0,0 +1,51 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 SpectatorId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a spectator id
+   * @param id string representing a spectator id
+   */
+  @JsonCreator
+  public SpectatorId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Create a spectator id from a string
+   * @param spectatorId string representing a spectator id
+   * @return SpectatorId
+   */
+  public static SpectatorId from(String spectatorId) {
+    return new SpectatorId(spectatorId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
new file mode 100644
index 0000000..7c84f0f
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
@@ -0,0 +1,66 @@
+package org.apache.helix.api.id;
+
+import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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 StateModelDefId extends Id {
+  public static final StateModelDefId SchedulerTaskQueue = StateModelDefId
+      .from(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE);
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a state model definition id
+   * @param id string representing a state model definition id
+   */
+  @JsonCreator
+  public StateModelDefId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Check if the underlying state model definition id is equal if case is ignored
+   * @param that the StateModelDefId to compare
+   * @return true if equal ignoring case, false otherwise
+   */
+  public boolean equalsIgnoreCase(StateModelDefId that) {
+    return _id.equalsIgnoreCase(that._id);
+  }
+
+  /**
+   * Get a concrete state model definition id
+   * @param stateModelDefId string state model identifier
+   * @return StateModelDefId
+   */
+  public static StateModelDefId from(String stateModelDefId) {
+    if (stateModelDefId == null) {
+      return null;
+    }
+    return new StateModelDefId(stateModelDefId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
new file mode 100644
index 0000000..795c14c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * 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.
+ */
+
+/**
+ * Id representing a state model factory
+ */
+public class StateModelFactoryId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a state model factory id
+   * @param id string representing a state model factory
+   */
+  @JsonCreator
+  public StateModelFactoryId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete state model factory id
+   * @param stateModelFactoryId the string version of the id
+   * @return StateModelFactoryId
+   */
+  public static StateModelFactoryId from(String stateModelFactoryId) {
+    if (stateModelFactoryId == null) {
+      return null;
+    }
+    return new StateModelFactoryId(stateModelFactoryId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
index 1c69dfe..6d65009 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
@@ -29,8 +29,8 @@ import java.util.Set;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
index 709b61e..512af80 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
@@ -25,8 +25,8 @@ import java.util.Set;
 
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.IdealState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
index 433b4a7..dd9fcf1 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
@@ -24,8 +24,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java
index be3a280..bb2ab17 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java
@@ -3,10 +3,10 @@ package org.apache.helix.controller.rebalancer.context;
 import java.util.Set;
 
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 import org.codehaus.jackson.annotate.JsonIgnore;
 
 /*

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java
index 97ea96a..fb0c512 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java
@@ -8,9 +8,9 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 import org.apache.helix.model.ResourceAssignment;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java
index c1efe81..3ccce3d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java
@@ -2,10 +2,10 @@ package org.apache.helix.controller.rebalancer.context;
 
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.codehaus.jackson.annotate.JsonIgnore;
 import org.testng.collections.Maps;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java
index a1b8406..189df64 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java
@@ -13,9 +13,9 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java
index 2e274be..11a1b47 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java
@@ -1,6 +1,6 @@
 package org.apache.helix.controller.rebalancer.context;
 
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.IdealState.RebalanceMode;
 
 /*

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java
index 3925c2b..b24125c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java
@@ -7,8 +7,8 @@ import java.util.Set;
 
 import org.apache.helix.HelixConstants.StateModelToken;
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.codehaus.jackson.annotate.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java
index 7d5fed2..31d37a8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java
@@ -1,7 +1,7 @@
 package org.apache.helix.controller.rebalancer.context;
 
-import org.apache.helix.api.NamespacedConfig;
 import org.apache.helix.api.Scope;
+import org.apache.helix.api.config.NamespacedConfig;
 import org.apache.helix.model.ResourceConfiguration;
 import org.apache.helix.util.HelixUtil;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java
index 87863b5..ea35525 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java
@@ -4,10 +4,10 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.api.Partition;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.StateModelDefId;
-import org.apache.helix.api.StateModelFactoryId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.api.id.StateModelFactoryId;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java
index 6fe3f54..c112fcf 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java
@@ -6,9 +6,9 @@ import java.util.Set;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 import org.apache.helix.model.ResourceAssignment;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java
index 6cdfbb6..d6d163c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java
@@ -3,9 +3,9 @@ package org.apache.helix.controller.rebalancer.context;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.IdealState.RebalanceMode;
 import org.codehaus.jackson.annotate.JsonIgnore;
 import org.codehaus.jackson.annotate.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java
index 603edd0..4f2e10c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java
@@ -31,13 +31,13 @@ import java.util.Set;
 import org.apache.helix.HelixConstants.StateModelToken;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ClusterConfig;
 import org.apache.helix.api.Participant;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
+import org.apache.helix.api.config.ClusterConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 dbbcb1e..9699dcb 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -22,9 +22,9 @@ package org.apache.helix.controller.stages;
 import java.util.Map;
 
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.AutoRebalancer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
index dfa60c6..362bbb6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
@@ -23,8 +23,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.id.ParticipantId;
 import org.apache.helix.model.Partition;
 
 @Deprecated

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
index d731ebd..3f5682e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CurrentStateComputationStage.java
@@ -22,8 +22,8 @@ package org.apache.helix.controller.stages;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.CurrentState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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 e2c8854..9c24bd6 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
@@ -26,12 +26,12 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java
index 7434c2a..11d7969 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java
@@ -24,11 +24,11 @@ import java.util.Set;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.ParticipantId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceConfig;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.controller.rebalancer.context.Rebalancer;


[4/6] [HELIX-238] Refactor, add update to accessors, test update logic

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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
new file mode 100644
index 0000000..8e07d97
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
@@ -0,0 +1,553 @@
+package org.apache.helix.api.accessor;
+
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.Controller;
+import org.apache.helix.api.Participant;
+import org.apache.helix.api.Resource;
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.config.ClusterConfig;
+import org.apache.helix.api.config.ParticipantConfig;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.config.UserConfig;
+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.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.model.ClusterConfiguration;
+import org.apache.helix.model.ClusterConstraints;
+import org.apache.helix.model.ClusterConstraints.ConstraintType;
+import org.apache.helix.model.CurrentState;
+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.Message;
+import org.apache.helix.model.PauseSignal;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.ResourceConfiguration;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+public class ClusterAccessor {
+  private static Logger LOG = Logger.getLogger(ClusterAccessor.class);
+
+  private final HelixDataAccessor _accessor;
+  private final PropertyKey.Builder _keyBuilder;
+  private final ClusterId _clusterId;
+
+  public ClusterAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
+    _accessor = accessor;
+    _keyBuilder = accessor.keyBuilder();
+    _clusterId = clusterId;
+  }
+
+  /**
+   * create a new cluster, fail if it already exists
+   * @return true if created, false if creation failed
+   */
+  public boolean createCluster(ClusterConfig cluster) {
+    boolean created = _accessor.createProperty(_keyBuilder.cluster(), null);
+    if (!created) {
+      LOG.error("Cluster already created. Aborting.");
+      return false;
+    }
+    initClusterStructure();
+    Map<StateModelDefId, StateModelDefinition> stateModelDefs = cluster.getStateModelMap();
+    for (StateModelDefinition stateModelDef : stateModelDefs.values()) {
+      addStateModelDefinitionToCluster(stateModelDef);
+    }
+    Map<ResourceId, ResourceConfig> resources = cluster.getResourceMap();
+    for (ResourceConfig resource : resources.values()) {
+      addResourceToCluster(resource);
+    }
+    Map<ParticipantId, ParticipantConfig> participants = cluster.getParticipantMap();
+    for (ParticipantConfig participant : participants.values()) {
+      addParticipantToCluster(participant);
+    }
+    _accessor.createProperty(_keyBuilder.constraints(), null);
+    for (ClusterConstraints constraints : cluster.getConstraintMap().values()) {
+      _accessor.createProperty(_keyBuilder.constraint(constraints.getType().toString()),
+          constraints);
+    }
+    _accessor.createProperty(_keyBuilder.clusterConfig(),
+        ClusterConfiguration.from(cluster.getUserConfig()));
+    if (cluster.isPaused()) {
+      pauseCluster();
+    }
+
+    return true;
+  }
+
+  /**
+   * Update the cluster configuration
+   * @param clusterDelta change to the cluster configuration
+   * @return updated ClusterConfig, or null if there was an error
+   */
+  public ClusterConfig updateCluster(ClusterConfig.Delta clusterDelta) {
+    Cluster cluster = readCluster();
+    if (cluster == null) {
+      LOG.error("Cluster does not exist, cannot be updated");
+      return null;
+    }
+    ClusterConfig config = clusterDelta.mergeInto(cluster.getConfig());
+    boolean status = setBasicClusterConfig(config);
+    return status ? config : null;
+  }
+
+  /**
+   * Set a cluster config minus state model, participants, and resources
+   * @param config ClusterConfig
+   * @return true if correctly set, false otherwise
+   */
+  private boolean setBasicClusterConfig(ClusterConfig config) {
+    if (config == null) {
+      return false;
+    }
+    ClusterConfiguration configuration = ClusterConfiguration.from(config.getUserConfig());
+    _accessor.setProperty(_keyBuilder.clusterConfig(), configuration);
+    Map<ConstraintType, ClusterConstraints> constraints = config.getConstraintMap();
+    for (ConstraintType type : constraints.keySet()) {
+      ClusterConstraints constraint = constraints.get(type);
+      _accessor.setProperty(_keyBuilder.constraint(type.toString()), constraint);
+    }
+    return true;
+  }
+
+  /**
+   * drop a cluster
+   * @return true if the cluster was dropped, false if there was an error
+   */
+  public boolean dropCluster() {
+    LOG.info("Dropping cluster: " + _clusterId);
+    List<String> liveInstanceNames = _accessor.getChildNames(_keyBuilder.liveInstances());
+    if (liveInstanceNames.size() > 0) {
+      LOG.error("Can't drop cluster: " + _clusterId + " because there are running participant: "
+          + liveInstanceNames + ", shutdown participants first.");
+      return false;
+    }
+
+    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
+    if (leader != null) {
+      LOG.error("Can't drop cluster: " + _clusterId + ", because leader: " + leader.getId()
+          + " are running, shutdown leader first.");
+      return false;
+    }
+
+    return _accessor.removeProperty(_keyBuilder.cluster());
+  }
+
+  /**
+   * read entire cluster data
+   * @return cluster snapshot
+   */
+  public Cluster readCluster() {
+    /**
+     * map of instance-id to instance-config
+     */
+    Map<String, InstanceConfig> instanceConfigMap =
+        _accessor.getChildValuesMap(_keyBuilder.instanceConfigs());
+
+    /**
+     * map of resource-id to ideal-state
+     */
+    Map<String, IdealState> idealStateMap = _accessor.getChildValuesMap(_keyBuilder.idealStates());
+
+    /**
+     * map of instance-id to live-instance
+     */
+    Map<String, LiveInstance> liveInstanceMap =
+        _accessor.getChildValuesMap(_keyBuilder.liveInstances());
+
+    /**
+     * map of participant-id to map of message-id to message
+     */
+    Map<String, Map<String, Message>> messageMap = new HashMap<String, Map<String, Message>>();
+    for (String instanceName : liveInstanceMap.keySet()) {
+      Map<String, Message> instanceMsgMap =
+          _accessor.getChildValuesMap(_keyBuilder.messages(instanceName));
+      messageMap.put(instanceName, instanceMsgMap);
+    }
+
+    /**
+     * map of participant-id to map of resource-id to current-state
+     */
+    Map<String, Map<String, CurrentState>> currentStateMap =
+        new HashMap<String, Map<String, CurrentState>>();
+    for (String participantName : liveInstanceMap.keySet()) {
+      LiveInstance liveInstance = liveInstanceMap.get(participantName);
+      SessionId sessionId = liveInstance.getSessionId();
+      Map<String, CurrentState> instanceCurStateMap =
+          _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName,
+              sessionId.stringify()));
+
+      currentStateMap.put(participantName, instanceCurStateMap);
+    }
+
+    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
+
+    /**
+     * map of constraint-type to constraints
+     */
+    Map<String, ClusterConstraints> constraintMap =
+        _accessor.getChildValuesMap(_keyBuilder.constraints());
+
+    /**
+     * Map of resource id to external view
+     */
+    Map<String, ExternalView> externalViewMap =
+        _accessor.getChildValuesMap(_keyBuilder.externalViews());
+
+    /**
+     * Map of resource id to user configuration
+     */
+    Map<String, ResourceConfiguration> resourceConfigMap =
+        _accessor.getChildValuesMap(_keyBuilder.resourceConfigs());
+
+    /**
+     * Map of resource id to resource assignment
+     */
+    Map<String, ResourceAssignment> resourceAssignmentMap =
+        _accessor.getChildValuesMap(_keyBuilder.resourceAssignments());
+
+    // read all the resources
+    Map<ResourceId, Resource> resourceMap = new HashMap<ResourceId, Resource>();
+    for (String resourceName : idealStateMap.keySet()) {
+      ResourceId resourceId = ResourceId.from(resourceName);
+      resourceMap.put(resourceId, ResourceAccessor.createResource(resourceId,
+          resourceConfigMap.get(resourceName), idealStateMap.get(resourceName),
+          externalViewMap.get(resourceName), resourceAssignmentMap.get(resourceName)));
+    }
+
+    // read all the participants
+    Map<ParticipantId, Participant> participantMap = new HashMap<ParticipantId, Participant>();
+    for (String participantName : instanceConfigMap.keySet()) {
+      InstanceConfig instanceConfig = instanceConfigMap.get(participantName);
+      UserConfig userConfig = UserConfig.from(instanceConfig);
+      LiveInstance liveInstance = liveInstanceMap.get(participantName);
+      Map<String, Message> instanceMsgMap = messageMap.get(participantName);
+
+      ParticipantId participantId = ParticipantId.from(participantName);
+
+      participantMap.put(participantId, ParticipantAccessor.createParticipant(participantId,
+          instanceConfig, userConfig, liveInstance, instanceMsgMap,
+          currentStateMap.get(participantName)));
+    }
+
+    // read the controllers
+    Map<ControllerId, Controller> controllerMap = new HashMap<ControllerId, Controller>();
+    ControllerId leaderId = null;
+    if (leader != null) {
+      leaderId = ControllerId.from(leader.getId());
+      controllerMap.put(leaderId, new Controller(leaderId, leader, true));
+    }
+
+    // read the constraints
+    Map<ConstraintType, ClusterConstraints> clusterConstraintMap =
+        new HashMap<ConstraintType, ClusterConstraints>();
+    for (String constraintType : constraintMap.keySet()) {
+      clusterConstraintMap.put(ConstraintType.valueOf(constraintType),
+          constraintMap.get(constraintType));
+    }
+
+    // read the pause status
+    PauseSignal pauseSignal = _accessor.getProperty(_keyBuilder.pause());
+    boolean isPaused = pauseSignal != null;
+
+    ClusterConfiguration clusterUserConfig = _accessor.getProperty(_keyBuilder.clusterConfig());
+    UserConfig userConfig;
+    if (clusterUserConfig != null) {
+      userConfig = UserConfig.from(clusterUserConfig);
+    } else {
+      userConfig = new UserConfig(Scope.cluster(_clusterId));
+    }
+
+    // read the state model definitions
+    StateModelDefinitionAccessor stateModelDefAccessor =
+        new StateModelDefinitionAccessor(_accessor);
+    Map<StateModelDefId, StateModelDefinition> stateModelMap =
+        stateModelDefAccessor.readStateModelDefinitions();
+
+    // create the cluster snapshot object
+    return new Cluster(_clusterId, resourceMap, participantMap, controllerMap, leaderId,
+        clusterConstraintMap, stateModelMap, userConfig, isPaused);
+  }
+
+  /**
+   * pause controller of cluster
+   */
+  public void pauseCluster() {
+    _accessor.createProperty(_keyBuilder.pause(), new PauseSignal("pause"));
+  }
+
+  /**
+   * resume controller of cluster
+   */
+  public void resumeCluster() {
+    _accessor.removeProperty(_keyBuilder.pause());
+  }
+
+  /**
+   * add a resource to cluster
+   * @param resource
+   * @return true if resource added, false if there was an error
+   */
+  public boolean addResourceToCluster(ResourceConfig resource) {
+    if (resource == null || resource.getRebalancerConfig() == null) {
+      LOG.error("Resource not fully defined with a rebalancer context");
+      return false;
+    }
+
+    if (!isClusterStructureValid()) {
+      LOG.error("Cluster: " + _clusterId + " structure is not valid");
+      return false;
+    }
+    RebalancerContext context =
+        resource.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
+    StateModelDefId stateModelDefId = context.getStateModelDefId();
+    if (_accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())) == null) {
+      LOG.error("State model: " + stateModelDefId + " not found in cluster: " + _clusterId);
+      return false;
+    }
+
+    ResourceId resourceId = resource.getId();
+    if (_accessor.getProperty(_keyBuilder.idealState(resourceId.stringify())) != null) {
+      LOG.error("Skip adding resource: " + resourceId
+          + ", because resource ideal state already exists in cluster: " + _clusterId);
+      return false;
+    }
+    if (_accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())) != null) {
+      LOG.error("Skip adding resource: " + resourceId
+          + ", because resource config already exists in cluster: " + _clusterId);
+      return false;
+    }
+
+    // Add resource user config
+    if (resource.getUserConfig() != null) {
+      ResourceConfiguration configuration = new ResourceConfiguration(resourceId);
+      configuration.setType(resource.getType());
+      configuration.addNamespacedConfig(resource.getUserConfig());
+      configuration.addNamespacedConfig(resource.getRebalancerConfig().toNamespacedConfig());
+      configuration.setBucketSize(resource.getBucketSize());
+      configuration.setBatchMessageMode(resource.getBatchMessageMode());
+      _accessor.createProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration);
+    }
+
+    // Create an IdealState from a RebalancerConfig (if the resource is partitioned)
+    RebalancerConfig rebalancerConfig = resource.getRebalancerConfig();
+    IdealState idealState =
+        ResourceAccessor.rebalancerConfigToIdealState(rebalancerConfig, resource.getBucketSize(),
+            resource.getBatchMessageMode());
+    if (idealState != null) {
+      _accessor.createProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
+    }
+    return true;
+  }
+
+  /**
+   * drop a resource from cluster
+   * @param resourceId
+   * @return true if removal succeeded, false otherwise
+   */
+  public boolean dropResourceFromCluster(ResourceId resourceId) {
+    if (_accessor.getProperty(_keyBuilder.idealState(resourceId.stringify())) == null) {
+      LOG.error("Skip removing resource: " + resourceId
+          + ", because resource ideal state already removed from cluster: " + _clusterId);
+      return false;
+    }
+    _accessor.removeProperty(_keyBuilder.idealState(resourceId.stringify()));
+    _accessor.removeProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
+    return true;
+  }
+
+  /**
+   * check if cluster structure is valid
+   * @return true if valid or false otherwise
+   */
+  public boolean isClusterStructureValid() {
+    List<String> paths = getRequiredPaths();
+    BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
+    if (baseAccessor != null) {
+      boolean[] existsResults = baseAccessor.exists(paths, 0);
+      for (boolean exists : existsResults) {
+        if (!exists) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Create empty persistent properties to ensure that there is a valid cluster structure
+   */
+  private void initClusterStructure() {
+    BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
+    List<String> paths = getRequiredPaths();
+    for (String path : paths) {
+      boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT);
+      if (!status && LOG.isDebugEnabled()) {
+        LOG.debug(path + " already exists");
+      }
+    }
+  }
+
+  /**
+   * Get all property paths that must be set for a cluster structure to be valid
+   * @return list of paths as strings
+   */
+  private List<String> getRequiredPaths() {
+    List<String> paths = new ArrayList<String>();
+    paths.add(_keyBuilder.cluster().getPath());
+    paths.add(_keyBuilder.idealStates().getPath());
+    paths.add(_keyBuilder.clusterConfigs().getPath());
+    paths.add(_keyBuilder.instanceConfigs().getPath());
+    paths.add(_keyBuilder.resourceConfigs().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
+   */
+  public boolean addParticipantToCluster(ParticipantConfig participant) {
+    if (participant == null) {
+      LOG.error("Participant not initialized");
+      return false;
+    }
+    if (!isClusterStructureValid()) {
+      LOG.error("Cluster: " + _clusterId + " structure is not valid");
+      return false;
+    }
+
+    ParticipantId participantId = participant.getId();
+    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())) != null) {
+      LOG.error("Config for participant: " + participantId + " already exists in cluster: "
+          + _clusterId);
+      return false;
+    }
+
+    // add empty root ZNodes
+    List<PropertyKey> createKeys = new ArrayList<PropertyKey>();
+    createKeys.add(_keyBuilder.messages(participantId.stringify()));
+    createKeys.add(_keyBuilder.currentStates(participantId.stringify()));
+    createKeys.add(_keyBuilder.participantErrors(participantId.stringify()));
+    createKeys.add(_keyBuilder.statusUpdates(participantId.stringify()));
+    for (PropertyKey key : createKeys) {
+      _accessor.createProperty(key, null);
+    }
+
+    // add the config
+    InstanceConfig instanceConfig = new InstanceConfig(participant.getId());
+    instanceConfig.setHostName(participant.getHostName());
+    instanceConfig.setPort(Integer.toString(participant.getPort()));
+    instanceConfig.setInstanceEnabled(participant.isEnabled());
+    UserConfig userConfig = participant.getUserConfig();
+    instanceConfig.addNamespacedConfig(userConfig);
+    Set<String> tags = participant.getTags();
+    for (String tag : tags) {
+      instanceConfig.addTag(tag);
+    }
+    Set<PartitionId> disabledPartitions = participant.getDisabledPartitions();
+    for (PartitionId partitionId : disabledPartitions) {
+      instanceConfig.setInstanceEnabledForPartition(partitionId, false);
+    }
+    _accessor.createProperty(_keyBuilder.instanceConfig(participantId.stringify()), instanceConfig);
+    _accessor.createProperty(_keyBuilder.messages(participantId.stringify()), null);
+    return true;
+  }
+
+  /**
+   * drop a participant from cluster
+   * @param participantId
+   * @return true if participant dropped, false if there was an error
+   */
+  public boolean dropParticipantFromCluster(ParticipantId participantId) {
+    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())) == null) {
+      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
+          + _clusterId);
+      return false;
+    }
+
+    if (_accessor.getProperty(_keyBuilder.instance(participantId.stringify())) == null) {
+      LOG.error("Participant: " + participantId + " structure does NOT exist in cluster: "
+          + _clusterId);
+      return false;
+    }
+
+    // delete participant config path
+    _accessor.removeProperty(_keyBuilder.instanceConfig(participantId.stringify()));
+
+    // delete participant path
+    _accessor.removeProperty(_keyBuilder.instance(participantId.stringify()));
+    return true;
+  }
+
+  /**
+   * Add a state model definition. Updates the existing state model definition if it already exists.
+   * @param stateModelDef fully initialized state model definition
+   * @return true if the model is persisted, false otherwise
+   */
+  public boolean addStateModelDefinitionToCluster(StateModelDefinition stateModelDef) {
+    if (!isClusterStructureValid()) {
+      LOG.error("Cluster: " + _clusterId + " structure is not valid");
+      return false;
+    }
+
+    StateModelDefinitionAccessor smdAccessor = new StateModelDefinitionAccessor(_accessor);
+    return smdAccessor.setStateModelDefinition(stateModelDef);
+  }
+
+  /**
+   * Remove a state model definition if it exists
+   * @param stateModelDefId state model definition id
+   * @return true if removed, false if it did not exist
+   */
+  public boolean dropStateModelDefinitionFromCluster(StateModelDefId stateModelDefId) {
+    return _accessor.removeProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java
new file mode 100644
index 0000000..609e458
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ControllerAccessor.java
@@ -0,0 +1,49 @@
+package org.apache.helix.api.accessor;
+
+/*
+ * 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.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.Controller;
+import org.apache.helix.api.id.ControllerId;
+import org.apache.helix.model.LiveInstance;
+
+public class ControllerAccessor {
+  private final HelixDataAccessor _accessor;
+  private final PropertyKey.Builder _keyBuilder;
+
+  public ControllerAccessor(HelixDataAccessor accessor) {
+    _accessor = accessor;
+    _keyBuilder = accessor.keyBuilder();
+  }
+
+  /**
+   * Read the leader controller if it is live
+   * @return Controller snapshot, or null
+   */
+  public Controller readLeader() {
+    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
+    if (leader != null) {
+      ControllerId leaderId = ControllerId.from(leader.getId());
+      return new Controller(leaderId, leader, true);
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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
new file mode 100644
index 0000000..c1a9250
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
@@ -0,0 +1,435 @@
+package org.apache.helix.api.accessor;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.I0Itec.zkclient.DataUpdater;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.api.Participant;
+import org.apache.helix.api.RunningInstance;
+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;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.IdealState.RebalanceMode;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.InstanceConfig.InstanceConfigProperty;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
+import org.apache.log4j.Logger;
+
+public class ParticipantAccessor {
+  private static final Logger LOG = Logger.getLogger(ParticipantAccessor.class);
+
+  private final HelixDataAccessor _accessor;
+  private final PropertyKey.Builder _keyBuilder;
+  private final ClusterId _clusterId;
+
+  public ParticipantAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
+    _clusterId = clusterId;
+    _accessor = accessor;
+    _keyBuilder = accessor.keyBuilder();
+  }
+
+  /**
+   * enable/disable a participant
+   * @param participantId
+   * @param isEnabled
+   */
+  void enableParticipant(ParticipantId participantId, boolean isEnabled) {
+    String participantName = participantId.stringify();
+    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantName)) == null) {
+      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
+          + _clusterId);
+      return;
+    }
+
+    InstanceConfig config = new InstanceConfig(participantName);
+    config.setInstanceEnabled(isEnabled);
+    _accessor.updateProperty(_keyBuilder.instanceConfig(participantName), config);
+
+  }
+
+  /**
+   * disable participant
+   * @param participantId
+   */
+  public void disableParticipant(ParticipantId participantId) {
+    enableParticipant(participantId, false);
+  }
+
+  /**
+   * enable participant
+   * @param participantId
+   */
+  public void enableParticipant(ParticipantId participantId) {
+    enableParticipant(participantId, true);
+  }
+
+  /**
+   * create messages for participant
+   * @param participantId
+   * @param msgMap map of message-id to message
+   */
+  public void insertMessagesToParticipant(ParticipantId participantId,
+      Map<MessageId, Message> msgMap) {
+    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
+    List<Message> msgs = new ArrayList<Message>();
+    for (MessageId msgId : msgMap.keySet()) {
+      msgKeys.add(_keyBuilder.message(participantId.stringify(), msgId.stringify()));
+      msgs.add(msgMap.get(msgId));
+    }
+
+    _accessor.createChildren(msgKeys, msgs);
+  }
+
+  /**
+   * set messages of participant
+   * @param participantId
+   * @param msgMap map of message-id to message
+   */
+  public void updateMessageStatus(ParticipantId participantId, Map<MessageId, Message> msgMap) {
+    String participantName = participantId.stringify();
+    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
+    List<Message> msgs = new ArrayList<Message>();
+    for (MessageId msgId : msgMap.keySet()) {
+      msgKeys.add(_keyBuilder.message(participantName, msgId.stringify()));
+      msgs.add(msgMap.get(msgId));
+    }
+    _accessor.setChildren(msgKeys, msgs);
+  }
+
+  /**
+   * delete messages from participant
+   * @param participantId
+   * @param msgIdSet
+   */
+  public void deleteMessagesFromParticipant(ParticipantId participantId, Set<MessageId> msgIdSet) {
+    String participantName = participantId.stringify();
+    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
+    for (MessageId msgId : msgIdSet) {
+      msgKeys.add(_keyBuilder.message(participantName, msgId.stringify()));
+    }
+
+    // TODO impl batch remove
+    for (PropertyKey msgKey : msgKeys) {
+      _accessor.removeProperty(msgKey);
+    }
+  }
+
+  /**
+   * enable/disable partitions on a participant
+   * @param enabled
+   * @param participantId
+   * @param resourceId
+   * @param partitionIdSet
+   */
+  void enablePartitionsForParticipant(final boolean enabled, final ParticipantId participantId,
+      final ResourceId resourceId, final Set<PartitionId> partitionIdSet) {
+    String participantName = participantId.stringify();
+    String resourceName = resourceId.stringify();
+
+    // check instanceConfig exists
+    PropertyKey instanceConfigKey = _keyBuilder.instanceConfig(participantName);
+    if (_accessor.getProperty(instanceConfigKey) == null) {
+      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
+          + _clusterId);
+      return;
+    }
+
+    // check resource exist. warn if not
+    IdealState idealState = _accessor.getProperty(_keyBuilder.idealState(resourceName));
+    if (idealState == null) {
+      LOG.warn("Disable partitions: " + partitionIdSet + " but Cluster: " + _clusterId
+          + ", resource: " + resourceId
+          + " does NOT exists. probably disable it during ERROR->DROPPED transtition");
+
+    } else {
+      // check partitions exist. warn if not
+      for (PartitionId partitionId : partitionIdSet) {
+        if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO && idealState
+            .getPreferenceList(partitionId) == null)
+            || (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED && idealState
+                .getParticipantStateMap(partitionId) == null)) {
+          LOG.warn("Cluster: " + _clusterId + ", resource: " + resourceId + ", partition: "
+              + partitionId + ", partition does NOT exist in ideal state");
+        }
+      }
+    }
+
+    // TODO merge list logic should go to znrecord updater
+    // update participantConfig
+    // could not use ZNRecordUpdater since it doesn't do listField merge/subtract
+    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
+    final List<String> partitionNames = new ArrayList<String>();
+    for (PartitionId partitionId : partitionIdSet) {
+      partitionNames.add(partitionId.stringify());
+    }
+
+    baseAccessor.update(instanceConfigKey.getPath(), new DataUpdater<ZNRecord>() {
+      @Override
+      public ZNRecord update(ZNRecord currentData) {
+        if (currentData == null) {
+          throw new HelixException("Cluster: " + _clusterId + ", instance: " + participantId
+              + ", participant config is null");
+        }
+
+        // TODO: merge with InstanceConfig.setInstanceEnabledForPartition
+        List<String> list =
+            currentData.getListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString());
+        Set<String> disabledPartitions = new HashSet<String>();
+        if (list != null) {
+          disabledPartitions.addAll(list);
+        }
+
+        if (enabled) {
+          disabledPartitions.removeAll(partitionNames);
+        } else {
+          disabledPartitions.addAll(partitionNames);
+        }
+
+        list = new ArrayList<String>(disabledPartitions);
+        Collections.sort(list);
+        currentData.setListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString(), list);
+        return currentData;
+      }
+    }, AccessOption.PERSISTENT);
+  }
+
+  /**
+   * disable partitions on a participant
+   * @param participantId
+   * @param resourceId
+   * @param disablePartitionIdSet
+   */
+  public void disablePartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
+      Set<PartitionId> disablePartitionIdSet) {
+    enablePartitionsForParticipant(false, participantId, resourceId, disablePartitionIdSet);
+  }
+
+  /**
+   * enable partitions on a participant
+   * @param participantId
+   * @param resourceId
+   * @param enablePartitionIdSet
+   */
+  public void enablePartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
+      Set<PartitionId> enablePartitionIdSet) {
+    enablePartitionsForParticipant(true, participantId, resourceId, enablePartitionIdSet);
+  }
+
+  /**
+   * reset partitions on a participant
+   * @param participantId
+   * @param resourceId
+   * @param resetPartitionIdSet
+   */
+  public void resetPartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
+      Set<PartitionId> resetPartitionIdSet) {
+    // TODO impl this
+  }
+
+  /**
+   * Update a participant configuration
+   * @param participantId the participant to update
+   * @param participantDelta changes to the participant
+   * @return ParticipantConfig, or null if participant is not persisted
+   */
+  public ParticipantConfig updateParticipant(ParticipantId participantId,
+      ParticipantConfig.Delta participantDelta) {
+    Participant participant = readParticipant(participantId);
+    if (participant == null) {
+      LOG.error("Participant " + participantId + " does not exist, cannot be updated");
+      return null;
+    }
+    ParticipantConfig config = participantDelta.mergeInto(participant.getConfig());
+    setParticipant(config);
+    return config;
+  }
+
+  /**
+   * Set the configuration of an existing participant
+   * @param participantConfig participant configuration
+   * @return true if config was set, false if there was an error
+   */
+  public boolean setParticipant(ParticipantConfig participantConfig) {
+    if (participantConfig == null) {
+      LOG.error("Participant config not initialized");
+      return false;
+    }
+    InstanceConfig instanceConfig = new InstanceConfig(participantConfig.getId());
+    instanceConfig.setHostName(participantConfig.getHostName());
+    instanceConfig.setPort(Integer.toString(participantConfig.getPort()));
+    for (String tag : participantConfig.getTags()) {
+      instanceConfig.addTag(tag);
+    }
+    for (PartitionId partitionId : participantConfig.getDisabledPartitions()) {
+      instanceConfig.setInstanceEnabledForPartition(partitionId, false);
+    }
+    instanceConfig.setInstanceEnabled(participantConfig.isEnabled());
+    instanceConfig.addNamespacedConfig(participantConfig.getUserConfig());
+    _accessor.setProperty(_keyBuilder.instanceConfig(participantConfig.getId().stringify()),
+        instanceConfig);
+    return true;
+  }
+
+  /**
+   * create a participant based on physical model
+   * @param participantId
+   * @param instanceConfig
+   * @param userConfig
+   * @param liveInstance
+   * @param instanceMsgMap map of message-id to message
+   * @param instanceCurStateMap map of resource-id to current-state
+   * @return participant
+   */
+  static Participant createParticipant(ParticipantId participantId, InstanceConfig instanceConfig,
+      UserConfig userConfig, LiveInstance liveInstance, Map<String, Message> instanceMsgMap,
+      Map<String, CurrentState> instanceCurStateMap) {
+
+    String hostName = instanceConfig.getHostName();
+
+    int port = -1;
+    try {
+      port = Integer.parseInt(instanceConfig.getPort());
+    } catch (IllegalArgumentException e) {
+      // keep as -1
+    }
+    if (port < 0 || port > 65535) {
+      port = -1;
+    }
+    boolean isEnabled = instanceConfig.getInstanceEnabled();
+
+    List<String> disabledPartitions = instanceConfig.getDisabledPartitions();
+    Set<PartitionId> disabledPartitionIdSet = Collections.emptySet();
+    if (disabledPartitions != null) {
+      disabledPartitionIdSet = new HashSet<PartitionId>();
+      for (String partitionId : disabledPartitions) {
+        disabledPartitionIdSet.add(PartitionId.from(PartitionId.extractResourceId(partitionId),
+            PartitionId.stripResourceId(partitionId)));
+      }
+    }
+
+    Set<String> tags = new HashSet<String>(instanceConfig.getTags());
+
+    RunningInstance runningInstance = null;
+    if (liveInstance != null) {
+      runningInstance =
+          new RunningInstance(liveInstance.getSessionId(), liveInstance.getHelixVersion(),
+              liveInstance.getProcessId());
+    }
+
+    Map<MessageId, Message> msgMap = new HashMap<MessageId, Message>();
+    if (instanceMsgMap != null) {
+      for (String msgId : instanceMsgMap.keySet()) {
+        Message message = instanceMsgMap.get(msgId);
+        msgMap.put(MessageId.from(msgId), message);
+      }
+    }
+
+    Map<ResourceId, CurrentState> curStateMap = new HashMap<ResourceId, CurrentState>();
+    if (instanceCurStateMap != null) {
+
+      for (String resourceName : instanceCurStateMap.keySet()) {
+        curStateMap.put(ResourceId.from(resourceName), instanceCurStateMap.get(resourceName));
+      }
+    }
+
+    return new Participant(participantId, hostName, port, isEnabled, disabledPartitionIdSet, tags,
+        runningInstance, curStateMap, msgMap, userConfig);
+  }
+
+  /**
+   * read participant related data
+   * @param participantId
+   * @return participant, or null if participant not available
+   */
+  public Participant readParticipant(ParticipantId participantId) {
+    // read physical model
+    String participantName = participantId.stringify();
+    InstanceConfig instanceConfig = _accessor.getProperty(_keyBuilder.instance(participantName));
+
+    if (instanceConfig == null) {
+      LOG.error("Participant " + participantId + " is not present on the cluster");
+      return null;
+    }
+
+    UserConfig userConfig = UserConfig.from(instanceConfig);
+    LiveInstance liveInstance = _accessor.getProperty(_keyBuilder.liveInstance(participantName));
+
+    Map<String, Message> instanceMsgMap = Collections.emptyMap();
+    Map<String, CurrentState> instanceCurStateMap = Collections.emptyMap();
+    if (liveInstance != null) {
+      SessionId sessionId = liveInstance.getSessionId();
+
+      instanceMsgMap = _accessor.getChildValuesMap(_keyBuilder.messages(participantName));
+      instanceCurStateMap =
+          _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName,
+              sessionId.stringify()));
+    }
+
+    return createParticipant(participantId, instanceConfig, userConfig, liveInstance,
+        instanceMsgMap, instanceCurStateMap);
+  }
+
+  /**
+   * update resource current state of a participant
+   * @param resourceId resource id
+   * @param participantId participant id
+   * @param sessionId session id
+   * @param curStateUpdate current state change delta
+   */
+  public void updateCurrentState(ResourceId resourceId, ParticipantId participantId,
+      SessionId sessionId, CurrentState curStateUpdate) {
+    _accessor.updateProperty(
+        _keyBuilder.currentState(participantId.stringify(), sessionId.stringify(),
+            resourceId.stringify()), curStateUpdate);
+  }
+
+  /**
+   * drop resource current state of a participant
+   * @param resourceId resource id
+   * @param participantId participant id
+   * @param sessionId session id
+   */
+  public void dropCurrentState(ResourceId resourceId, ParticipantId participantId,
+      SessionId sessionId) {
+    _accessor.removeProperty(_keyBuilder.currentState(participantId.stringify(),
+        sessionId.stringify(), resourceId.stringify()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/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
new file mode 100644
index 0000000..cd55684
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
@@ -0,0 +1,265 @@
+package org.apache.helix.api.accessor;
+
+/*
+ * 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.HelixConstants.StateModelToken;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.Resource;
+import org.apache.helix.api.Scope;
+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.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext;
+import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.IdealState.RebalanceMode;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.ResourceConfiguration;
+import org.apache.log4j.Logger;
+
+public class ResourceAccessor {
+  private static final Logger LOG = Logger.getLogger(ResourceAccessor.class);
+  private final HelixDataAccessor _accessor;
+  private final PropertyKey.Builder _keyBuilder;
+
+  public ResourceAccessor(HelixDataAccessor accessor) {
+    _accessor = accessor;
+    _keyBuilder = accessor.keyBuilder();
+  }
+
+  /**
+   * Read a single snapshot of a resource
+   * @param resourceId the resource id to read
+   * @return Resource or null if not present
+   */
+  public Resource readResource(ResourceId resourceId) {
+    ResourceConfiguration config =
+        _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
+    IdealState idealState = _accessor.getProperty(_keyBuilder.idealState(resourceId.stringify()));
+
+    if (config == null && idealState == null) {
+      LOG.error("Resource " + resourceId + " not present on the cluster");
+      return null;
+    }
+
+    ExternalView externalView =
+        _accessor.getProperty(_keyBuilder.externalView(resourceId.stringify()));
+    ResourceAssignment resourceAssignment =
+        _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify()));
+    return createResource(resourceId, config, idealState, externalView, resourceAssignment);
+  }
+
+  /**
+   * Update a resource configuration
+   * @param resourceId the resource id to update
+   * @param resourceDelta changes to the resource
+   * @return ResourceConfig, or null if the resource is not persisted
+   */
+  public ResourceConfig updateResource(ResourceId resourceId, ResourceConfig.Delta resourceDelta) {
+    Resource resource = readResource(resourceId);
+    if (resource == null) {
+      LOG.error("Resource " + resourceId + " does not exist, cannot be updated");
+      return null;
+    }
+    ResourceConfig config = resourceDelta.mergeInto(resource.getConfig());
+    setResource(config);
+    return config;
+  }
+
+  /**
+   * save resource assignment
+   * @param resourceId
+   * @param resourceAssignment
+   */
+  public void setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) {
+    _accessor.setProperty(_keyBuilder.resourceAssignment(resourceId.stringify()),
+        resourceAssignment);
+  }
+
+  /**
+   * get resource assignment
+   * @param resourceId
+   * @return resource assignment or null
+   */
+  public ResourceAssignment getResourceAssignment(ResourceId resourceId) {
+    return _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify()));
+  }
+
+  /**
+   * Set a physical resource configuration, which may include user-defined configuration, as well as
+   * rebalancer configuration
+   * @param resourceId
+   * @param configuration
+   */
+  void setConfiguration(ResourceId resourceId, ResourceConfiguration configuration) {
+    _accessor.setProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration);
+    // also set an ideal state if the resource supports it
+    RebalancerConfig rebalancerConfig = new RebalancerConfig(configuration);
+    IdealState idealState =
+        rebalancerConfigToIdealState(rebalancerConfig, configuration.getBucketSize(),
+            configuration.getBatchMessageMode());
+    if (idealState != null) {
+      _accessor.setProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
+    }
+  }
+
+  /**
+   * Persist an existing resource's logical configuration
+   * @param resourceConfig logical resource configuration
+   * @return true if resource is set, false otherwise
+   */
+  public boolean setResource(ResourceConfig resourceConfig) {
+    if (resourceConfig == null || resourceConfig.getRebalancerConfig() == null) {
+      LOG.error("Resource not fully defined with a rebalancer context");
+      return false;
+    }
+    ResourceId resourceId = resourceConfig.getId();
+    ResourceConfiguration config = new ResourceConfiguration(resourceId);
+    config.addNamespacedConfig(resourceConfig.getUserConfig());
+    config.addNamespacedConfig(resourceConfig.getRebalancerConfig().toNamespacedConfig());
+    config.setBucketSize(resourceConfig.getBucketSize());
+    config.setBatchMessageMode(resourceConfig.getBatchMessageMode());
+    setConfiguration(resourceId, config);
+    return true;
+  }
+
+  /**
+   * Get a resource configuration, which may include user-defined configuration, as well as
+   * rebalancer configuration
+   * @param resourceId
+   * @return configuration
+   */
+  public void getConfiguration(ResourceId resourceId) {
+    _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
+  }
+
+  /**
+   * set external view of a resource
+   * @param resourceId
+   * @param extView
+   */
+  public void setExternalView(ResourceId resourceId, ExternalView extView) {
+    _accessor.setProperty(_keyBuilder.externalView(resourceId.stringify()), extView);
+  }
+
+  /**
+   * drop external view of a resource
+   * @param resourceId
+   */
+  public void dropExternalView(ResourceId resourceId) {
+    _accessor.removeProperty(_keyBuilder.externalView(resourceId.stringify()));
+  }
+
+  /**
+   * Get an ideal state from a rebalancer config if the resource is partitioned
+   * @param config RebalancerConfig instance
+   * @param bucketSize bucket size to use
+   * @param batchMessageMode true if batch messaging allowed, false otherwise
+   * @return IdealState, or null
+   */
+  static IdealState rebalancerConfigToIdealState(RebalancerConfig config, int bucketSize,
+      boolean batchMessageMode) {
+    PartitionedRebalancerContext partitionedContext =
+        config.getRebalancerContext(PartitionedRebalancerContext.class);
+    if (partitionedContext != null) {
+      IdealState idealState = new IdealState(partitionedContext.getResourceId());
+      idealState.setRebalanceMode(partitionedContext.getRebalanceMode());
+      idealState.setRebalancerRef(partitionedContext.getRebalancerRef());
+      String replicas = null;
+      if (partitionedContext.anyLiveParticipant()) {
+        replicas = StateModelToken.ANY_LIVEINSTANCE.toString();
+      } else {
+        replicas = Integer.toString(partitionedContext.getReplicaCount());
+      }
+      idealState.setReplicas(replicas);
+      idealState.setNumPartitions(partitionedContext.getPartitionSet().size());
+      idealState.setInstanceGroupTag(partitionedContext.getParticipantGroupTag());
+      idealState.setMaxPartitionsPerInstance(partitionedContext.getMaxPartitionsPerParticipant());
+      idealState.setStateModelDefId(partitionedContext.getStateModelDefId());
+      idealState.setStateModelFactoryId(partitionedContext.getStateModelFactoryId());
+      idealState.setBucketSize(bucketSize);
+      idealState.setBatchMessageMode(batchMessageMode);
+      if (partitionedContext.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
+        SemiAutoRebalancerContext semiAutoContext =
+            config.getRebalancerContext(SemiAutoRebalancerContext.class);
+        for (PartitionId partitionId : semiAutoContext.getPartitionSet()) {
+          idealState.setPreferenceList(partitionId, semiAutoContext.getPreferenceList(partitionId));
+        }
+      } else if (partitionedContext.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
+        CustomRebalancerContext customContext =
+            config.getRebalancerContext(CustomRebalancerContext.class);
+        for (PartitionId partitionId : customContext.getPartitionSet()) {
+          idealState.setParticipantStateMap(partitionId,
+              customContext.getPreferenceMap(partitionId));
+        }
+      }
+      return idealState;
+    }
+    return null;
+  }
+
+  /**
+   * Create a resource snapshot instance from the physical model
+   * @param resourceId the resource id
+   * @param resourceConfiguration physical resource configuration
+   * @param idealState ideal state of the resource
+   * @param externalView external view of the resource
+   * @param resourceAssignment current resource assignment
+   * @return Resource
+   */
+  static Resource createResource(ResourceId resourceId,
+      ResourceConfiguration resourceConfiguration, IdealState idealState,
+      ExternalView externalView, ResourceAssignment resourceAssignment) {
+    UserConfig userConfig;
+    ResourceType type = ResourceType.DATA;
+    if (resourceConfiguration != null) {
+      userConfig = UserConfig.from(resourceConfiguration);
+      type = resourceConfiguration.getType();
+    } else {
+      userConfig = new UserConfig(Scope.resource(resourceId));
+    }
+    int bucketSize = 0;
+    boolean batchMessageMode = false;
+    RebalancerContext rebalancerContext;
+    if (idealState != null) {
+      rebalancerContext = PartitionedRebalancerContext.from(idealState);
+      bucketSize = idealState.getBucketSize();
+      batchMessageMode = idealState.getBatchMessageMode();
+    } else {
+      if (resourceConfiguration != null) {
+        bucketSize = resourceConfiguration.getBucketSize();
+        batchMessageMode = resourceConfiguration.getBatchMessageMode();
+        RebalancerConfig rebalancerConfig = new RebalancerConfig(resourceConfiguration);
+        rebalancerContext = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
+      } else {
+        rebalancerContext = new PartitionedRebalancerContext(RebalanceMode.NONE);
+      }
+    }
+    return new Resource(resourceId, type, idealState, resourceAssignment, externalView,
+        rebalancerContext, userConfig, bucketSize, batchMessageMode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/accessor/StateModelDefinitionAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/StateModelDefinitionAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/StateModelDefinitionAccessor.java
new file mode 100644
index 0000000..3816507
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/StateModelDefinitionAccessor.java
@@ -0,0 +1,70 @@
+package org.apache.helix.api.accessor;
+
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.model.StateModelDefinition;
+
+import com.google.common.collect.ImmutableMap;
+
+public class StateModelDefinitionAccessor {
+  private final HelixDataAccessor _accessor;
+  private final PropertyKey.Builder _keyBuilder;
+
+  /**
+   * @param accessor
+   */
+  public StateModelDefinitionAccessor(HelixDataAccessor accessor) {
+    _accessor = accessor;
+    _keyBuilder = accessor.keyBuilder();
+  }
+
+  /**
+   * Get all of the state model definitions available to the cluster
+   * @return map of state model ids to state model definition objects
+   */
+  public Map<StateModelDefId, StateModelDefinition> readStateModelDefinitions() {
+    Map<String, StateModelDefinition> stateModelDefs =
+        _accessor.getChildValuesMap(_keyBuilder.stateModelDefs());
+    Map<StateModelDefId, StateModelDefinition> stateModelDefMap =
+        new HashMap<StateModelDefId, StateModelDefinition>();
+
+    for (String stateModelDefName : stateModelDefs.keySet()) {
+      stateModelDefMap.put(StateModelDefId.from(stateModelDefName),
+          stateModelDefs.get(stateModelDefName));
+    }
+
+    return ImmutableMap.copyOf(stateModelDefMap);
+  }
+
+  /**
+   * Set a state model definition. Adds the state model definition if it does not exist
+   * @param stateModelDef fully initialized state model definition
+   * @return true if the model is persisted, false otherwise
+   */
+  public boolean setStateModelDefinition(StateModelDefinition stateModelDef) {
+    return _accessor.setProperty(_keyBuilder.stateModelDef(stateModelDef.getId()), stateModelDef);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
new file mode 100644
index 0000000..79b4f61
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ClusterConfig.java
@@ -0,0 +1,696 @@
+package org.apache.helix.api.config;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.State;
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.ConstraintId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
+import org.apache.helix.model.ClusterConstraints;
+import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
+import org.apache.helix.model.ClusterConstraints.ConstraintType;
+import org.apache.helix.model.ClusterConstraints.ConstraintValue;
+import org.apache.helix.model.ConstraintItem;
+import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.model.Transition;
+import org.apache.helix.model.builder.ConstraintItemBuilder;
+import org.apache.log4j.Logger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/*
+ * 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.
+ */
+
+/**
+ * Configuration properties of a cluster
+ */
+public class ClusterConfig {
+  private static final Logger LOG = Logger.getLogger(ClusterConfig.class);
+
+  private final ClusterId _id;
+  private final Map<ResourceId, ResourceConfig> _resourceMap;
+  private final Map<ParticipantId, ParticipantConfig> _participantMap;
+  private final Map<ConstraintType, ClusterConstraints> _constraintMap;
+  private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
+  private final UserConfig _userConfig;
+  private final boolean _isPaused;
+
+  /**
+   * Initialize a cluster configuration. Also see ClusterConfig.Builder
+   * @param id cluster id
+   * @param resourceMap map of resource id to resource config
+   * @param participantMap map of participant id to participant config
+   * @param constraintMap map of constraint type to all constraints of that type
+   * @param stateModelMap map of state model id to state model definition
+   * @param userConfig user-defined cluster properties
+   * @param isPaused true if paused, false if active
+   */
+  private ClusterConfig(ClusterId id, Map<ResourceId, ResourceConfig> resourceMap,
+      Map<ParticipantId, ParticipantConfig> participantMap,
+      Map<ConstraintType, ClusterConstraints> constraintMap,
+      Map<StateModelDefId, StateModelDefinition> stateModelMap, UserConfig userConfig,
+      boolean isPaused) {
+    _id = id;
+    _resourceMap = ImmutableMap.copyOf(resourceMap);
+    _participantMap = ImmutableMap.copyOf(participantMap);
+    _constraintMap = ImmutableMap.copyOf(constraintMap);
+    _stateModelMap = ImmutableMap.copyOf(stateModelMap);
+    _userConfig = userConfig;
+    _isPaused = isPaused;
+  }
+
+  /**
+   * Get cluster id
+   * @return cluster id
+   */
+  public ClusterId getId() {
+    return _id;
+  }
+
+  /**
+   * Get resources in the cluster
+   * @return a map of resource id to resource, or empty map if none
+   */
+  public Map<ResourceId, ResourceConfig> getResourceMap() {
+    return _resourceMap;
+  }
+
+  /**
+   * Get all the constraints on the cluster
+   * @return map of constraint type to constraints
+   */
+  public Map<ConstraintType, ClusterConstraints> getConstraintMap() {
+    return _constraintMap;
+  }
+
+  /**
+   * Get the maximum number of participants that can be in a state
+   * @param scope the scope for the bound
+   * @param stateModelDefId the state model of the state
+   * @param state the constrained state
+   * @return The upper bound, which can be "-1" if unspecified, a numerical upper bound, "R" for
+   *         number of replicas, or "N" for number of participants
+   */
+  public String getStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+      State state) {
+    // set up attributes to match based on the scope
+    ClusterConstraints stateConstraints = getConstraintMap().get(ConstraintType.STATE_CONSTRAINT);
+    Map<ConstraintAttribute, String> matchAttributes = Maps.newHashMap();
+    matchAttributes.put(ConstraintAttribute.STATE, state.toString());
+    matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString());
+    switch (scope.getType()) {
+    case CLUSTER:
+      // cluster is implicit
+      break;
+    case RESOURCE:
+      matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify());
+      break;
+    default:
+      LOG.error("Unsupported scope for state constraint: " + scope);
+      return "-1";
+    }
+    Set<ConstraintItem> matches = stateConstraints.match(matchAttributes);
+    int value = -1;
+    for (ConstraintItem item : matches) {
+      // match: if an R or N is found, always choose that one
+      // otherwise, take the minimum of the counts specified in the constraints
+      String constraintValue = item.getConstraintValue();
+      if (constraintValue != null) {
+        if (constraintValue.equals(ConstraintValue.N.toString())
+            || constraintValue.equals(ConstraintValue.R.toString())) {
+          return constraintValue;
+        } else {
+          try {
+            int current = Integer.parseInt(constraintValue);
+            if (value == -1 || current < value) {
+              value = current;
+            }
+          } catch (NumberFormatException e) {
+            LOG.error("Invalid state upper bound: " + constraintValue);
+          }
+        }
+      }
+    }
+    return Integer.toString(value);
+  }
+
+  /**
+   * Get the limit of simultaneous execution of a transition
+   * @param scope the scope under which the transition is constrained
+   * @param stateModelDefId the state model of which the transition is a part
+   * @param transition the constrained transition
+   * @return the limit, or Integer.MAX_VALUE if there is no limit
+   */
+  public int getTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+      Transition transition) {
+    // set up attributes to match based on the scope
+    ClusterConstraints transitionConstraints =
+        getConstraintMap().get(ConstraintType.MESSAGE_CONSTRAINT);
+    Map<ConstraintAttribute, String> matchAttributes = Maps.newHashMap();
+    matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString());
+    matchAttributes.put(ConstraintAttribute.MESSAGE_TYPE, MessageType.STATE_TRANSITION.toString());
+    matchAttributes.put(ConstraintAttribute.TRANSITION, transition.toString());
+    switch (scope.getType()) {
+    case CLUSTER:
+      // cluster is implicit
+      break;
+    case RESOURCE:
+      matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify());
+      break;
+    case PARTICIPANT:
+      matchAttributes.put(ConstraintAttribute.INSTANCE, scope.getScopedId().stringify());
+      break;
+    default:
+      LOG.error("Unsupported scope for transition constraints: " + scope);
+      return Integer.MAX_VALUE;
+    }
+    Set<ConstraintItem> matches = transitionConstraints.match(matchAttributes);
+    int value = Integer.MAX_VALUE;
+    for (ConstraintItem item : matches) {
+      String constraintValue = item.getConstraintValue();
+      if (constraintValue != null) {
+        try {
+          int current = Integer.parseInt(constraintValue);
+          if (current < value) {
+            value = current;
+          }
+        } catch (NumberFormatException e) {
+          LOG.error("Invalid in-flight transition cap: " + constraintValue);
+        }
+      }
+    }
+    return value;
+  }
+
+  /**
+   * Get participants of the cluster
+   * @return a map of participant id to participant, or empty map if none
+   */
+  public Map<ParticipantId, ParticipantConfig> getParticipantMap() {
+    return _participantMap;
+  }
+
+  /**
+   * Get all the state model definitions on the cluster
+   * @return map of state model definition id to state model definition
+   */
+  public Map<StateModelDefId, StateModelDefinition> getStateModelMap() {
+    return _stateModelMap;
+  }
+
+  /**
+   * Get user-specified configuration properties of this cluster
+   * @return UserConfig properties
+   */
+  public UserConfig getUserConfig() {
+    return _userConfig;
+  }
+
+  /**
+   * Check the paused status of the cluster
+   * @return true if paused, false otherwise
+   */
+  public boolean isPaused() {
+    return _isPaused;
+  }
+
+  /**
+   * Update context for a ClusterConfig
+   */
+  public static class Delta {
+    private enum Fields {
+      USER_CONFIG
+    }
+
+    private Set<Fields> _updateFields;
+    private Map<ConstraintType, Set<ConstraintId>> _removedConstraints;
+    private Builder _builder;
+
+    /**
+     * Instantiate the delta for a cluster config
+     * @param clusterId the cluster to update
+     */
+    public Delta(ClusterId clusterId) {
+      _updateFields = Sets.newHashSet();
+      _removedConstraints = Maps.newHashMap();
+      for (ConstraintType type : ConstraintType.values()) {
+        Set<ConstraintId> constraints = Sets.newHashSet();
+        _removedConstraints.put(type, constraints);
+      }
+      _builder = new Builder(clusterId);
+    }
+
+    /**
+     * Add a state upper bound constraint
+     * @param scope scope under which the constraint is valid
+     * @param stateModelDefId identifier of the state model that owns the state
+     * @param state the state to constrain
+     * @param upperBound maximum number of replicas per partition in the state
+     * @return Delta
+     */
+    public Delta addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        State state, int upperBound) {
+      return addStateUpperBoundConstraint(scope, stateModelDefId, state,
+          Integer.toString(upperBound));
+    }
+
+    /**
+     * Add a state upper bound constraint
+     * @param scope scope under which the constraint is valid
+     * @param stateModelDefId identifier of the state model that owns the state
+     * @param state the state to constrain
+     * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a
+     *          number, or the currently supported special bound values:<br />
+     *          "R" - Refers to the number of replicas specified during resource
+     *          creation. This allows having different replication factor for each
+     *          resource without having to create a different state machine. <br />
+     *          "N" - Refers to all nodes in the cluster. Useful for resources that need
+     *          to exist on all nodes. This way one can add/remove nodes without having
+     *          the change the bounds.
+     * @return Delta
+     */
+    public Delta addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        State state, String dynamicUpperBound) {
+      _builder.addStateUpperBoundConstraint(scope, stateModelDefId, state, dynamicUpperBound);
+      return this;
+    }
+
+    /**
+     * Remove state upper bound constraint
+     * @param scope scope under which the constraint is valid
+     * @param stateModelDefId identifier of the state model that owns the state
+     * @param state the state to constrain
+     * @return Delta
+     */
+    public Delta removeStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        State state) {
+      _removedConstraints.get(ConstraintType.STATE_CONSTRAINT).add(
+          ConstraintId.from(scope, stateModelDefId, state));
+      return this;
+    }
+
+    /**
+     * Add a constraint on the maximum number of in-flight transitions of a certain type
+     * @param scope scope of the constraint
+     * @param stateModelDefId identifies the state model containing the transition
+     * @param transition the transition to constrain
+     * @param maxInFlightTransitions number of allowed in-flight transitions in the scope
+     * @return Delta
+     */
+    public Delta addTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        Transition transition, int maxInFlightTransitions) {
+      _builder.addTransitionConstraint(scope, stateModelDefId, transition, maxInFlightTransitions);
+      return this;
+    }
+
+    /**
+     * Remove a constraint on the maximum number of in-flight transitions of a certain type
+     * @param scope scope of the constraint
+     * @param stateModelDefId identifies the state model containing the transition
+     * @param transition the transition to constrain
+     * @return Delta
+     */
+    public Delta removeTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        Transition transition) {
+      _removedConstraints.get(ConstraintType.MESSAGE_CONSTRAINT).add(
+          ConstraintId.from(scope, stateModelDefId, transition));
+      return this;
+    }
+
+    /**
+     * Add a single constraint item
+     * @param type type of the constraint item
+     * @param constraintId unique constraint id
+     * @param item instantiated ConstraintItem
+     * @return Delta
+     */
+    public Delta addConstraintItem(ConstraintType type, ConstraintId constraintId,
+        ConstraintItem item) {
+      _builder.addConstraint(type, constraintId, item);
+      return this;
+    }
+
+    /**
+     * Remove a single constraint item
+     * @param type type of the constraint item
+     * @param constraintId unique constraint id
+     * @return Delta
+     */
+    public Delta removeConstraintItem(ConstraintType type, ConstraintId constraintId) {
+      _removedConstraints.get(type).add(constraintId);
+      return this;
+    }
+
+    /*
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Delta setUserConfig(UserConfig userConfig) {
+      _builder.userConfig(userConfig);
+      _updateFields.add(Fields.USER_CONFIG);
+      return this;
+    }
+
+    /**
+     * Create a ClusterConfig that is the combination of an existing ClusterConfig and this delta
+     * @param orig the original ClusterConfig
+     * @return updated ClusterConfig
+     */
+    public ClusterConfig mergeInto(ClusterConfig orig) {
+      // copy in original and updated fields
+      ClusterConfig deltaConfig = _builder.build();
+      Builder builder =
+          new Builder(orig.getId()).addResources(orig.getResourceMap().values())
+              .addParticipants(orig.getParticipantMap().values())
+              .addStateModelDefinitions(orig.getStateModelMap().values())
+              .userConfig(orig.getUserConfig()).pausedStatus(orig.isPaused());
+      for (Fields field : _updateFields) {
+        switch (field) {
+        case USER_CONFIG:
+          builder.userConfig(deltaConfig.getUserConfig());
+          break;
+        }
+      }
+      // add constraint deltas
+      for (ConstraintType type : ConstraintType.values()) {
+        ClusterConstraints constraints;
+        if (orig.getConstraintMap().containsKey(type)) {
+          constraints = orig.getConstraintMap().get(type);
+        } else {
+          constraints = new ClusterConstraints(type);
+        }
+        // add new constraints
+        if (deltaConfig.getConstraintMap().containsKey(type)) {
+          ClusterConstraints deltaConstraints = deltaConfig.getConstraintMap().get(type);
+          for (ConstraintId constraintId : deltaConstraints.getConstraintItems().keySet()) {
+            ConstraintItem constraintItem = deltaConstraints.getConstraintItem(constraintId);
+            constraints.addConstraintItem(constraintId, constraintItem);
+          }
+        }
+        // remove constraints
+        for (ConstraintId constraintId : _removedConstraints.get(type)) {
+          constraints.removeConstraintItem(constraintId);
+        }
+        builder.addConstraint(constraints);
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Assembles a cluster configuration
+   */
+  public static class Builder {
+    private final ClusterId _id;
+    private final Map<ResourceId, ResourceConfig> _resourceMap;
+    private final Map<ParticipantId, ParticipantConfig> _participantMap;
+    private final Map<ConstraintType, ClusterConstraints> _constraintMap;
+    private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
+    private UserConfig _userConfig;
+    private boolean _isPaused;
+
+    /**
+     * Initialize builder for a cluster
+     * @param id cluster id
+     */
+    public Builder(ClusterId id) {
+      _id = id;
+      _resourceMap = new HashMap<ResourceId, ResourceConfig>();
+      _participantMap = new HashMap<ParticipantId, ParticipantConfig>();
+      _constraintMap = new HashMap<ConstraintType, ClusterConstraints>();
+      _stateModelMap = new HashMap<StateModelDefId, StateModelDefinition>();
+      _isPaused = false;
+      _userConfig = new UserConfig(Scope.cluster(id));
+    }
+
+    /**
+     * Add a resource to the cluster
+     * @param resource resource configuration
+     * @return Builder
+     */
+    public Builder addResource(ResourceConfig resource) {
+      _resourceMap.put(resource.getId(), resource);
+      return this;
+    }
+
+    /**
+     * Add multiple resources to the cluster
+     * @param resources resource configurations
+     * @return Builder
+     */
+    public Builder addResources(Collection<ResourceConfig> resources) {
+      for (ResourceConfig resource : resources) {
+        addResource(resource);
+      }
+      return this;
+    }
+
+    /**
+     * Add a participant to the cluster
+     * @param participant participant configuration
+     * @return Builder
+     */
+    public Builder addParticipant(ParticipantConfig participant) {
+      _participantMap.put(participant.getId(), participant);
+      return this;
+    }
+
+    /**
+     * Add multiple participants to the cluster
+     * @param participants participant configurations
+     * @return Builder
+     */
+    public Builder addParticipants(Collection<ParticipantConfig> participants) {
+      for (ParticipantConfig participant : participants) {
+        addParticipant(participant);
+      }
+      return this;
+    }
+
+    /**
+     * Add a constraint to the cluster
+     * @param constraint cluster constraint of a specific type
+     * @return Builder
+     */
+    public Builder addConstraint(ClusterConstraints constraint) {
+      ClusterConstraints existConstraints = getConstraintsInstance(constraint.getType());
+      for (ConstraintId constraintId : constraint.getConstraintItems().keySet()) {
+        existConstraints
+            .addConstraintItem(constraintId, constraint.getConstraintItem(constraintId));
+      }
+      return this;
+    }
+
+    /**
+     * Add a single constraint item
+     * @param type type of the constraint
+     * @param constraintId unique constraint identifier
+     * @param item instantiated ConstraintItem
+     * @return Builder
+     */
+    public Builder addConstraint(ConstraintType type, ConstraintId constraintId, ConstraintItem item) {
+      ClusterConstraints existConstraints = getConstraintsInstance(type);
+      existConstraints.addConstraintItem(constraintId, item);
+      return this;
+    }
+
+    /**
+     * Add multiple constraints to the cluster
+     * @param constraints cluster constraints of multiple distinct types
+     * @return Builder
+     */
+    public Builder addConstraints(Collection<ClusterConstraints> constraints) {
+      for (ClusterConstraints constraint : constraints) {
+        addConstraint(constraint);
+      }
+      return this;
+    }
+
+    /**
+     * Add a constraint on the maximum number of in-flight transitions of a certain type
+     * @param scope scope of the constraint
+     * @param stateModelDefId identifies the state model containing the transition
+     * @param transition the transition to constrain
+     * @param maxInFlightTransitions number of allowed in-flight transitions in the scope
+     * @return Builder
+     */
+    public Builder addTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        Transition transition, int maxInFlightTransitions) {
+      Map<String, String> attributes = Maps.newHashMap();
+      attributes.put(ConstraintAttribute.MESSAGE_TYPE.toString(),
+          MessageType.STATE_TRANSITION.toString());
+      attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(),
+          Integer.toString(maxInFlightTransitions));
+      attributes.put(ConstraintAttribute.TRANSITION.toString(), transition.toString());
+      attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify());
+      switch (scope.getType()) {
+      case CLUSTER:
+        // cluster is implicit
+        break;
+      case RESOURCE:
+        attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify());
+        break;
+      case PARTICIPANT:
+        attributes.put(ConstraintAttribute.INSTANCE.toString(), scope.getScopedId().stringify());
+        break;
+      default:
+        LOG.error("Unsupported scope for adding a transition constraint: " + scope);
+        return this;
+      }
+      ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build();
+      ClusterConstraints constraints = getConstraintsInstance(ConstraintType.MESSAGE_CONSTRAINT);
+      constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, transition), item);
+      return this;
+    }
+
+    /**
+     * Add a state upper bound constraint
+     * @param scope scope under which the constraint is valid
+     * @param stateModelDefId identifier of the state model that owns the state
+     * @param state the state to constrain
+     * @param upperBound maximum number of replicas per partition in the state
+     * @return Builder
+     */
+    public Builder addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        State state, int upperBound) {
+      return addStateUpperBoundConstraint(scope, stateModelDefId, state,
+          Integer.toString(upperBound));
+    }
+
+    /**
+     * Add a state upper bound constraint
+     * @param scope scope under which the constraint is valid
+     * @param stateModelDefId identifier of the state model that owns the state
+     * @param state the state to constrain
+     * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a
+     *          number, or the currently supported special bound values:<br />
+     *          "R" - Refers to the number of replicas specified during resource
+     *          creation. This allows having different replication factor for each
+     *          resource without having to create a different state machine. <br />
+     *          "N" - Refers to all nodes in the cluster. Useful for resources that need
+     *          to exist on all nodes. This way one can add/remove nodes without having
+     *          the change the bounds.
+     * @return Builder
+     */
+    public Builder addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
+        State state, String dynamicUpperBound) {
+      Map<String, String> attributes = Maps.newHashMap();
+      attributes.put(ConstraintAttribute.STATE.toString(), state.toString());
+      attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify());
+      attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(), dynamicUpperBound);
+      switch (scope.getType()) {
+      case CLUSTER:
+        // cluster is implicit
+        break;
+      case RESOURCE:
+        attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify());
+        break;
+      default:
+        LOG.error("Unsupported scope for adding a state constraint: " + scope);
+        return this;
+      }
+      ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build();
+      ClusterConstraints constraints = getConstraintsInstance(ConstraintType.STATE_CONSTRAINT);
+      constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, state), item);
+      return this;
+    }
+
+    /**
+     * Add a state model definition to the cluster
+     * @param stateModelDef state model definition of the cluster
+     * @return Builder
+     */
+    public Builder addStateModelDefinition(StateModelDefinition stateModelDef) {
+      _stateModelMap.put(stateModelDef.getStateModelDefId(), stateModelDef);
+      // add state constraints from the state model definition
+      for (State state : stateModelDef.getStatesPriorityList()) {
+        if (!stateModelDef.getNumParticipantsPerState(state).equals("-1")) {
+          addStateUpperBoundConstraint(Scope.cluster(_id), stateModelDef.getStateModelDefId(),
+              state, stateModelDef.getNumParticipantsPerState(state));
+        }
+      }
+      return this;
+    }
+
+    /**
+     * Add multiple state model definitions
+     * @param stateModelDefs collection of state model definitions for the cluster
+     * @return Builder
+     */
+    public Builder addStateModelDefinitions(Collection<StateModelDefinition> stateModelDefs) {
+      for (StateModelDefinition stateModelDef : stateModelDefs) {
+        addStateModelDefinition(stateModelDef);
+      }
+      return this;
+    }
+
+    /**
+     * Set the paused status of the cluster
+     * @param isPaused true if paused, false otherwise
+     * @return Builder
+     */
+    public Builder pausedStatus(boolean isPaused) {
+      _isPaused = isPaused;
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Builder userConfig(UserConfig userConfig) {
+      _userConfig = userConfig;
+      return this;
+    }
+
+    /**
+     * Create the cluster configuration
+     * @return ClusterConfig
+     */
+    public ClusterConfig build() {
+      return new ClusterConfig(_id, _resourceMap, _participantMap, _constraintMap, _stateModelMap,
+          _userConfig, _isPaused);
+    }
+
+    /**
+     * Get a valid instance of ClusterConstraints for a type
+     * @param type the type
+     * @return ClusterConstraints
+     */
+    private ClusterConstraints getConstraintsInstance(ConstraintType type) {
+      ClusterConstraints constraints = _constraintMap.get(type);
+      if (constraints == null) {
+        constraints = new ClusterConstraints(type);
+        _constraintMap.put(type, constraints);
+      }
+      return constraints;
+    }
+  }
+}


[5/6] [HELIX-238] Refactor, add update to accessors, test update logic

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ParticipantAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/ParticipantAccessor.java
deleted file mode 100644
index 11e3608..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ParticipantAccessor.java
+++ /dev/null
@@ -1,393 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.I0Itec.zkclient.DataUpdater;
-import org.apache.helix.AccessOption;
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixException;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.InstanceConfig;
-import org.apache.helix.model.InstanceConfig.InstanceConfigProperty;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message;
-import org.apache.log4j.Logger;
-
-public class ParticipantAccessor {
-  private static final Logger LOG = Logger.getLogger(ParticipantAccessor.class);
-
-  private final HelixDataAccessor _accessor;
-  private final PropertyKey.Builder _keyBuilder;
-  private final ClusterId _clusterId;
-
-  public ParticipantAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
-    _clusterId = clusterId;
-    _accessor = accessor;
-    _keyBuilder = accessor.keyBuilder();
-  }
-
-  /**
-   * enable/disable a participant
-   * @param participantId
-   * @param isEnabled
-   */
-  void enableParticipant(ParticipantId participantId, boolean isEnabled) {
-    String participantName = participantId.stringify();
-    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantName)) == null) {
-      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
-          + _clusterId);
-      return;
-    }
-
-    InstanceConfig config = new InstanceConfig(participantName);
-    config.setInstanceEnabled(isEnabled);
-    _accessor.updateProperty(_keyBuilder.instanceConfig(participantName), config);
-
-  }
-
-  /**
-   * disable participant
-   * @param participantId
-   */
-  public void disableParticipant(ParticipantId participantId) {
-    enableParticipant(participantId, false);
-  }
-
-  /**
-   * enable participant
-   * @param participantId
-   */
-  public void enableParticipant(ParticipantId participantId) {
-    enableParticipant(participantId, true);
-  }
-
-  /**
-   * create messages for participant
-   * @param participantId
-   * @param msgMap map of message-id to message
-   */
-  public void insertMessagesToParticipant(ParticipantId participantId,
-      Map<MessageId, Message> msgMap) {
-    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
-    List<Message> msgs = new ArrayList<Message>();
-    for (MessageId msgId : msgMap.keySet()) {
-      msgKeys.add(_keyBuilder.message(participantId.stringify(), msgId.stringify()));
-      msgs.add(msgMap.get(msgId));
-    }
-
-    _accessor.createChildren(msgKeys, msgs);
-  }
-
-  /**
-   * set messages of participant
-   * @param participantId
-   * @param msgMap map of message-id to message
-   */
-  public void updateMessageStatus(ParticipantId participantId, Map<MessageId, Message> msgMap) {
-    String participantName = participantId.stringify();
-    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
-    List<Message> msgs = new ArrayList<Message>();
-    for (MessageId msgId : msgMap.keySet()) {
-      msgKeys.add(_keyBuilder.message(participantName, msgId.stringify()));
-      msgs.add(msgMap.get(msgId));
-    }
-    _accessor.setChildren(msgKeys, msgs);
-  }
-
-  /**
-   * delete messages from participant
-   * @param participantId
-   * @param msgIdSet
-   */
-  public void deleteMessagesFromParticipant(ParticipantId participantId, Set<MessageId> msgIdSet) {
-    String participantName = participantId.stringify();
-    List<PropertyKey> msgKeys = new ArrayList<PropertyKey>();
-    for (MessageId msgId : msgIdSet) {
-      msgKeys.add(_keyBuilder.message(participantName, msgId.stringify()));
-    }
-
-    // TODO impl batch remove
-    for (PropertyKey msgKey : msgKeys) {
-      _accessor.removeProperty(msgKey);
-    }
-  }
-
-  /**
-   * enable/disable partitions on a participant
-   * @param enabled
-   * @param participantId
-   * @param resourceId
-   * @param partitionIdSet
-   */
-  void enablePartitionsForParticipant(final boolean enabled, final ParticipantId participantId,
-      final ResourceId resourceId, final Set<PartitionId> partitionIdSet) {
-    String participantName = participantId.stringify();
-    String resourceName = resourceId.stringify();
-
-    // check instanceConfig exists
-    PropertyKey instanceConfigKey = _keyBuilder.instanceConfig(participantName);
-    if (_accessor.getProperty(instanceConfigKey) == null) {
-      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
-          + _clusterId);
-      return;
-    }
-
-    // check resource exist. warn if not
-    IdealState idealState = _accessor.getProperty(_keyBuilder.idealState(resourceName));
-    if (idealState == null) {
-      LOG.warn("Disable partitions: " + partitionIdSet + " but Cluster: " + _clusterId
-          + ", resource: " + resourceId
-          + " does NOT exists. probably disable it during ERROR->DROPPED transtition");
-
-    } else {
-      // check partitions exist. warn if not
-      for (PartitionId partitionId : partitionIdSet) {
-        if ((idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO && idealState
-            .getPreferenceList(partitionId) == null)
-            || (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED && idealState
-                .getParticipantStateMap(partitionId) == null)) {
-          LOG.warn("Cluster: " + _clusterId + ", resource: " + resourceId + ", partition: "
-              + partitionId + ", partition does NOT exist in ideal state");
-        }
-      }
-    }
-
-    // TODO merge list logic should go to znrecord updater
-    // update participantConfig
-    // could not use ZNRecordUpdater since it doesn't do listField merge/subtract
-    BaseDataAccessor<ZNRecord> baseAccessor = _accessor.getBaseDataAccessor();
-    final List<String> partitionNames = new ArrayList<String>();
-    for (PartitionId partitionId : partitionIdSet) {
-      partitionNames.add(partitionId.stringify());
-    }
-
-    baseAccessor.update(instanceConfigKey.getPath(), new DataUpdater<ZNRecord>() {
-      @Override
-      public ZNRecord update(ZNRecord currentData) {
-        if (currentData == null) {
-          throw new HelixException("Cluster: " + _clusterId + ", instance: " + participantId
-              + ", participant config is null");
-        }
-
-        // TODO: merge with InstanceConfig.setInstanceEnabledForPartition
-        List<String> list =
-            currentData.getListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString());
-        Set<String> disabledPartitions = new HashSet<String>();
-        if (list != null) {
-          disabledPartitions.addAll(list);
-        }
-
-        if (enabled) {
-          disabledPartitions.removeAll(partitionNames);
-        } else {
-          disabledPartitions.addAll(partitionNames);
-        }
-
-        list = new ArrayList<String>(disabledPartitions);
-        Collections.sort(list);
-        currentData.setListField(InstanceConfigProperty.HELIX_DISABLED_PARTITION.toString(), list);
-        return currentData;
-      }
-    }, AccessOption.PERSISTENT);
-  }
-
-  /**
-   * disable partitions on a participant
-   * @param participantId
-   * @param resourceId
-   * @param disablePartitionIdSet
-   */
-  public void disablePartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
-      Set<PartitionId> disablePartitionIdSet) {
-    enablePartitionsForParticipant(false, participantId, resourceId, disablePartitionIdSet);
-  }
-
-  /**
-   * enable partitions on a participant
-   * @param participantId
-   * @param resourceId
-   * @param enablePartitionIdSet
-   */
-  public void enablePartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
-      Set<PartitionId> enablePartitionIdSet) {
-    enablePartitionsForParticipant(true, participantId, resourceId, enablePartitionIdSet);
-  }
-
-  /**
-   * reset partitions on a participant
-   * @param participantId
-   * @param resourceId
-   * @param resetPartitionIdSet
-   */
-  public void resetPartitionsForParticipant(ParticipantId participantId, ResourceId resourceId,
-      Set<PartitionId> resetPartitionIdSet) {
-    // TODO impl this
-  }
-
-  /**
-   * Update a participant configuration
-   * @param participantId the participant to update
-   * @param participantDelta changes to the participant
-   * @return ParticipantConfig, or null if participant is not persisted
-   */
-  public ParticipantConfig updateParticipant(ParticipantId participantId,
-      ParticipantConfig.Delta participantDelta) {
-    Participant participant = readParticipant(participantId);
-    if (participant == null) {
-      LOG.error("Participant " + participantId + " does not exist, cannot be updated");
-      return null;
-    }
-    ParticipantConfig config = participantDelta.mergeInto(participant.getConfig());
-    // TODO: persist this
-    return config;
-  }
-
-  /**
-   * create a participant based on physical model
-   * @param participantId
-   * @param instanceConfig
-   * @param userConfig
-   * @param liveInstance
-   * @param instanceMsgMap map of message-id to message
-   * @param instanceCurStateMap map of resource-id to current-state
-   * @return participant
-   */
-  static Participant createParticipant(ParticipantId participantId, InstanceConfig instanceConfig,
-      UserConfig userConfig, LiveInstance liveInstance, Map<String, Message> instanceMsgMap,
-      Map<String, CurrentState> instanceCurStateMap) {
-
-    String hostName = instanceConfig.getHostName();
-
-    int port = -1;
-    try {
-      port = Integer.parseInt(instanceConfig.getPort());
-    } catch (IllegalArgumentException e) {
-      // keep as -1
-    }
-    if (port < 0 || port > 65535) {
-      port = -1;
-    }
-    boolean isEnabled = instanceConfig.getInstanceEnabled();
-
-    List<String> disabledPartitions = instanceConfig.getDisabledPartitions();
-    Set<PartitionId> disabledPartitionIdSet = Collections.emptySet();
-    if (disabledPartitions != null) {
-      disabledPartitionIdSet = new HashSet<PartitionId>();
-      for (String partitionId : disabledPartitions) {
-        disabledPartitionIdSet.add(PartitionId.from(PartitionId.extractResourceId(partitionId),
-            PartitionId.stripResourceId(partitionId)));
-      }
-    }
-
-    Set<String> tags = new HashSet<String>(instanceConfig.getTags());
-
-    RunningInstance runningInstance = null;
-    if (liveInstance != null) {
-      runningInstance =
-          new RunningInstance(liveInstance.getSessionId(), liveInstance.getHelixVersion(),
-              liveInstance.getProcessId());
-    }
-
-    Map<MessageId, Message> msgMap = new HashMap<MessageId, Message>();
-    if (instanceMsgMap != null) {
-      for (String msgId : instanceMsgMap.keySet()) {
-        Message message = instanceMsgMap.get(msgId);
-        msgMap.put(MessageId.from(msgId), message);
-      }
-    }
-
-    Map<ResourceId, CurrentState> curStateMap = new HashMap<ResourceId, CurrentState>();
-    if (instanceCurStateMap != null) {
-
-      for (String resourceName : instanceCurStateMap.keySet()) {
-        curStateMap.put(ResourceId.from(resourceName), instanceCurStateMap.get(resourceName));
-      }
-    }
-
-    return new Participant(participantId, hostName, port, isEnabled, disabledPartitionIdSet, tags,
-        runningInstance, curStateMap, msgMap, userConfig);
-  }
-
-  /**
-   * read participant related data
-   * @param participantId
-   * @return participant
-   */
-  public Participant readParticipant(ParticipantId participantId) {
-    // read physical model
-    String participantName = participantId.stringify();
-    InstanceConfig instanceConfig = _accessor.getProperty(_keyBuilder.instance(participantName));
-    UserConfig userConfig = UserConfig.from(instanceConfig);
-    LiveInstance liveInstance = _accessor.getProperty(_keyBuilder.liveInstance(participantName));
-
-    Map<String, Message> instanceMsgMap = Collections.emptyMap();
-    Map<String, CurrentState> instanceCurStateMap = Collections.emptyMap();
-    if (liveInstance != null) {
-      SessionId sessionId = liveInstance.getSessionId();
-
-      instanceMsgMap = _accessor.getChildValuesMap(_keyBuilder.messages(participantName));
-      instanceCurStateMap =
-          _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName,
-              sessionId.stringify()));
-    }
-
-    return createParticipant(participantId, instanceConfig, userConfig, liveInstance,
-        instanceMsgMap, instanceCurStateMap);
-  }
-
-  /**
-   * update resource current state of a participant
-   * @param resourceId resource id
-   * @param participantId participant id
-   * @param sessionId session id
-   * @param curStateUpdate current state change delta
-   */
-  public void updateCurrentState(ResourceId resourceId, ParticipantId participantId,
-      SessionId sessionId, CurrentState curStateUpdate) {
-    _accessor.updateProperty(
-        _keyBuilder.currentState(participantId.stringify(), sessionId.stringify(),
-            resourceId.stringify()), curStateUpdate);
-  }
-
-  /**
-   * drop resource current state of a participant
-   * @param resourceId resource id
-   * @param participantId participant id
-   * @param sessionId session id
-   */
-  public void dropCurrentState(ResourceId resourceId, ParticipantId participantId,
-      SessionId sessionId) {
-    _accessor.removeProperty(_keyBuilder.currentState(participantId.stringify(),
-        sessionId.stringify(), resourceId.stringify()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ParticipantConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ParticipantConfig.java b/helix-core/src/main/java/org/apache/helix/api/ParticipantConfig.java
deleted file mode 100644
index 5498ca3..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ParticipantConfig.java
+++ /dev/null
@@ -1,371 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-
-/*
- * 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.
- */
-
-/**
- * Configuration properties of a Helix participant
- */
-public class ParticipantConfig {
-  private final ParticipantId _id;
-  private final String _hostName;
-  private final int _port;
-  private final boolean _isEnabled;
-  private final Set<PartitionId> _disabledPartitions;
-  private final Set<String> _tags;
-  private final UserConfig _userConfig;
-
-  /**
-   * Initialize a participant configuration. Also see ParticipantConfig.Builder
-   * @param id participant id
-   * @param hostName host where participant can be reached
-   * @param port port to use to contact participant
-   * @param isEnabled true if enabled, false if disabled
-   * @param disabledPartitions set of partitions, if any to disable on this participant
-   * @param tags tags to set for the participant
-   */
-  public ParticipantConfig(ParticipantId id, String hostName, int port, boolean isEnabled,
-      Set<PartitionId> disabledPartitions, Set<String> tags, UserConfig userConfig) {
-    _id = id;
-    _hostName = hostName;
-    _port = port;
-    _isEnabled = isEnabled;
-    _disabledPartitions = ImmutableSet.copyOf(disabledPartitions);
-    _tags = ImmutableSet.copyOf(tags);
-    _userConfig = userConfig;
-  }
-
-  /**
-   * Get the host name of the participant
-   * @return host name, or null if not applicable
-   */
-  public String getHostName() {
-    return _hostName;
-  }
-
-  /**
-   * Get the port of the participant
-   * @return port number, or -1 if not applicable
-   */
-  public int getPort() {
-    return _port;
-  }
-
-  /**
-   * Get if the participant is enabled
-   * @return true if enabled or false otherwise
-   */
-  public boolean isEnabled() {
-    return _isEnabled;
-  }
-
-  /**
-   * Get disabled partition id's
-   * @return set of disabled partition id's, or empty set if none
-   */
-  public Set<PartitionId> getDisabledPartitions() {
-    return _disabledPartitions;
-  }
-
-  /**
-   * Get tags
-   * @return set of tags
-   */
-  public Set<String> getTags() {
-    return _tags;
-  }
-
-  /**
-   * Check if participant has a tag
-   * @param tag tag to check
-   * @return true if tagged, false otherwise
-   */
-  public boolean hasTag(String tag) {
-    return _tags.contains(tag);
-  }
-
-  /**
-   * Get user-specified configuration properties of this participant
-   * @return UserConfig properties
-   */
-  public UserConfig getUserConfig() {
-    return _userConfig;
-  }
-
-  /**
-   * Get the participant id
-   * @return ParticipantId
-   */
-  public ParticipantId getId() {
-    return _id;
-  }
-
-  /**
-   * Update context for a ParticipantConfig
-   */
-  public static class Delta {
-    private enum Fields {
-      HOST_NAME,
-      PORT,
-      ENABLED,
-      USER_CONFIG
-    }
-
-    private Set<Fields> _updateFields;
-    private Set<String> _removedTags;
-    private Set<PartitionId> _removedDisabledPartitions;
-    private Builder _builder;
-
-    /**
-     * Instantiate the delta for a participant config
-     * @param participantId the participant to update
-     */
-    public Delta(ParticipantId participantId) {
-      _updateFields = Sets.newHashSet();
-      _removedTags = Sets.newHashSet();
-      _removedDisabledPartitions = Sets.newHashSet();
-      _builder = new Builder(participantId);
-    }
-
-    /**
-     * Set the participant host name
-     * @param hostName reachable host when live
-     * @return Delta
-     */
-    public Delta setHostName(String hostName) {
-      _builder.hostName(hostName);
-      _updateFields.add(Fields.HOST_NAME);
-      return this;
-    }
-
-    /**
-     * Set the participant port
-     * @param port port number
-     * @return Delta
-     */
-    public Delta setPort(int port) {
-      _builder.port(port);
-      _updateFields.add(Fields.PORT);
-      return this;
-    }
-
-    /**
-     * Set whether or not the participant is enabled
-     * @param isEnabled true if enabled, false otherwise
-     * @return Delta
-     */
-    public Delta setEnabled(boolean isEnabled) {
-      _builder.enabled(isEnabled);
-      _updateFields.add(Fields.ENABLED);
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Delta
-     */
-    public Delta setUserConfig(UserConfig userConfig) {
-      _builder.userConfig(userConfig);
-      _updateFields.add(Fields.USER_CONFIG);
-      return this;
-    }
-
-    /**
-     * Add an new tag for this participant
-     * @param tag the tag to add
-     * @return Delta
-     */
-    public Delta addTag(String tag) {
-      _builder.addTag(tag);
-      return this;
-    }
-
-    /**
-     * Remove a tag for this participant
-     * @param tag the tag to remove
-     * @return Delta
-     */
-    public Delta removeTag(String tag) {
-      _removedTags.add(tag);
-      return this;
-    }
-
-    /**
-     * Add a partition to disable for this participant
-     * @param partitionId the partition to disable
-     * @return Delta
-     */
-    public Delta addDisabledPartition(PartitionId partitionId) {
-      _builder.addDisabledPartition(partitionId);
-      return this;
-    }
-
-    /**
-     * Remove a partition from the disabled set for this participant
-     * @param partitionId the partition to enable
-     * @return Delta
-     */
-    public Delta removeDisabledPartition(PartitionId partitionId) {
-      _removedDisabledPartitions.add(partitionId);
-      return this;
-    }
-
-    /**
-     * Create a ParticipantConfig that is the combination of an existing ParticipantConfig and this
-     * delta
-     * @param orig the original ParticipantConfig
-     * @return updated ParticipantConfig
-     */
-    public ParticipantConfig mergeInto(ParticipantConfig orig) {
-      ParticipantConfig deltaConfig = _builder.build();
-      Builder builder =
-          new Builder(orig.getId()).hostName(orig.getHostName()).port(orig.getPort())
-              .userConfig(orig.getUserConfig());
-      for (Fields field : _updateFields) {
-        switch (field) {
-        case HOST_NAME:
-          builder.hostName(deltaConfig.getHostName());
-          break;
-        case PORT:
-          builder.port(deltaConfig.getPort());
-          break;
-        case ENABLED:
-          builder.enabled(deltaConfig.isEnabled());
-          break;
-        case USER_CONFIG:
-          builder.userConfig(deltaConfig.getUserConfig());
-          break;
-        }
-      }
-      Set<String> tags = Sets.newHashSet(orig.getTags());
-      tags.addAll(deltaConfig.getTags());
-      tags.removeAll(_removedTags);
-      for (String tag : tags) {
-        builder.addTag(tag);
-      }
-      Set<PartitionId> disabledPartitions = Sets.newHashSet(orig.getDisabledPartitions());
-      disabledPartitions.addAll(deltaConfig.getDisabledPartitions());
-      for (PartitionId partitionId : disabledPartitions) {
-        builder.addDisabledPartition(partitionId);
-      }
-      return builder.build();
-    }
-  }
-
-  /**
-   * Assemble a participant
-   */
-  public static class Builder {
-    private final ParticipantId _id;
-    private String _hostName;
-    private int _port;
-    private boolean _isEnabled;
-    private final Set<PartitionId> _disabledPartitions;
-    private final Set<String> _tags;
-    private UserConfig _userConfig;
-
-    /**
-     * Build a participant with a given id
-     * @param id participant id
-     */
-    public Builder(ParticipantId id) {
-      _id = id;
-      _disabledPartitions = new HashSet<PartitionId>();
-      _tags = new HashSet<String>();
-      _isEnabled = true;
-      _userConfig = new UserConfig(Scope.participant(id));
-    }
-
-    /**
-     * Set the participant host name
-     * @param hostName reachable host when live
-     * @return Builder
-     */
-    public Builder hostName(String hostName) {
-      _hostName = hostName;
-      return this;
-    }
-
-    /**
-     * Set the participant port
-     * @param port port number
-     * @return Builder
-     */
-    public Builder port(int port) {
-      _port = port;
-      return this;
-    }
-
-    /**
-     * Set whether or not the participant is enabled
-     * @param isEnabled true if enabled, false otherwise
-     * @return Builder
-     */
-    public Builder enabled(boolean isEnabled) {
-      _isEnabled = isEnabled;
-      return this;
-    }
-
-    /**
-     * Add a partition to disable for this participant
-     * @param partitionId the partition to disable
-     * @return Builder
-     */
-    public Builder addDisabledPartition(PartitionId partitionId) {
-      _disabledPartitions.add(partitionId);
-      return this;
-    }
-
-    /**
-     * Add an arbitrary tag for this participant
-     * @param tag the tag to add
-     * @return Builder
-     */
-    public Builder addTag(String tag) {
-      _tags.add(tag);
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Builder
-     */
-    public Builder userConfig(UserConfig userConfig) {
-      _userConfig = userConfig;
-      return this;
-    }
-
-    /**
-     * Assemble the participant
-     * @return instantiated Participant
-     */
-    public ParticipantConfig build() {
-      return new ParticipantConfig(_id, _hostName, _port, _isEnabled, _disabledPartitions, _tags,
-          _userConfig);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java b/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
deleted file mode 100644
index 59aa93b..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 ParticipantId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Instantiate for a participant with a string name
-   * @param id string participant id
-   */
-  @JsonCreator
-  public ParticipantId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete participant id
-   * @param participantId string participant identifier
-   * @return ParticipantId
-   */
-  public static ParticipantId from(String participantId) {
-    if (participantId == null) {
-      return null;
-    }
-    return new ParticipantId(participantId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Partition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Partition.java b/helix-core/src/main/java/org/apache/helix/api/Partition.java
index 3853f61..b295d72 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Partition.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Partition.java
@@ -1,5 +1,6 @@
 package org.apache.helix.api;
 
+import org.apache.helix.api.id.PartitionId;
 import org.codehaus.jackson.annotate.JsonCreator;
 import org.codehaus.jackson.annotate.JsonProperty;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/PartitionId.java b/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
deleted file mode 100644
index 8f852c2..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
+++ /dev/null
@@ -1,112 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 PartitionId extends Id {
-  @JsonProperty("resourceId")
-  private final ResourceId _resourceId;
-  @JsonProperty("partitionName")
-  private final String _partitionName;
-
-  /**
-   * Instantiate for a resource and suffix
-   * @param resourceId resource that the partition belongs to
-   * @param partitionName name of the partition relative to the resource
-   */
-  @JsonCreator
-  public PartitionId(@JsonProperty("resourceId") ResourceId resourceId,
-      @JsonProperty("partitionName") String partitionName) {
-    _resourceId = resourceId;
-    _partitionName = partitionName;
-  }
-
-  /**
-   * Get the id of the resource containing this partition
-   * @return ResourceId
-   */
-  public ResourceId getResourceId() {
-    return _resourceId;
-  }
-
-  @Override
-  public String stringify() {
-    // check in case the partition name is instantiated incorrectly
-    if (_resourceId.stringify().equals(_partitionName)) {
-      return _partitionName;
-    }
-    return String.format("%s_%s", _resourceId, _partitionName);
-  }
-
-  /**
-   * @param partitionName
-   * @return
-   */
-  public static String stripResourceId(String partitionName) {
-    if (partitionName == null || !partitionName.contains("_")) {
-      return partitionName;
-    }
-    return partitionName.substring(partitionName.lastIndexOf("_") + 1);
-  }
-
-  /**
-   * @param partitionName
-   * @return
-   */
-  public static ResourceId extractResourceId(String partitionName) {
-    if (partitionName == null || !partitionName.contains("_")) {
-      return ResourceId.from(partitionName);
-    }
-    return ResourceId.from(partitionName.substring(0, partitionName.lastIndexOf("_")));
-  }
-
-  /**
-   * Get a concrete partition id
-   * @param partitionId string partition identifier
-   * @return PartitionId
-   */
-  public static PartitionId from(String partitionId) {
-    if (partitionId == null) {
-      return null;
-    }
-    return new PartitionId(extractResourceId(partitionId), stripResourceId(partitionId));
-  }
-
-  /**
-   * Same as {@link PartitionId#from(String)}.
-   * @param partitionId string partition identifier
-   * @return PartitionId
-   */
-  public static PartitionId valueOf(String partitionId) {
-    return from(partitionId);
-  }
-
-  /**
-   * Get a concrete partition id
-   * @param resourceId resource identifier
-   * @param partitionSuffix partition identifier relative to a resource
-   * @return PartitionId
-   */
-  public static PartitionId from(ResourceId resourceId, String partitionSuffix) {
-    return new PartitionId(resourceId, partitionSuffix);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ProcId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ProcId.java b/helix-core/src/main/java/org/apache/helix/api/ProcId.java
deleted file mode 100644
index 4668f57..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ProcId.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 ProcId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a process id
-   * @param id string representation of a process id
-   */
-  @JsonCreator
-  public ProcId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete process id
-   * @param processId string process identifier (e.g. pid@host)
-   * @return ProcId
-   */
-  public static ProcId from(String processId) {
-    if (processId == null) {
-      return null;
-    }
-    return new ProcId(processId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Resource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Resource.java b/helix-core/src/main/java/org/apache/helix/api/Resource.java
index 2c3b7ca..e9d07bd 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Resource.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Resource.java
@@ -24,7 +24,13 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.api.ResourceConfig.ResourceType;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.config.SchedulerTaskConfig;
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.config.ResourceConfig.ResourceType;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
 import org.apache.helix.controller.rebalancer.context.RebalancerContext;
 import org.apache.helix.model.ExternalView;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/ResourceAccessor.java
deleted file mode 100644
index 5adec4e..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ResourceAccessor.java
+++ /dev/null
@@ -1,233 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.HelixConstants.StateModelToken;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.api.ResourceConfig.ResourceType;
-import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.ResourceConfiguration;
-import org.apache.log4j.Logger;
-
-public class ResourceAccessor {
-  private static final Logger LOG = Logger.getLogger(ResourceAccessor.class);
-  private final HelixDataAccessor _accessor;
-  private final PropertyKey.Builder _keyBuilder;
-
-  public ResourceAccessor(HelixDataAccessor accessor) {
-    _accessor = accessor;
-    _keyBuilder = accessor.keyBuilder();
-  }
-
-  /**
-   * Read a single snapshot of a resource
-   * @param resourceId the resource id to read
-   * @return Resource
-   */
-  public Resource readResource(ResourceId resourceId) {
-    ResourceConfiguration config =
-        _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
-    IdealState idealState = _accessor.getProperty(_keyBuilder.idealState(resourceId.stringify()));
-    ExternalView externalView =
-        _accessor.getProperty(_keyBuilder.externalView(resourceId.stringify()));
-    ResourceAssignment resourceAssignment =
-        _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify()));
-    return createResource(resourceId, config, idealState, externalView, resourceAssignment);
-  }
-
-  /**
-   * Update a resource configuration
-   * @param resourceId the resource id to update
-   * @param resourceDelta changes to the resource
-   * @return ResourceConfig, or null if the resource is not persisted
-   */
-  public ResourceConfig updateResource(ResourceId resourceId, ResourceConfig.Delta resourceDelta) {
-    Resource resource = readResource(resourceId);
-    if (resource == null) {
-      LOG.error("Resource " + resourceId + " does not exist, cannot be updated");
-      return null;
-    }
-    ResourceConfig config = resourceDelta.mergeInto(resource.getConfig());
-    // TODO: persist this
-    return config;
-  }
-
-  /**
-   * save resource assignment
-   * @param resourceId
-   * @param resourceAssignment
-   */
-  public void setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) {
-    _accessor.setProperty(_keyBuilder.resourceAssignment(resourceId.stringify()),
-        resourceAssignment);
-  }
-
-  /**
-   * save resource assignment
-   * @param resourceId
-   * @return resource assignment or null
-   */
-  public ResourceAssignment getResourceAssignment(ResourceId resourceId) {
-    return _accessor.getProperty(_keyBuilder.resourceAssignment(resourceId.stringify()));
-  }
-
-  /**
-   * Set a resource configuration, which may include user-defined configuration, as well as
-   * rebalancer configuration
-   * @param resourceId
-   * @param configuration
-   */
-  public void setConfiguration(ResourceId resourceId, ResourceConfiguration configuration) {
-    _accessor.setProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration);
-    // also set an ideal state if the resource supports it
-    RebalancerConfig rebalancerConfig = new RebalancerConfig(configuration);
-    IdealState idealState =
-        rebalancerConfigToIdealState(rebalancerConfig, configuration.getBucketSize(),
-            configuration.getBatchMessageMode());
-    if (idealState != null) {
-      _accessor.setProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
-    }
-  }
-
-  /**
-   * Get a resource configuration, which may include user-defined configuration, as well as
-   * rebalancer configuration
-   * @param resourceId
-   * @return configuration
-   */
-  public void getConfiguration(ResourceId resourceId) {
-    _accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
-  }
-
-  /**
-   * set external view of a resource
-   * @param resourceId
-   * @param extView
-   */
-  public void setExternalView(ResourceId resourceId, ExternalView extView) {
-    _accessor.setProperty(_keyBuilder.externalView(resourceId.stringify()), extView);
-  }
-
-  /**
-   * drop external view of a resource
-   * @param resourceId
-   */
-  public void dropExternalView(ResourceId resourceId) {
-    _accessor.removeProperty(_keyBuilder.externalView(resourceId.stringify()));
-  }
-
-  /**
-   * Get an ideal state from a rebalancer config if the resource is partitioned
-   * @param config RebalancerConfig instance
-   * @param bucketSize bucket size to use
-   * @param batchMessageMode true if batch messaging allowed, false otherwise
-   * @return IdealState, or null
-   */
-  static IdealState rebalancerConfigToIdealState(RebalancerConfig config, int bucketSize,
-      boolean batchMessageMode) {
-    PartitionedRebalancerContext partitionedContext =
-        config.getRebalancerContext(PartitionedRebalancerContext.class);
-    if (partitionedContext != null) {
-      IdealState idealState = new IdealState(partitionedContext.getResourceId());
-      idealState.setRebalanceMode(partitionedContext.getRebalanceMode());
-      idealState.setRebalancerRef(partitionedContext.getRebalancerRef());
-      String replicas = null;
-      if (partitionedContext.anyLiveParticipant()) {
-        replicas = StateModelToken.ANY_LIVEINSTANCE.toString();
-      } else {
-        replicas = Integer.toString(partitionedContext.getReplicaCount());
-      }
-      idealState.setReplicas(replicas);
-      idealState.setNumPartitions(partitionedContext.getPartitionSet().size());
-      idealState.setInstanceGroupTag(partitionedContext.getParticipantGroupTag());
-      idealState.setMaxPartitionsPerInstance(partitionedContext.getMaxPartitionsPerParticipant());
-      idealState.setStateModelDefId(partitionedContext.getStateModelDefId());
-      idealState.setStateModelFactoryId(partitionedContext.getStateModelFactoryId());
-      idealState.setBucketSize(bucketSize);
-      idealState.setBatchMessageMode(batchMessageMode);
-      if (partitionedContext.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
-        SemiAutoRebalancerContext semiAutoContext =
-            config.getRebalancerContext(SemiAutoRebalancerContext.class);
-        for (PartitionId partitionId : semiAutoContext.getPartitionSet()) {
-          idealState.setPreferenceList(partitionId, semiAutoContext.getPreferenceList(partitionId));
-        }
-      } else if (partitionedContext.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
-        CustomRebalancerContext customContext =
-            config.getRebalancerContext(CustomRebalancerContext.class);
-        for (PartitionId partitionId : customContext.getPartitionSet()) {
-          idealState.setParticipantStateMap(partitionId,
-              customContext.getPreferenceMap(partitionId));
-        }
-      }
-      return idealState;
-    }
-    return null;
-  }
-
-  /**
-   * Create a resource snapshot instance from the physical model
-   * @param resourceId the resource id
-   * @param resourceConfiguration physical resource configuration
-   * @param idealState ideal state of the resource
-   * @param externalView external view of the resource
-   * @param resourceAssignment current resource assignment
-   * @return Resource
-   */
-  static Resource createResource(ResourceId resourceId,
-      ResourceConfiguration resourceConfiguration, IdealState idealState,
-      ExternalView externalView, ResourceAssignment resourceAssignment) {
-    UserConfig userConfig;
-    ResourceType type = ResourceType.DATA;
-    if (resourceConfiguration != null) {
-      userConfig = UserConfig.from(resourceConfiguration);
-      type = resourceConfiguration.getType();
-    } else {
-      userConfig = new UserConfig(Scope.resource(resourceId));
-    }
-    int bucketSize = 0;
-    boolean batchMessageMode = false;
-    RebalancerContext rebalancerContext;
-    if (idealState != null) {
-      rebalancerContext = PartitionedRebalancerContext.from(idealState);
-      bucketSize = idealState.getBucketSize();
-      batchMessageMode = idealState.getBatchMessageMode();
-    } else {
-      if (resourceConfiguration != null) {
-        bucketSize = resourceConfiguration.getBucketSize();
-        batchMessageMode = resourceConfiguration.getBatchMessageMode();
-        RebalancerConfig rebalancerConfig = new RebalancerConfig(resourceConfiguration);
-        rebalancerContext = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
-      } else {
-        rebalancerContext = new PartitionedRebalancerContext(RebalanceMode.NONE);
-      }
-    }
-    return new Resource(resourceId, type, idealState, resourceAssignment, externalView,
-        rebalancerContext, userConfig, bucketSize, batchMessageMode);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
deleted file mode 100644
index dc3dc1d..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
+++ /dev/null
@@ -1,369 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-
-import com.google.common.collect.Sets;
-
-/*
- * 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.
- */
-
-/**
- * Full configuration of a Helix resource. Typically used to add or modify resources on a cluster
- */
-public class ResourceConfig {
-  /**
-   * Type of a resource. A resource is any entity that can be managed by Helix.
-   */
-  public enum ResourceType {
-    /**
-     * A resource that is persistent, and potentially partitioned and replicated.
-     */
-    DATA
-  }
-
-  private final ResourceId _id;
-  private final RebalancerConfig _rebalancerConfig;
-  private final SchedulerTaskConfig _schedulerTaskConfig;
-  private final UserConfig _userConfig;
-  private final int _bucketSize;
-  private final boolean _batchMessageMode;
-  private final ResourceType _resourceType;
-
-  /**
-   * Instantiate a configuration. Consider using ResourceConfig.Builder
-   * @param id resource id
-   * @param partitionMap map of partition identifiers to partition objects
-   * @param schedulerTaskConfig configuration for scheduler tasks associated with the resource
-   * @param rebalancerConfig configuration for rebalancing the resource
-   * @param userConfig user-defined resource properties
-   * @param bucketSize bucket size for this resource
-   * @param batchMessageMode whether or not batch messaging is allowed
-   */
-  public ResourceConfig(ResourceId id, ResourceType resourceType,
-      SchedulerTaskConfig schedulerTaskConfig, RebalancerConfig rebalancerConfig,
-      UserConfig userConfig, int bucketSize, boolean batchMessageMode) {
-    _id = id;
-    _resourceType = resourceType;
-    _schedulerTaskConfig = schedulerTaskConfig;
-    _rebalancerConfig = rebalancerConfig;
-    _userConfig = userConfig;
-    _bucketSize = bucketSize;
-    _batchMessageMode = batchMessageMode;
-  }
-
-  /**
-   * Get the subunits of the resource
-   * @return map of subunit id to subunit or empty map if none
-   */
-  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap() {
-    return _rebalancerConfig.getRebalancerContext(RebalancerContext.class).getSubUnitMap();
-  }
-
-  /**
-   * Get a subunit that the resource contains
-   * @param subUnitId the subunit id to look up
-   * @return Partition or null if none is present with the given id
-   */
-  public Partition getSubUnit(PartitionId subUnitId) {
-    return getSubUnitMap().get(subUnitId);
-  }
-
-  /**
-   * Get the set of subunit ids that the resource contains
-   * @return subunit id set, or empty if none
-   */
-  public Set<? extends PartitionId> getSubUnitSet() {
-    return getSubUnitMap().keySet();
-  }
-
-  /**
-   * Get the resource properties configuring rebalancing
-   * @return RebalancerConfig properties
-   */
-  public RebalancerConfig getRebalancerConfig() {
-    return _rebalancerConfig;
-  }
-
-  /**
-   * Get the resource id
-   * @return ResourceId
-   */
-  public ResourceId getId() {
-    return _id;
-  }
-
-  /**
-   * Get the resource type
-   * @return ResourceType
-   */
-  public ResourceType getType() {
-    return _resourceType;
-  }
-
-  /**
-   * Get the properties configuring scheduler tasks
-   * @return SchedulerTaskConfig properties
-   */
-  public SchedulerTaskConfig getSchedulerTaskConfig() {
-    return _schedulerTaskConfig;
-  }
-
-  /**
-   * Get user-specified configuration properties of this resource
-   * @return UserConfig properties
-   */
-  public UserConfig getUserConfig() {
-    return _userConfig;
-  }
-
-  /**
-   * Get the bucket size for this resource
-   * @return bucket size
-   */
-  public int getBucketSize() {
-    return _bucketSize;
-  }
-
-  /**
-   * Get the batch message mode
-   * @return true if enabled, false if disabled
-   */
-  public boolean getBatchMessageMode() {
-    return _batchMessageMode;
-  }
-
-  @Override
-  public String toString() {
-    return getSubUnitMap().toString();
-  }
-
-  /**
-   * Update context for a ResourceConfig
-   */
-  public static class Delta {
-    private enum Fields {
-      TYPE,
-      REBALANCER_CONTEXT,
-      USER_CONFIG,
-      BUCKET_SIZE,
-      BATCH_MESSAGE_MODE
-    }
-
-    private Set<Fields> _updateFields;
-    private Builder _builder;
-
-    /**
-     * Instantiate the delta for a resource config
-     * @param resourceId the resource to update
-     */
-    public Delta(ResourceId resourceId) {
-      _builder = new Builder(resourceId);
-      _updateFields = Sets.newHashSet();
-    }
-
-    /**
-     * Set the type of this resource
-     * @param type ResourceType
-     * @return Delta
-     */
-    public Delta setType(ResourceType type) {
-      _builder.type(type);
-      _updateFields.add(Fields.TYPE);
-      return this;
-    }
-
-    /**
-     * Set the rebalancer configuration
-     * @param context properties of interest for rebalancing
-     * @return Delta
-     */
-    public Delta setRebalancerContext(RebalancerContext context) {
-      _builder.rebalancerContext(context);
-      _updateFields.add(Fields.REBALANCER_CONTEXT);
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Delta
-     */
-    public Delta setUserConfig(UserConfig userConfig) {
-      _builder.userConfig(userConfig);
-      _updateFields.add(Fields.USER_CONFIG);
-      return this;
-    }
-
-    /**
-     * Set the bucket size
-     * @param bucketSize the size to use
-     * @return Delta
-     */
-    public Delta setBucketSize(int bucketSize) {
-      _builder.bucketSize(bucketSize);
-      _updateFields.add(Fields.BUCKET_SIZE);
-      return this;
-    }
-
-    /**
-     * Set the batch message mode
-     * @param batchMessageMode true to enable, false to disable
-     * @return Delta
-     */
-    public Delta setBatchMessageMode(boolean batchMessageMode) {
-      _builder.batchMessageMode(batchMessageMode);
-      _updateFields.add(Fields.BATCH_MESSAGE_MODE);
-      return this;
-    }
-
-    /**
-     * Create a ResourceConfig that is the combination of an existing ResourceConfig and this delta
-     * @param orig the original ResourceConfig
-     * @return updated ResourceConfig
-     */
-    public ResourceConfig mergeInto(ResourceConfig orig) {
-      ResourceConfig deltaConfig = _builder.build();
-      Builder builder =
-          new Builder(orig.getId())
-              .type(orig.getType())
-              .rebalancerContext(
-                  orig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class))
-              .schedulerTaskConfig(orig.getSchedulerTaskConfig()).userConfig(orig.getUserConfig())
-              .bucketSize(orig.getBucketSize()).batchMessageMode(orig.getBatchMessageMode());
-      for (Fields field : _updateFields) {
-        switch (field) {
-        case TYPE:
-          builder.type(deltaConfig.getType());
-          break;
-        case REBALANCER_CONTEXT:
-          builder.rebalancerContext(deltaConfig.getRebalancerConfig().getRebalancerContext(
-              RebalancerContext.class));
-          break;
-        case USER_CONFIG:
-          builder.userConfig(deltaConfig.getUserConfig());
-          break;
-        case BUCKET_SIZE:
-          builder.bucketSize(deltaConfig.getBucketSize());
-          break;
-        case BATCH_MESSAGE_MODE:
-          builder.batchMessageMode(deltaConfig.getBatchMessageMode());
-          break;
-        }
-      }
-      return builder.build();
-    }
-  }
-
-  /**
-   * Assembles a ResourceConfig
-   */
-  public static class Builder {
-    private final ResourceId _id;
-    private ResourceType _type;
-    private RebalancerConfig _rebalancerConfig;
-    private SchedulerTaskConfig _schedulerTaskConfig;
-    private UserConfig _userConfig;
-    private int _bucketSize;
-    private boolean _batchMessageMode;
-
-    /**
-     * Build a Resource with an id
-     * @param id resource id
-     */
-    public Builder(ResourceId id) {
-      _id = id;
-      _type = ResourceType.DATA;
-      _bucketSize = 0;
-      _batchMessageMode = false;
-      _userConfig = new UserConfig(Scope.resource(id));
-    }
-
-    /**
-     * Set the type of this resource
-     * @param type ResourceType
-     * @return Builder
-     */
-    public Builder type(ResourceType type) {
-      _type = type;
-      return this;
-    }
-
-    /**
-     * Set the rebalancer configuration
-     * @param rebalancerContext properties of interest for rebalancing
-     * @return Builder
-     */
-    public Builder rebalancerContext(RebalancerContext rebalancerContext) {
-      _rebalancerConfig = new RebalancerConfig(rebalancerContext);
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Builder
-     */
-    public Builder userConfig(UserConfig userConfig) {
-      _userConfig = userConfig;
-      return this;
-    }
-
-    /**
-     * @param schedulerTaskConfig
-     * @return
-     */
-    public Builder schedulerTaskConfig(SchedulerTaskConfig schedulerTaskConfig) {
-      _schedulerTaskConfig = schedulerTaskConfig;
-      return this;
-    }
-
-    /**
-     * Set the bucket size
-     * @param bucketSize the size to use
-     * @return Builder
-     */
-    public Builder bucketSize(int bucketSize) {
-      _bucketSize = bucketSize;
-      return this;
-    }
-
-    /**
-     * Set the batch message mode
-     * @param batchMessageMode true to enable, false to disable
-     * @return Builder
-     */
-    public Builder batchMessageMode(boolean batchMessageMode) {
-      _batchMessageMode = batchMessageMode;
-      return this;
-    }
-
-    /**
-     * Create a Resource object
-     * @return instantiated Resource
-     */
-    public ResourceConfig build() {
-      return new ResourceConfig(_id, _type, _schedulerTaskConfig, _rebalancerConfig, _userConfig,
-          _bucketSize, _batchMessageMode);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ResourceId.java b/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
deleted file mode 100644
index dfffe5e..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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.
- */
-
-/**
- * Identifies a resource
- */
-public class ResourceId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Create a resource id
-   * @param id string representation of a resource id
-   */
-  @JsonCreator
-  public ResourceId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  /**
-   * Get a concrete resource id for a string name
-   * @param resourceId string resource identifier
-   * @return ResourceId
-   */
-  public static ResourceId from(String resourceId) {
-    if (resourceId == null) {
-      return null;
-    }
-    return new ResourceId(resourceId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java b/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java
index 4effd24..4e5aedb 100644
--- a/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java
+++ b/helix-core/src/main/java/org/apache/helix/api/RunningInstance.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+import org.apache.helix.api.id.ProcId;
+import org.apache.helix.api.id.SessionId;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/SchedulerTaskConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SchedulerTaskConfig.java b/helix-core/src/main/java/org/apache/helix/api/SchedulerTaskConfig.java
deleted file mode 100644
index e7d0779..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/SchedulerTaskConfig.java
+++ /dev/null
@@ -1,68 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.model.Message;
-
-import com.google.common.collect.ImmutableMap;
-
-public class SchedulerTaskConfig {
-  // TODO refactor using Transition logical model
-  private final Map<String, Integer> _transitionTimeoutMap;
-
-  private final Map<PartitionId, Message> _innerMessageMap;
-
-  public SchedulerTaskConfig(Map<String, Integer> transitionTimeoutMap,
-      Map<PartitionId, Message> innerMsgMap) {
-    _transitionTimeoutMap = ImmutableMap.copyOf(transitionTimeoutMap);
-    _innerMessageMap = ImmutableMap.copyOf(innerMsgMap);
-  }
-
-  /**
-   * Get inner message for a partition
-   * @param partitionId
-   * @return inner message
-   */
-  public Message getInnerMessage(PartitionId partitionId) {
-    return _innerMessageMap.get(partitionId);
-  }
-
-  /**
-   * Get timeout for a transition
-   * @param transition
-   * @return timeout or -1 if not available
-   */
-  public int getTransitionTimeout(String transition) {
-    Integer timeout = _transitionTimeoutMap.get(transition);
-    if (timeout == null) {
-      return -1;
-    }
-
-    return timeout;
-  }
-
-  /**
-   * Get timeout for an inner message
-   * @param transition
-   * @param partitionId
-   * @return timeout or -1 if not available
-   */
-  public int getTimeout(String transition, PartitionId partitionId) {
-    Integer timeout = getTransitionTimeout(transition);
-    if (timeout == null) {
-      Message innerMessage = getInnerMessage(partitionId);
-      timeout = innerMessage.getTimeout();
-    }
-
-    return timeout;
-  }
-
-  /**
-   * Get partition-id set
-   * @return partition-id set
-   */
-  public Set<PartitionId> getPartitionSet() {
-    return _innerMessageMap.keySet();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Scope.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Scope.java b/helix-core/src/main/java/org/apache/helix/api/Scope.java
index 4bff194..4d4eead 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Scope.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Scope.java
@@ -1,5 +1,11 @@
 package org.apache.helix.api;
 
+import org.apache.helix.api.id.ClusterId;
+import org.apache.helix.api.id.Id;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/SessionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SessionId.java b/helix-core/src/main/java/org/apache/helix/api/SessionId.java
deleted file mode 100644
index df36193..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/SessionId.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 SessionId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a session id
-   * @param id string representing a session id
-   */
-  @JsonCreator
-  public SessionId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete session id
-   * @param sessionId string session identifier
-   * @return SessionId
-   */
-  public static SessionId from(String sessionId) {
-    if (sessionId == null) {
-      return null;
-    }
-    return new SessionId(sessionId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Spectator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Spectator.java b/helix-core/src/main/java/org/apache/helix/api/Spectator.java
index e25601b..993de2f 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Spectator.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Spectator.java
@@ -1,5 +1,7 @@
 package org.apache.helix.api;
 
+import org.apache.helix.api.id.SpectatorId;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java b/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
deleted file mode 100644
index 851eb0d..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 SpectatorId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a spectator id
-   * @param id string representing a spectator id
-   */
-  @JsonCreator
-  public SpectatorId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Create a spectator id from a string
-   * @param spectatorId string representing a spectator id
-   * @return SpectatorId
-   */
-  public static SpectatorId from(String spectatorId) {
-    return new SpectatorId(spectatorId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java b/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
deleted file mode 100644
index 50ea020..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
+++ /dev/null
@@ -1,66 +0,0 @@
-package org.apache.helix.api;
-
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 StateModelDefId extends Id {
-  public static final StateModelDefId SchedulerTaskQueue = StateModelDefId
-      .from(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE);
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a state model definition id
-   * @param id string representing a state model definition id
-   */
-  @JsonCreator
-  public StateModelDefId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Check if the underlying state model definition id is equal if case is ignored
-   * @param that the StateModelDefId to compare
-   * @return true if equal ignoring case, false otherwise
-   */
-  public boolean equalsIgnoreCase(StateModelDefId that) {
-    return _id.equalsIgnoreCase(that._id);
-  }
-
-  /**
-   * Get a concrete state model definition id
-   * @param stateModelDefId string state model identifier
-   * @return StateModelDefId
-   */
-  public static StateModelDefId from(String stateModelDefId) {
-    if (stateModelDefId == null) {
-      return null;
-    }
-    return new StateModelDefId(stateModelDefId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/StateModelDefinitionAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/StateModelDefinitionAccessor.java b/helix-core/src/main/java/org/apache/helix/api/StateModelDefinitionAccessor.java
deleted file mode 100644
index 60b6210..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/StateModelDefinitionAccessor.java
+++ /dev/null
@@ -1,69 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.HashMap;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.model.StateModelDefinition;
-
-import com.google.common.collect.ImmutableMap;
-
-public class StateModelDefinitionAccessor {
-  private final HelixDataAccessor _accessor;
-  private final PropertyKey.Builder _keyBuilder;
-
-  /**
-   * @param accessor
-   */
-  public StateModelDefinitionAccessor(HelixDataAccessor accessor) {
-    _accessor = accessor;
-    _keyBuilder = accessor.keyBuilder();
-  }
-
-  /**
-   * Get all of the state model definitions available to the cluster
-   * @return map of state model ids to state model definition objects
-   */
-  public Map<StateModelDefId, StateModelDefinition> readStateModelDefinitions() {
-    Map<String, StateModelDefinition> stateModelDefs =
-        _accessor.getChildValuesMap(_keyBuilder.stateModelDefs());
-    Map<StateModelDefId, StateModelDefinition> stateModelDefMap =
-        new HashMap<StateModelDefId, StateModelDefinition>();
-
-    for (String stateModelDefName : stateModelDefs.keySet()) {
-      stateModelDefMap.put(StateModelDefId.from(stateModelDefName),
-          stateModelDefs.get(stateModelDefName));
-    }
-
-    return ImmutableMap.copyOf(stateModelDefMap);
-  }
-
-  /**
-   * Set a state model definition. Adds the state model definition if it does not exist
-   * @param stateModelDef fully initialized state model definition
-   * @return true if the model is persisted, false otherwise
-   */
-  public boolean setStateModelDefinition(StateModelDefinition stateModelDef) {
-    return _accessor.setProperty(_keyBuilder.stateModelDef(stateModelDef.getId()), stateModelDef);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java b/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
deleted file mode 100644
index 422b4e9..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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.
- */
-
-/**
- * Id representing a state model factory
- */
-public class StateModelFactoryId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a state model factory id
-   * @param id string representing a state model factory
-   */
-  @JsonCreator
-  public StateModelFactoryId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete state model factory id
-   * @param stateModelFactoryId the string version of the id
-   * @return StateModelFactoryId
-   */
-  public static StateModelFactoryId from(String stateModelFactoryId) {
-    if (stateModelFactoryId == null) {
-      return null;
-    }
-    return new StateModelFactoryId(stateModelFactoryId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/UserConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/UserConfig.java b/helix-core/src/main/java/org/apache/helix/api/UserConfig.java
deleted file mode 100644
index 181c071..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/UserConfig.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package org.apache.helix.api;
-
-import org.apache.helix.HelixProperty;
-
-/*
- * 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.
- */
-
-/**
- * Generic user-defined configuration of Helix components
- */
-public class UserConfig extends NamespacedConfig {
-  /**
-   * Instantiate a UserConfig. It is intended for use only by entities that can be identified
-   * @param scope scope of the configuration, e.g. cluster, resource, partition, participant, etc
-   */
-  public UserConfig(Scope<?> scope) {
-    super(scope, UserConfig.class.getSimpleName());
-  }
-
-  /**
-   * Instantiate a UserConfig from an existing HelixProperty
-   * @param property property wrapping a configuration
-   */
-  private UserConfig(HelixProperty property) {
-    super(property, UserConfig.class.getSimpleName());
-  }
-
-  /**
-   * Get a UserConfig that filters out the user-specific configurations in a property
-   * @param property the property to extract from
-   * @return UserConfig
-   */
-  public static UserConfig from(HelixProperty property) {
-    return new UserConfig(property);
-  }
-}


[6/6] git commit: [HELIX-238] Refactor, add update to accessors, test update logic

Posted by ka...@apache.org.
[HELIX-238] Refactor, add update to accessors, test update logic


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

Branch: refs/heads/helix-logical-model
Commit: c070a76514bda1f6269f1db1197289f7b588ced7
Parents: 41b6e77
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Thu Sep 26 15:02:19 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Thu Sep 26 16:03:49 2013 -0700

----------------------------------------------------------------------
 .../resources/SchedulerTasksResource.java       |   4 +-
 .../java/org/apache/helix/HelixProperty.java    |   2 +-
 .../main/java/org/apache/helix/api/Cluster.java |  10 +
 .../org/apache/helix/api/ClusterAccessor.java   | 502 -------------
 .../org/apache/helix/api/ClusterConfig.java     | 704 -------------------
 .../java/org/apache/helix/api/ClusterId.java    |  57 --
 .../java/org/apache/helix/api/ConstraintId.java |  78 --
 .../java/org/apache/helix/api/Controller.java   |   1 +
 .../apache/helix/api/ControllerAccessor.java    |  47 --
 .../java/org/apache/helix/api/ControllerId.java |  54 --
 .../src/main/java/org/apache/helix/api/Id.java  |  55 --
 .../java/org/apache/helix/api/MessageId.java    |  54 --
 .../org/apache/helix/api/NamespacedConfig.java  | 227 ------
 .../java/org/apache/helix/api/Participant.java  |   6 +
 .../apache/helix/api/ParticipantAccessor.java   | 393 -----------
 .../org/apache/helix/api/ParticipantConfig.java | 371 ----------
 .../org/apache/helix/api/ParticipantId.java     |  54 --
 .../java/org/apache/helix/api/Partition.java    |   1 +
 .../java/org/apache/helix/api/PartitionId.java  | 112 ---
 .../main/java/org/apache/helix/api/ProcId.java  |  54 --
 .../java/org/apache/helix/api/Resource.java     |   8 +-
 .../org/apache/helix/api/ResourceAccessor.java  | 233 ------
 .../org/apache/helix/api/ResourceConfig.java    | 369 ----------
 .../java/org/apache/helix/api/ResourceId.java   |  57 --
 .../org/apache/helix/api/RunningInstance.java   |   3 +
 .../apache/helix/api/SchedulerTaskConfig.java   |  68 --
 .../main/java/org/apache/helix/api/Scope.java   |   6 +
 .../java/org/apache/helix/api/SessionId.java    |  54 --
 .../java/org/apache/helix/api/Spectator.java    |   2 +
 .../java/org/apache/helix/api/SpectatorId.java  |  51 --
 .../org/apache/helix/api/StateModelDefId.java   |  66 --
 .../helix/api/StateModelDefinitionAccessor.java |  69 --
 .../apache/helix/api/StateModelFactoryId.java   |  57 --
 .../java/org/apache/helix/api/UserConfig.java   |  52 --
 .../helix/api/accessor/ClusterAccessor.java     | 553 +++++++++++++++
 .../helix/api/accessor/ControllerAccessor.java  |  49 ++
 .../helix/api/accessor/ParticipantAccessor.java | 435 ++++++++++++
 .../helix/api/accessor/ResourceAccessor.java    | 265 +++++++
 .../accessor/StateModelDefinitionAccessor.java  |  70 ++
 .../apache/helix/api/config/ClusterConfig.java  | 696 ++++++++++++++++++
 .../helix/api/config/NamespacedConfig.java      | 228 ++++++
 .../helix/api/config/ParticipantConfig.java     | 382 ++++++++++
 .../apache/helix/api/config/ResourceConfig.java | 373 ++++++++++
 .../helix/api/config/SchedulerTaskConfig.java   |  69 ++
 .../org/apache/helix/api/config/UserConfig.java |  53 ++
 .../java/org/apache/helix/api/id/ClusterId.java |  57 ++
 .../org/apache/helix/api/id/ConstraintId.java   |  80 +++
 .../org/apache/helix/api/id/ControllerId.java   |  54 ++
 .../main/java/org/apache/helix/api/id/Id.java   |  55 ++
 .../java/org/apache/helix/api/id/MessageId.java |  54 ++
 .../org/apache/helix/api/id/ParticipantId.java  |  54 ++
 .../org/apache/helix/api/id/PartitionId.java    | 112 +++
 .../java/org/apache/helix/api/id/ProcId.java    |  54 ++
 .../org/apache/helix/api/id/ResourceId.java     |  57 ++
 .../java/org/apache/helix/api/id/SessionId.java |  54 ++
 .../org/apache/helix/api/id/SpectatorId.java    |  51 ++
 .../apache/helix/api/id/StateModelDefId.java    |  66 ++
 .../helix/api/id/StateModelFactoryId.java       |  57 ++
 .../controller/rebalancer/AutoRebalancer.java   |   4 +-
 .../controller/rebalancer/CustomRebalancer.java |   4 +-
 .../rebalancer/SemiAutoRebalancer.java          |   4 +-
 .../context/BasicRebalancerContext.java         |   8 +-
 .../rebalancer/context/CustomRebalancer.java    |   4 +-
 .../context/CustomRebalancerContext.java        |   6 +-
 .../rebalancer/context/FullAutoRebalancer.java  |   4 +-
 .../context/FullAutoRebalancerContext.java      |   2 +-
 .../context/PartitionedRebalancerContext.java   |   4 +-
 .../rebalancer/context/RebalancerConfig.java    |   2 +-
 .../rebalancer/context/RebalancerContext.java   |   8 +-
 .../rebalancer/context/SemiAutoRebalancer.java  |   4 +-
 .../context/SemiAutoRebalancerContext.java      |   6 +-
 .../util/NewConstraintBasedAssignment.java      |   8 +-
 .../stages/BestPossibleStateCalcStage.java      |   4 +-
 .../stages/BestPossibleStateOutput.java         |   2 +-
 .../stages/CurrentStateComputationStage.java    |   4 +-
 .../stages/MessageGenerationPhase.java          |  10 +-
 .../stages/NewBestPossibleStateCalcStage.java   |  10 +-
 .../stages/NewBestPossibleStateOutput.java      |   2 +-
 .../stages/NewCompatibilityCheckStage.java      |   2 +-
 .../stages/NewCurrentStateComputationStage.java |  14 +-
 .../stages/NewExternalViewComputeStage.java     |  12 +-
 .../stages/NewMessageGenerationStage.java       |  18 +-
 .../controller/stages/NewMessageOutput.java     |   4 +-
 .../stages/NewMessageSelectionStage.java        |  10 +-
 .../stages/NewMessageThrottleStage.java         |   8 +-
 .../stages/NewReadClusterDataStage.java         |   4 +-
 .../stages/NewResourceComputationStage.java     |   8 +-
 .../stages/NewTaskAssignmentStage.java          |   8 +-
 .../stages/PersistAssignmentStage.java          |   4 +-
 .../controller/stages/ResourceCurrentState.java |   8 +-
 .../controller/stages/TaskAssignmentStage.java  |   2 +-
 .../strategy/EspressoRelayStrategy.java         |   2 +-
 .../manager/zk/CurStateCarryOverUpdater.java    |   4 +-
 .../DefaultSchedulerMessageHandlerFactory.java  |   8 +-
 .../apache/helix/manager/zk/ZKHelixAdmin.java   |  14 +-
 .../messaging/DefaultMessagingService.java      |   8 +-
 .../messaging/handling/BatchMessageHandler.java |   4 +-
 .../messaging/handling/GroupMessageHandler.java |   2 +-
 .../handling/HelixStateTransitionHandler.java   |   6 +-
 .../messaging/handling/HelixTaskExecutor.java   |   4 +-
 .../helix/model/ClusterConfiguration.java       |   4 +-
 .../apache/helix/model/ClusterConstraints.java  |   2 +-
 .../org/apache/helix/model/CurrentState.java    |   8 +-
 .../org/apache/helix/model/ExternalView.java    |   6 +-
 .../java/org/apache/helix/model/IdealState.java |  10 +-
 .../org/apache/helix/model/InstanceConfig.java  |   4 +-
 .../org/apache/helix/model/LiveInstance.java    |   6 +-
 .../java/org/apache/helix/model/Message.java    |  12 +-
 .../helix/model/PartitionConfiguration.java     |   4 +-
 .../apache/helix/model/ResourceAssignment.java  |   6 +-
 .../helix/model/ResourceConfiguration.java      |   4 +-
 .../helix/model/StateModelDefinition.java       |   2 +-
 .../builder/ClusterConstraintsBuilder.java      |   2 +-
 .../model/builder/CurrentStateBuilder.java      |   8 +-
 .../helix/model/builder/IdealStateBuilder.java  |   6 +-
 .../builder/MessageConstraintItemBuilder.java   |   4 +-
 .../builder/ResourceAssignmentBuilder.java      |   6 +-
 .../builder/StateConstraintItemBuilder.java     |   2 +-
 .../monitoring/mbeans/ResourceMonitor.java      |   4 +-
 .../participant/HelixCustomCodeRunner.java      |   4 +-
 .../participant/HelixStateMachineEngine.java    |  10 +-
 .../helix/tools/ClusterStateVerifier.java       |  10 +-
 .../org/apache/helix/tools/MessagePoster.java   |   6 +-
 .../org/apache/helix/util/RebalanceUtil.java    |   2 +-
 .../org/apache/helix/util/StatusUpdateUtil.java |   2 +-
 .../org/apache/helix/TestHelixTaskExecutor.java |  10 +-
 .../org/apache/helix/TestHelixTaskHandler.java  |  10 +-
 .../test/java/org/apache/helix/TestHelper.java  |   8 +-
 .../java/org/apache/helix/TestZKCallback.java   |  10 +-
 .../java/org/apache/helix/ZkUnitTestBase.java   |   6 +-
 .../test/java/org/apache/helix/api/TestId.java  |   9 +
 .../apache/helix/api/TestNamespacedConfig.java  |   2 +
 .../org/apache/helix/api/TestNewStages.java     |   6 +
 .../org/apache/helix/api/TestUpdateConfig.java  | 157 +++++
 .../context/TestSerializeRebalancerContext.java |   8 +-
 .../helix/controller/stages/BaseStageTest.java  |  12 +-
 .../TestBestPossibleCalcStageCompatibility.java |  10 +-
 .../stages/TestBestPossibleStateCalcStage.java  |   8 +-
 .../stages/TestCompatibilityCheckStage.java     |   2 +-
 .../TestCurrentStateComputationStage.java       |  12 +-
 .../stages/TestMessageThrottleStage.java        |   6 +-
 .../stages/TestMsgSelectionStage.java           |  14 +-
 .../stages/TestRebalancePipeline.java           |   6 +-
 .../stages/TestResourceComputationStage.java    |   8 +-
 .../strategy/TestAutoRebalanceStrategy.java     |  71 +-
 .../strategy/TestNewAutoRebalanceStrategy.java  |  16 +-
 .../TestAddStateModelFactoryAfterConnect.java   |   2 +-
 .../TestCustomizedIdealStateRebalancer.java     |   4 +-
 .../TestEnablePartitionDuringDisable.java       |   2 +-
 .../integration/TestInvalidAutoIdealState.java  |   2 +-
 .../TestMessagePartitionStateMismatch.java      |  10 +-
 .../helix/integration/TestMessageThrottle2.java |   2 +-
 .../helix/integration/TestMessagingService.java |   4 +-
 .../TestParticipantErrorMessage.java            |   8 +-
 .../helix/integration/TestRenamePartition.java  |   2 +-
 .../helix/integration/TestSchedulerMessage.java |   4 +-
 .../integration/TestSchedulerMsgContraints.java |   4 +-
 .../integration/TestSchedulerMsgUsingQueue.java |   4 +-
 .../TestSessionExpiryInTransition.java          |   2 +-
 .../integration/TestStateTransitionTimeout.java |   4 +-
 .../helix/integration/TestZkReconnect.java      |   8 +-
 .../manager/TestParticipantManager.java         |   2 +-
 .../TestDefaultControllerMsgHandlerFactory.java |   2 +-
 .../helix/manager/zk/TestZNRecordSizeLimit.java |   2 +-
 .../helix/manager/zk/TestZkHelixAdmin.java      |   2 +-
 .../helix/messaging/TestAsyncCallback.java      |   2 +-
 .../helix/messaging/TestAsyncCallbackSvc.java   |   4 +-
 .../messaging/TestDefaultMessagingService.java  |   2 +-
 .../handling/TestHelixTaskExecutor.java         |   4 +-
 .../helix/mock/controller/MockController.java   |   6 +-
 .../mock/controller/MockControllerProcess.java  |   2 +-
 .../helix/mock/participant/DummyProcess.java    |   2 +-
 .../helix/mock/participant/ErrTransition.java   |   2 +-
 .../StoreAccessOneNodeTransition.java           |   2 +-
 .../org/apache/helix/model/TestConstraint.java  |   2 +-
 .../org/apache/helix/model/TestIdealState.java  |   6 +-
 .../TestDistControllerStateModel.java           |   4 +-
 .../apache/helix/tools/TestHelixAdminCli.java   |   4 +-
 .../apache/helix/examples/BootstrapHandler.java |   2 +-
 .../apache/helix/examples/DummyParticipant.java |   2 +-
 .../apache/helix/examples/NewModelExample.java  |  20 +-
 .../org/apache/helix/examples/Quickstart.java   |   2 +-
 .../helix/taskexecution/TaskStateModel.java     |   2 +-
 183 files changed, 4577 insertions(+), 4201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java
index 2d3966c..40c527a 100644
--- a/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java
+++ b/helix-admin-webapp/src/main/java/org/apache/helix/webapp/resources/SchedulerTasksResource.java
@@ -31,8 +31,8 @@ import org.apache.helix.HelixException;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.SessionId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.LiveInstance;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/HelixProperty.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/HelixProperty.java b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
index f52d51c..9c0c25e 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixProperty.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixProperty.java
@@ -27,7 +27,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.api.NamespacedConfig;
+import org.apache.helix.api.config.NamespacedConfig;
 
 /**
  * A wrapper class for ZNRecord. Used as a base class for IdealState, CurrentState, etc.

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Cluster.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Cluster.java b/helix-core/src/main/java/org/apache/helix/api/Cluster.java
index 9e71904..3d24498 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Cluster.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Cluster.java
@@ -22,6 +22,16 @@ package org.apache.helix.api;
 import java.util.Collections;
 import java.util.Map;
 
+import org.apache.helix.api.config.ClusterConfig;
+import org.apache.helix.api.config.ParticipantConfig;
+import org.apache.helix.api.config.ResourceConfig;
+import org.apache.helix.api.config.UserConfig;
+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.SpectatorId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.StateModelDefinition;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
deleted file mode 100644
index 6302e33..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
+++ /dev/null
@@ -1,502 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.AccessOption;
-import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.model.ClusterConfiguration;
-import org.apache.helix.model.ClusterConstraints;
-import org.apache.helix.model.ClusterConstraints.ConstraintType;
-import org.apache.helix.model.CurrentState;
-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.Message;
-import org.apache.helix.model.PauseSignal;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.ResourceConfiguration;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-public class ClusterAccessor {
-  private static Logger LOG = Logger.getLogger(ClusterAccessor.class);
-
-  private final HelixDataAccessor _accessor;
-  private final PropertyKey.Builder _keyBuilder;
-  private final ClusterId _clusterId;
-
-  public ClusterAccessor(ClusterId clusterId, HelixDataAccessor accessor) {
-    _accessor = accessor;
-    _keyBuilder = accessor.keyBuilder();
-    _clusterId = clusterId;
-  }
-
-  /**
-   * create a new cluster, fail if it already exists
-   * @return true if created, false if creation failed
-   */
-  public boolean createCluster(ClusterConfig cluster) {
-    boolean created = _accessor.createProperty(_keyBuilder.cluster(), null);
-    if (!created) {
-      LOG.error("Cluster already created. Aborting.");
-      return false;
-    }
-    initClusterStructure();
-    Map<StateModelDefId, StateModelDefinition> stateModelDefs = cluster.getStateModelMap();
-    for (StateModelDefinition stateModelDef : stateModelDefs.values()) {
-      addStateModelDefinitionToCluster(stateModelDef);
-    }
-    Map<ResourceId, ResourceConfig> resources = cluster.getResourceMap();
-    for (ResourceConfig resource : resources.values()) {
-      addResourceToCluster(resource);
-    }
-    Map<ParticipantId, ParticipantConfig> participants = cluster.getParticipantMap();
-    for (ParticipantConfig participant : participants.values()) {
-      addParticipantToCluster(participant);
-    }
-    _accessor.createProperty(_keyBuilder.constraints(), null);
-    for (ClusterConstraints constraints : cluster.getConstraintMap().values()) {
-      _accessor.createProperty(_keyBuilder.constraint(constraints.getType().toString()),
-          constraints);
-    }
-    _accessor.createProperty(_keyBuilder.clusterConfig(),
-        ClusterConfiguration.from(cluster.getUserConfig()));
-    if (cluster.isPaused()) {
-      pauseCluster();
-    }
-
-    return true;
-  }
-
-  public ClusterConfig updateCluster(ClusterConfig.Delta clusterDelta) {
-    Cluster cluster = readCluster();
-    if (cluster == null) {
-      LOG.error("Cluster does not exist, cannot be updated");
-      return null;
-    }
-    ClusterConfig config = clusterDelta.mergeInto(cluster.getConfig());
-    // TODO: persist this
-    return config;
-  }
-
-  /**
-   * drop a cluster
-   * @return true if the cluster was dropped, false if there was an error
-   */
-  public boolean dropCluster() {
-    LOG.info("Dropping cluster: " + _clusterId);
-    List<String> liveInstanceNames = _accessor.getChildNames(_keyBuilder.liveInstances());
-    if (liveInstanceNames.size() > 0) {
-      LOG.error("Can't drop cluster: " + _clusterId + " because there are running participant: "
-          + liveInstanceNames + ", shutdown participants first.");
-      return false;
-    }
-
-    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
-    if (leader != null) {
-      LOG.error("Can't drop cluster: " + _clusterId + ", because leader: " + leader.getId()
-          + " are running, shutdown leader first.");
-      return false;
-    }
-
-    return _accessor.removeProperty(_keyBuilder.cluster());
-  }
-
-  /**
-   * read entire cluster data
-   * @return cluster
-   */
-  public Cluster readCluster() {
-    /**
-     * map of instance-id to instance-config
-     */
-    Map<String, InstanceConfig> instanceConfigMap =
-        _accessor.getChildValuesMap(_keyBuilder.instanceConfigs());
-
-    /**
-     * map of resource-id to ideal-state
-     */
-    Map<String, IdealState> idealStateMap = _accessor.getChildValuesMap(_keyBuilder.idealStates());
-
-    /**
-     * map of instance-id to live-instance
-     */
-    Map<String, LiveInstance> liveInstanceMap =
-        _accessor.getChildValuesMap(_keyBuilder.liveInstances());
-
-    /**
-     * map of participant-id to map of message-id to message
-     */
-    Map<String, Map<String, Message>> messageMap = new HashMap<String, Map<String, Message>>();
-    for (String instanceName : liveInstanceMap.keySet()) {
-      Map<String, Message> instanceMsgMap =
-          _accessor.getChildValuesMap(_keyBuilder.messages(instanceName));
-      messageMap.put(instanceName, instanceMsgMap);
-    }
-
-    /**
-     * map of participant-id to map of resource-id to current-state
-     */
-    Map<String, Map<String, CurrentState>> currentStateMap =
-        new HashMap<String, Map<String, CurrentState>>();
-    for (String participantName : liveInstanceMap.keySet()) {
-      LiveInstance liveInstance = liveInstanceMap.get(participantName);
-      SessionId sessionId = liveInstance.getSessionId();
-      Map<String, CurrentState> instanceCurStateMap =
-          _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName,
-              sessionId.stringify()));
-
-      currentStateMap.put(participantName, instanceCurStateMap);
-    }
-
-    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
-
-    /**
-     * map of constraint-type to constraints
-     */
-    Map<String, ClusterConstraints> constraintMap =
-        _accessor.getChildValuesMap(_keyBuilder.constraints());
-
-    /**
-     * Map of resource id to external view
-     */
-    Map<String, ExternalView> externalViewMap =
-        _accessor.getChildValuesMap(_keyBuilder.externalViews());
-
-    /**
-     * Map of resource id to user configuration
-     */
-    Map<String, ResourceConfiguration> resourceConfigMap =
-        _accessor.getChildValuesMap(_keyBuilder.resourceConfigs());
-
-    /**
-     * Map of resource id to resource assignment
-     */
-    Map<String, ResourceAssignment> resourceAssignmentMap =
-        _accessor.getChildValuesMap(_keyBuilder.resourceAssignments());
-
-    // read all the resources
-    Map<ResourceId, Resource> resourceMap = new HashMap<ResourceId, Resource>();
-    for (String resourceName : idealStateMap.keySet()) {
-      ResourceId resourceId = ResourceId.from(resourceName);
-      resourceMap.put(resourceId, ResourceAccessor.createResource(resourceId,
-          resourceConfigMap.get(resourceName), idealStateMap.get(resourceName),
-          externalViewMap.get(resourceName), resourceAssignmentMap.get(resourceName)));
-    }
-
-    // read all the participants
-    Map<ParticipantId, Participant> participantMap = new HashMap<ParticipantId, Participant>();
-    for (String participantName : instanceConfigMap.keySet()) {
-      InstanceConfig instanceConfig = instanceConfigMap.get(participantName);
-      UserConfig userConfig = UserConfig.from(instanceConfig);
-      LiveInstance liveInstance = liveInstanceMap.get(participantName);
-      Map<String, Message> instanceMsgMap = messageMap.get(participantName);
-
-      ParticipantId participantId = ParticipantId.from(participantName);
-
-      participantMap.put(participantId, ParticipantAccessor.createParticipant(participantId,
-          instanceConfig, userConfig, liveInstance, instanceMsgMap,
-          currentStateMap.get(participantName)));
-    }
-
-    // read the controllers
-    Map<ControllerId, Controller> controllerMap = new HashMap<ControllerId, Controller>();
-    ControllerId leaderId = null;
-    if (leader != null) {
-      leaderId = ControllerId.from(leader.getId());
-      controllerMap.put(leaderId, new Controller(leaderId, leader, true));
-    }
-
-    // read the constraints
-    Map<ConstraintType, ClusterConstraints> clusterConstraintMap =
-        new HashMap<ConstraintType, ClusterConstraints>();
-    for (String constraintType : constraintMap.keySet()) {
-      clusterConstraintMap.put(ConstraintType.valueOf(constraintType),
-          constraintMap.get(constraintType));
-    }
-
-    // read the pause status
-    PauseSignal pauseSignal = _accessor.getProperty(_keyBuilder.pause());
-    boolean isPaused = pauseSignal != null;
-
-    ClusterConfiguration clusterUserConfig = _accessor.getProperty(_keyBuilder.clusterConfig());
-    UserConfig userConfig;
-    if (clusterUserConfig != null) {
-      userConfig = UserConfig.from(clusterUserConfig);
-    } else {
-      userConfig = new UserConfig(Scope.cluster(_clusterId));
-    }
-
-    // read the state model definitions
-    StateModelDefinitionAccessor stateModelDefAccessor =
-        new StateModelDefinitionAccessor(_accessor);
-    Map<StateModelDefId, StateModelDefinition> stateModelMap =
-        stateModelDefAccessor.readStateModelDefinitions();
-
-    // create the cluster snapshot object
-    return new Cluster(_clusterId, resourceMap, participantMap, controllerMap, leaderId,
-        clusterConstraintMap, stateModelMap, userConfig, isPaused);
-  }
-
-  /**
-   * pause controller of cluster
-   */
-  public void pauseCluster() {
-    _accessor.createProperty(_keyBuilder.pause(), new PauseSignal("pause"));
-  }
-
-  /**
-   * resume controller of cluster
-   */
-  public void resumeCluster() {
-    _accessor.removeProperty(_keyBuilder.pause());
-  }
-
-  /**
-   * add a resource to cluster
-   * @param resource
-   * @return true if resource added, false if there was an error
-   */
-  public boolean addResourceToCluster(ResourceConfig resource) {
-    if (!isClusterStructureValid()) {
-      LOG.error("Cluster: " + _clusterId + " structure is not valid");
-      return false;
-    }
-    RebalancerContext context =
-        resource.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
-    StateModelDefId stateModelDefId = context.getStateModelDefId();
-    if (_accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())) == null) {
-      LOG.error("State model: " + stateModelDefId + " not found in cluster: " + _clusterId);
-      return false;
-    }
-
-    ResourceId resourceId = resource.getId();
-    if (_accessor.getProperty(_keyBuilder.idealState(resourceId.stringify())) != null) {
-      LOG.error("Skip adding resource: " + resourceId
-          + ", because resource ideal state already exists in cluster: " + _clusterId);
-      return false;
-    }
-    if (_accessor.getProperty(_keyBuilder.resourceConfig(resourceId.stringify())) != null) {
-      LOG.error("Skip adding resource: " + resourceId
-          + ", because resource config already exists in cluster: " + _clusterId);
-      return false;
-    }
-
-    // Add resource user config
-    if (resource.getUserConfig() != null) {
-      ResourceConfiguration configuration = new ResourceConfiguration(resourceId);
-      configuration.setType(resource.getType());
-      configuration.addNamespacedConfig(resource.getUserConfig());
-      configuration.addNamespacedConfig(resource.getRebalancerConfig().toNamespacedConfig());
-      configuration.setBucketSize(resource.getBucketSize());
-      configuration.setBatchMessageMode(resource.getBatchMessageMode());
-      _accessor.createProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration);
-    }
-
-    // Create an IdealState from a RebalancerConfig (if the resource is partitioned)
-    RebalancerConfig rebalancerConfig = resource.getRebalancerConfig();
-    IdealState idealState =
-        ResourceAccessor.rebalancerConfigToIdealState(rebalancerConfig, resource.getBucketSize(),
-            resource.getBatchMessageMode());
-    if (idealState != null) {
-      _accessor.createProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
-    }
-    return true;
-  }
-
-  /**
-   * drop a resource from cluster
-   * @param resourceId
-   * @return true if removal succeeded, false otherwise
-   */
-  public boolean dropResourceFromCluster(ResourceId resourceId) {
-    if (_accessor.getProperty(_keyBuilder.idealState(resourceId.stringify())) == null) {
-      LOG.error("Skip removing resource: " + resourceId
-          + ", because resource ideal state already removed from cluster: " + _clusterId);
-      return false;
-    }
-    _accessor.removeProperty(_keyBuilder.idealState(resourceId.stringify()));
-    _accessor.removeProperty(_keyBuilder.resourceConfig(resourceId.stringify()));
-    return true;
-  }
-
-  /**
-   * check if cluster structure is valid
-   * @return true if valid or false otherwise
-   */
-  public boolean isClusterStructureValid() {
-    List<String> paths = getRequiredPaths();
-    BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
-    boolean[] existsResults = baseAccessor.exists(paths, 0);
-    for (boolean exists : existsResults) {
-      if (!exists) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Create empty persistent properties to ensure that there is a valid cluster structure
-   */
-  private void initClusterStructure() {
-    BaseDataAccessor<?> baseAccessor = _accessor.getBaseDataAccessor();
-    List<String> paths = getRequiredPaths();
-    for (String path : paths) {
-      boolean status = baseAccessor.create(path, null, AccessOption.PERSISTENT);
-      if (!status && LOG.isDebugEnabled()) {
-        LOG.debug(path + " already exists");
-      }
-    }
-  }
-
-  /**
-   * Get all property paths that must be set for a cluster structure to be valid
-   * @return list of paths as strings
-   */
-  private List<String> getRequiredPaths() {
-    List<String> paths = new ArrayList<String>();
-    paths.add(_keyBuilder.cluster().getPath());
-    paths.add(_keyBuilder.idealStates().getPath());
-    paths.add(_keyBuilder.clusterConfigs().getPath());
-    paths.add(_keyBuilder.instanceConfigs().getPath());
-    paths.add(_keyBuilder.resourceConfigs().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
-   */
-  public boolean addParticipantToCluster(ParticipantConfig participant) {
-    if (!isClusterStructureValid()) {
-      LOG.error("Cluster: " + _clusterId + " structure is not valid");
-      return false;
-    }
-
-    ParticipantId participantId = participant.getId();
-    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())) != null) {
-      LOG.error("Config for participant: " + participantId + " already exists in cluster: "
-          + _clusterId);
-      return false;
-    }
-
-    // add empty root ZNodes
-    List<PropertyKey> createKeys = new ArrayList<PropertyKey>();
-    createKeys.add(_keyBuilder.messages(participantId.stringify()));
-    createKeys.add(_keyBuilder.currentStates(participantId.stringify()));
-    createKeys.add(_keyBuilder.participantErrors(participantId.stringify()));
-    createKeys.add(_keyBuilder.statusUpdates(participantId.stringify()));
-    for (PropertyKey key : createKeys) {
-      _accessor.createProperty(key, null);
-    }
-
-    // add the config
-    InstanceConfig instanceConfig = new InstanceConfig(participant.getId());
-    instanceConfig.setHostName(participant.getHostName());
-    instanceConfig.setPort(Integer.toString(participant.getPort()));
-    instanceConfig.setInstanceEnabled(participant.isEnabled());
-    UserConfig userConfig = participant.getUserConfig();
-    instanceConfig.addNamespacedConfig(userConfig);
-    Set<String> tags = participant.getTags();
-    for (String tag : tags) {
-      instanceConfig.addTag(tag);
-    }
-    Set<PartitionId> disabledPartitions = participant.getDisabledPartitions();
-    for (PartitionId partitionId : disabledPartitions) {
-      instanceConfig.setInstanceEnabledForPartition(partitionId, false);
-    }
-    _accessor.createProperty(_keyBuilder.instanceConfig(participantId.stringify()), instanceConfig);
-    _accessor.createProperty(_keyBuilder.messages(participantId.stringify()), null);
-    return true;
-  }
-
-  /**
-   * drop a participant from cluster
-   * @param participantId
-   * @return true if participant dropped, false if there was an error
-   */
-  public boolean dropParticipantFromCluster(ParticipantId participantId) {
-    if (_accessor.getProperty(_keyBuilder.instanceConfig(participantId.stringify())) == null) {
-      LOG.error("Config for participant: " + participantId + " does NOT exist in cluster: "
-          + _clusterId);
-      return false;
-    }
-
-    if (_accessor.getProperty(_keyBuilder.instance(participantId.stringify())) == null) {
-      LOG.error("Participant: " + participantId + " structure does NOT exist in cluster: "
-          + _clusterId);
-      return false;
-    }
-
-    // delete participant config path
-    _accessor.removeProperty(_keyBuilder.instanceConfig(participantId.stringify()));
-
-    // delete participant path
-    _accessor.removeProperty(_keyBuilder.instance(participantId.stringify()));
-    return true;
-  }
-
-  /**
-   * Add a state model definition. Updates the existing state model definition if it already exists.
-   * @param stateModelDef fully initialized state model definition
-   * @return true if the model is persisted, false otherwise
-   */
-  public boolean addStateModelDefinitionToCluster(StateModelDefinition stateModelDef) {
-    if (!isClusterStructureValid()) {
-      LOG.error("Cluster: " + _clusterId + " structure is not valid");
-      return false;
-    }
-
-    StateModelDefinitionAccessor smdAccessor = new StateModelDefinitionAccessor(_accessor);
-    return smdAccessor.setStateModelDefinition(stateModelDef);
-  }
-
-  /**
-   * Remove a state model definition if it exists
-   * @param stateModelDefId state model definition id
-   * @return true if removed, false if it did not exist
-   */
-  public boolean dropStateModelDefinitionFromCluster(StateModelDefId stateModelDefId) {
-    return _accessor.removeProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ClusterConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/api/ClusterConfig.java
deleted file mode 100644
index 590fb01..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ClusterConfig.java
+++ /dev/null
@@ -1,704 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.model.ClusterConstraints;
-import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
-import org.apache.helix.model.ClusterConstraints.ConstraintType;
-import org.apache.helix.model.ClusterConstraints.ConstraintValue;
-import org.apache.helix.model.ConstraintItem;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.model.Transition;
-import org.apache.helix.model.builder.ConstraintItemBuilder;
-import org.apache.log4j.Logger;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-/*
- * 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.
- */
-
-/**
- * Configuration properties of a cluster
- */
-public class ClusterConfig {
-  private static final Logger LOG = Logger.getLogger(ClusterConfig.class);
-
-  private final ClusterId _id;
-  private final Map<ResourceId, ResourceConfig> _resourceMap;
-  private final Map<ParticipantId, ParticipantConfig> _participantMap;
-  private final Map<ConstraintType, ClusterConstraints> _constraintMap;
-  private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
-  private final UserConfig _userConfig;
-  private final boolean _isPaused;
-
-  /**
-   * Initialize a cluster configuration. Also see ClusterConfig.Builder
-   * @param id cluster id
-   * @param resourceMap map of resource id to resource config
-   * @param participantMap map of participant id to participant config
-   * @param constraintMap map of constraint type to all constraints of that type
-   * @param stateModelMap map of state model id to state model definition
-   * @param userConfig user-defined cluster properties
-   * @param isPaused true if paused, false if active
-   */
-  private ClusterConfig(ClusterId id, Map<ResourceId, ResourceConfig> resourceMap,
-      Map<ParticipantId, ParticipantConfig> participantMap,
-      Map<ConstraintType, ClusterConstraints> constraintMap,
-      Map<StateModelDefId, StateModelDefinition> stateModelMap, UserConfig userConfig,
-      boolean isPaused) {
-    _id = id;
-    _resourceMap = ImmutableMap.copyOf(resourceMap);
-    _participantMap = ImmutableMap.copyOf(participantMap);
-    _constraintMap = ImmutableMap.copyOf(constraintMap);
-    _stateModelMap = ImmutableMap.copyOf(stateModelMap);
-    _userConfig = userConfig;
-    _isPaused = isPaused;
-  }
-
-  /**
-   * Get cluster id
-   * @return cluster id
-   */
-  public ClusterId getId() {
-    return _id;
-  }
-
-  /**
-   * Get resources in the cluster
-   * @return a map of resource id to resource, or empty map if none
-   */
-  public Map<ResourceId, ResourceConfig> getResourceMap() {
-    return _resourceMap;
-  }
-
-  /**
-   * Get all the constraints on the cluster
-   * @return map of constraint type to constraints
-   */
-  public Map<ConstraintType, ClusterConstraints> getConstraintMap() {
-    return _constraintMap;
-  }
-
-  /**
-   * Get the maximum number of participants that can be in a state
-   * @param scope the scope for the bound
-   * @param stateModelDefId the state model of the state
-   * @param state the constrained state
-   * @return The upper bound, which can be "-1" if unspecified, a numerical upper bound, "R" for
-   *         number of replicas, or "N" for number of participants
-   */
-  public String getStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-      State state) {
-    // set up attributes to match based on the scope
-    ClusterConstraints stateConstraints = getConstraintMap().get(ConstraintType.STATE_CONSTRAINT);
-    Map<ConstraintAttribute, String> matchAttributes = Maps.newHashMap();
-    matchAttributes.put(ConstraintAttribute.STATE, state.toString());
-    matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString());
-    switch (scope.getType()) {
-    case CLUSTER:
-      // cluster is implicit
-      break;
-    case RESOURCE:
-      matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify());
-      break;
-    default:
-      LOG.error("Unsupported scope for state constraint: " + scope);
-      return "-1";
-    }
-    Set<ConstraintItem> matches = stateConstraints.match(matchAttributes);
-    int value = -1;
-    for (ConstraintItem item : matches) {
-      // match: if an R or N is found, always choose that one
-      // otherwise, take the minimum of the counts specified in the constraints
-      String constraintValue = item.getConstraintValue();
-      if (constraintValue != null) {
-        if (constraintValue.equals(ConstraintValue.N.toString())
-            || constraintValue.equals(ConstraintValue.R.toString())) {
-          return constraintValue;
-        } else {
-          try {
-            int current = Integer.parseInt(constraintValue);
-            if (value == -1 || current < value) {
-              value = current;
-            }
-          } catch (NumberFormatException e) {
-            LOG.error("Invalid state upper bound: " + constraintValue);
-          }
-        }
-      }
-    }
-    return Integer.toString(value);
-  }
-
-  /**
-   * Get the limit of simultaneous execution of a transition
-   * @param scope the scope under which the transition is constrained
-   * @param stateModelDefId the state model of which the transition is a part
-   * @param transition the constrained transition
-   * @return the limit, or Integer.MAX_VALUE if there is no limit
-   */
-  public int getTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-      Transition transition) {
-    // set up attributes to match based on the scope
-    ClusterConstraints transitionConstraints =
-        getConstraintMap().get(ConstraintType.MESSAGE_CONSTRAINT);
-    Map<ConstraintAttribute, String> matchAttributes = Maps.newHashMap();
-    matchAttributes.put(ConstraintAttribute.STATE_MODEL, stateModelDefId.toString());
-    matchAttributes.put(ConstraintAttribute.MESSAGE_TYPE, MessageType.STATE_TRANSITION.toString());
-    matchAttributes.put(ConstraintAttribute.TRANSITION, transition.toString());
-    switch (scope.getType()) {
-    case CLUSTER:
-      // cluster is implicit
-      break;
-    case RESOURCE:
-      matchAttributes.put(ConstraintAttribute.RESOURCE, scope.getScopedId().stringify());
-      break;
-    case PARTICIPANT:
-      matchAttributes.put(ConstraintAttribute.INSTANCE, scope.getScopedId().stringify());
-      break;
-    default:
-      LOG.error("Unsupported scope for transition constraints: " + scope);
-      return Integer.MAX_VALUE;
-    }
-    Set<ConstraintItem> matches = transitionConstraints.match(matchAttributes);
-    int value = Integer.MAX_VALUE;
-    for (ConstraintItem item : matches) {
-      String constraintValue = item.getConstraintValue();
-      if (constraintValue != null) {
-        try {
-          int current = Integer.parseInt(constraintValue);
-          if (current < value) {
-            value = current;
-          }
-        } catch (NumberFormatException e) {
-          LOG.error("Invalid in-flight transition cap: " + constraintValue);
-        }
-      }
-    }
-    return value;
-  }
-
-  /**
-   * Get participants of the cluster
-   * @return a map of participant id to participant, or empty map if none
-   */
-  public Map<ParticipantId, ParticipantConfig> getParticipantMap() {
-    return _participantMap;
-  }
-
-  /**
-   * Get all the state model definitions on the cluster
-   * @return map of state model definition id to state model definition
-   */
-  public Map<StateModelDefId, StateModelDefinition> getStateModelMap() {
-    return _stateModelMap;
-  }
-
-  /**
-   * Get user-specified configuration properties of this cluster
-   * @return UserConfig properties
-   */
-  public UserConfig getUserConfig() {
-    return _userConfig;
-  }
-
-  /**
-   * Check the paused status of the cluster
-   * @return true if paused, false otherwise
-   */
-  public boolean isPaused() {
-    return _isPaused;
-  }
-
-  /**
-   * Update context for a ClusterConfig
-   */
-  public static class Delta {
-    private enum Fields {
-      PAUSE_STATUS,
-      USER_CONFIG
-    }
-
-    private Set<Fields> _updateFields;
-    private Map<ConstraintType, Set<ConstraintId>> _removedConstraints;
-    private Builder _builder;
-
-    /**
-     * Instantiate the delta for a cluster config
-     * @param clusterId the cluster to update
-     */
-    public Delta(ClusterId clusterId) {
-      _updateFields = Sets.newHashSet();
-      _removedConstraints = Maps.newHashMap();
-      for (ConstraintType type : ConstraintType.values()) {
-        Set<ConstraintId> constraints = Sets.newHashSet();
-        _removedConstraints.put(type, constraints);
-      }
-      _builder = new Builder(clusterId);
-    }
-
-    /**
-     * Add a state upper bound constraint
-     * @param scope scope under which the constraint is valid
-     * @param stateModelDefId identifier of the state model that owns the state
-     * @param state the state to constrain
-     * @param upperBound maximum number of replicas per partition in the state
-     * @return Delta
-     */
-    public Delta addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        State state, int upperBound) {
-      return addStateUpperBoundConstraint(scope, stateModelDefId, state,
-          Integer.toString(upperBound));
-    }
-
-    /**
-     * Add a state upper bound constraint
-     * @param scope scope under which the constraint is valid
-     * @param stateModelDefId identifier of the state model that owns the state
-     * @param state the state to constrain
-     * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a
-     *          number, or the currently supported special bound values:<br />
-     *          "R" - Refers to the number of replicas specified during resource
-     *          creation. This allows having different replication factor for each
-     *          resource without having to create a different state machine. <br />
-     *          "N" - Refers to all nodes in the cluster. Useful for resources that need
-     *          to exist on all nodes. This way one can add/remove nodes without having
-     *          the change the bounds.
-     * @return Delta
-     */
-    public Delta addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        State state, String dynamicUpperBound) {
-      _builder.addStateUpperBoundConstraint(scope, stateModelDefId, state, dynamicUpperBound);
-      return this;
-    }
-
-    /**
-     * Remove state upper bound constraint
-     * @param scope scope under which the constraint is valid
-     * @param stateModelDefId identifier of the state model that owns the state
-     * @param state the state to constrain
-     * @return Delta
-     */
-    public Delta removeStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        State state) {
-      _removedConstraints.get(ConstraintType.STATE_CONSTRAINT).add(
-          ConstraintId.from(scope, stateModelDefId, state));
-      return this;
-    }
-
-    /**
-     * Add a constraint on the maximum number of in-flight transitions of a certain type
-     * @param scope scope of the constraint
-     * @param stateModelDefId identifies the state model containing the transition
-     * @param transition the transition to constrain
-     * @param maxInFlightTransitions number of allowed in-flight transitions in the scope
-     * @return Delta
-     */
-    public Delta addTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        Transition transition, int maxInFlightTransitions) {
-      _builder.addTransitionConstraint(scope, stateModelDefId, transition, maxInFlightTransitions);
-      return this;
-    }
-
-    /**
-     * Remove a constraint on the maximum number of in-flight transitions of a certain type
-     * @param scope scope of the constraint
-     * @param stateModelDefId identifies the state model containing the transition
-     * @param transition the transition to constrain
-     * @return Delta
-     */
-    public Delta removeTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        Transition transition) {
-      _removedConstraints.get(ConstraintType.MESSAGE_CONSTRAINT).add(
-          ConstraintId.from(scope, stateModelDefId, transition));
-      return this;
-    }
-
-    /**
-     * Add a single constraint item
-     * @param type type of the constraint item
-     * @param constraintId unique constraint id
-     * @param item instantiated ConstraintItem
-     * @return Delta
-     */
-    public Delta addConstraintItem(ConstraintType type, ConstraintId constraintId,
-        ConstraintItem item) {
-      _builder.addConstraint(type, constraintId, item);
-      return this;
-    }
-
-    /**
-     * Remove a single constraint item
-     * @param type type of the constraint item
-     * @param constraintId unique constraint id
-     * @return Delta
-     */
-    public Delta removeConstraintItem(ConstraintType type, ConstraintId constraintId) {
-      _removedConstraints.get(type).add(constraintId);
-      return this;
-    }
-
-    /**
-     * Set the paused status of the cluster
-     * @param isPaused true if paused, false otherwise
-     * @return Delta
-     */
-    public Delta setPausedStatus(boolean isPaused) {
-      _builder.pausedStatus(isPaused);
-      _updateFields.add(Fields.PAUSE_STATUS);
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Builder
-     */
-    public Delta setUserConfig(UserConfig userConfig) {
-      _builder.userConfig(userConfig);
-      _updateFields.add(Fields.USER_CONFIG);
-      return this;
-    }
-
-    /**
-     * Create a ClusterConfig that is the combination of an existing ClusterConfig and this delta
-     * @param orig the original ClusterConfig
-     * @return updated ClusterConfig
-     */
-    public ClusterConfig mergeInto(ClusterConfig orig) {
-      // copy in original and updated fields
-      ClusterConfig deltaConfig = _builder.build();
-      Builder builder =
-          new Builder(orig.getId()).addResources(orig.getResourceMap().values())
-              .addParticipants(orig.getParticipantMap().values())
-              .addStateModelDefinitions(orig.getStateModelMap().values())
-              .userConfig(orig.getUserConfig()).pausedStatus(orig.isPaused());
-      for (Fields field : _updateFields) {
-        switch (field) {
-        case PAUSE_STATUS:
-          _builder.pausedStatus(deltaConfig.isPaused());
-          break;
-        case USER_CONFIG:
-          _builder.userConfig(deltaConfig.getUserConfig());
-          break;
-        }
-      }
-      // add constraint deltas
-      for (ConstraintType type : ConstraintType.values()) {
-        ClusterConstraints constraints;
-        if (orig.getConstraintMap().containsKey(type)) {
-          constraints = orig.getConstraintMap().get(type);
-        } else {
-          constraints = new ClusterConstraints(type);
-        }
-        // add new constraints
-        if (deltaConfig.getConstraintMap().containsKey(type)) {
-          ClusterConstraints deltaConstraints = deltaConfig.getConstraintMap().get(type);
-          for (ConstraintId constraintId : deltaConstraints.getConstraintItems().keySet()) {
-            ConstraintItem constraintItem = deltaConstraints.getConstraintItem(constraintId);
-            constraints.addConstraintItem(constraintId, constraintItem);
-          }
-        }
-        // remove constraints
-        for (ConstraintId constraintId : _removedConstraints.get(type)) {
-          constraints.removeConstraintItem(constraintId);
-        }
-        builder.addConstraint(constraints);
-      }
-      return builder.build();
-    }
-  }
-
-  /**
-   * Assembles a cluster configuration
-   */
-  public static class Builder {
-    private final ClusterId _id;
-    private final Map<ResourceId, ResourceConfig> _resourceMap;
-    private final Map<ParticipantId, ParticipantConfig> _participantMap;
-    private final Map<ConstraintType, ClusterConstraints> _constraintMap;
-    private final Map<StateModelDefId, StateModelDefinition> _stateModelMap;
-    private UserConfig _userConfig;
-    private boolean _isPaused;
-
-    /**
-     * Initialize builder for a cluster
-     * @param id cluster id
-     */
-    public Builder(ClusterId id) {
-      _id = id;
-      _resourceMap = new HashMap<ResourceId, ResourceConfig>();
-      _participantMap = new HashMap<ParticipantId, ParticipantConfig>();
-      _constraintMap = new HashMap<ConstraintType, ClusterConstraints>();
-      _stateModelMap = new HashMap<StateModelDefId, StateModelDefinition>();
-      _isPaused = false;
-      _userConfig = new UserConfig(Scope.cluster(id));
-    }
-
-    /**
-     * Add a resource to the cluster
-     * @param resource resource configuration
-     * @return Builder
-     */
-    public Builder addResource(ResourceConfig resource) {
-      _resourceMap.put(resource.getId(), resource);
-      return this;
-    }
-
-    /**
-     * Add multiple resources to the cluster
-     * @param resources resource configurations
-     * @return Builder
-     */
-    public Builder addResources(Collection<ResourceConfig> resources) {
-      for (ResourceConfig resource : resources) {
-        addResource(resource);
-      }
-      return this;
-    }
-
-    /**
-     * Add a participant to the cluster
-     * @param participant participant configuration
-     * @return Builder
-     */
-    public Builder addParticipant(ParticipantConfig participant) {
-      _participantMap.put(participant.getId(), participant);
-      return this;
-    }
-
-    /**
-     * Add multiple participants to the cluster
-     * @param participants participant configurations
-     * @return Builder
-     */
-    public Builder addParticipants(Collection<ParticipantConfig> participants) {
-      for (ParticipantConfig participant : participants) {
-        addParticipant(participant);
-      }
-      return this;
-    }
-
-    /**
-     * Add a constraint to the cluster
-     * @param constraint cluster constraint of a specific type
-     * @return Builder
-     */
-    public Builder addConstraint(ClusterConstraints constraint) {
-      ClusterConstraints existConstraints = getConstraintsInstance(constraint.getType());
-      for (ConstraintId constraintId : constraint.getConstraintItems().keySet()) {
-        existConstraints
-            .addConstraintItem(constraintId, constraint.getConstraintItem(constraintId));
-      }
-      return this;
-    }
-
-    /**
-     * Add a single constraint item
-     * @param type type of the constraint
-     * @param constraintId unique constraint identifier
-     * @param item instantiated ConstraintItem
-     * @return Builder
-     */
-    public Builder addConstraint(ConstraintType type, ConstraintId constraintId, ConstraintItem item) {
-      ClusterConstraints existConstraints = getConstraintsInstance(type);
-      existConstraints.addConstraintItem(constraintId, item);
-      return this;
-    }
-
-    /**
-     * Add multiple constraints to the cluster
-     * @param constraints cluster constraints of multiple distinct types
-     * @return Builder
-     */
-    public Builder addConstraints(Collection<ClusterConstraints> constraints) {
-      for (ClusterConstraints constraint : constraints) {
-        addConstraint(constraint);
-      }
-      return this;
-    }
-
-    /**
-     * Add a constraint on the maximum number of in-flight transitions of a certain type
-     * @param scope scope of the constraint
-     * @param stateModelDefId identifies the state model containing the transition
-     * @param transition the transition to constrain
-     * @param maxInFlightTransitions number of allowed in-flight transitions in the scope
-     * @return Builder
-     */
-    public Builder addTransitionConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        Transition transition, int maxInFlightTransitions) {
-      Map<String, String> attributes = Maps.newHashMap();
-      attributes.put(ConstraintAttribute.MESSAGE_TYPE.toString(),
-          MessageType.STATE_TRANSITION.toString());
-      attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(),
-          Integer.toString(maxInFlightTransitions));
-      attributes.put(ConstraintAttribute.TRANSITION.toString(), transition.toString());
-      attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify());
-      switch (scope.getType()) {
-      case CLUSTER:
-        // cluster is implicit
-        break;
-      case RESOURCE:
-        attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify());
-        break;
-      case PARTICIPANT:
-        attributes.put(ConstraintAttribute.INSTANCE.toString(), scope.getScopedId().stringify());
-        break;
-      default:
-        LOG.error("Unsupported scope for adding a transition constraint: " + scope);
-        return this;
-      }
-      ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build();
-      ClusterConstraints constraints = getConstraintsInstance(ConstraintType.MESSAGE_CONSTRAINT);
-      constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, transition), item);
-      return this;
-    }
-
-    /**
-     * Add a state upper bound constraint
-     * @param scope scope under which the constraint is valid
-     * @param stateModelDefId identifier of the state model that owns the state
-     * @param state the state to constrain
-     * @param upperBound maximum number of replicas per partition in the state
-     * @return Builder
-     */
-    public Builder addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        State state, int upperBound) {
-      return addStateUpperBoundConstraint(scope, stateModelDefId, state,
-          Integer.toString(upperBound));
-    }
-
-    /**
-     * Add a state upper bound constraint
-     * @param scope scope under which the constraint is valid
-     * @param stateModelDefId identifier of the state model that owns the state
-     * @param state the state to constrain
-     * @param dynamicUpperBound the upper bound of replicas per partition in the state, can be a
-     *          number, or the currently supported special bound values:<br />
-     *          "R" - Refers to the number of replicas specified during resource
-     *          creation. This allows having different replication factor for each
-     *          resource without having to create a different state machine. <br />
-     *          "N" - Refers to all nodes in the cluster. Useful for resources that need
-     *          to exist on all nodes. This way one can add/remove nodes without having
-     *          the change the bounds.
-     * @return Builder
-     */
-    public Builder addStateUpperBoundConstraint(Scope<?> scope, StateModelDefId stateModelDefId,
-        State state, String dynamicUpperBound) {
-      Map<String, String> attributes = Maps.newHashMap();
-      attributes.put(ConstraintAttribute.STATE.toString(), state.toString());
-      attributes.put(ConstraintAttribute.STATE_MODEL.toString(), stateModelDefId.stringify());
-      attributes.put(ConstraintAttribute.CONSTRAINT_VALUE.toString(), dynamicUpperBound);
-      switch (scope.getType()) {
-      case CLUSTER:
-        // cluster is implicit
-        break;
-      case RESOURCE:
-        attributes.put(ConstraintAttribute.RESOURCE.toString(), scope.getScopedId().stringify());
-        break;
-      default:
-        LOG.error("Unsupported scope for adding a state constraint: " + scope);
-        return this;
-      }
-      ConstraintItem item = new ConstraintItemBuilder().addConstraintAttributes(attributes).build();
-      ClusterConstraints constraints = getConstraintsInstance(ConstraintType.STATE_CONSTRAINT);
-      constraints.addConstraintItem(ConstraintId.from(scope, stateModelDefId, state), item);
-      return this;
-    }
-
-    /**
-     * Add a state model definition to the cluster
-     * @param stateModelDef state model definition of the cluster
-     * @return Builder
-     */
-    public Builder addStateModelDefinition(StateModelDefinition stateModelDef) {
-      _stateModelMap.put(stateModelDef.getStateModelDefId(), stateModelDef);
-      // add state constraints from the state model definition
-      for (State state : stateModelDef.getStatesPriorityList()) {
-        if (!stateModelDef.getNumParticipantsPerState(state).equals("-1")) {
-          addStateUpperBoundConstraint(Scope.cluster(_id), stateModelDef.getStateModelDefId(),
-              state, stateModelDef.getNumParticipantsPerState(state));
-        }
-      }
-      return this;
-    }
-
-    /**
-     * Add multiple state model definitions
-     * @param stateModelDefs collection of state model definitions for the cluster
-     * @return Builder
-     */
-    public Builder addStateModelDefinitions(Collection<StateModelDefinition> stateModelDefs) {
-      for (StateModelDefinition stateModelDef : stateModelDefs) {
-        addStateModelDefinition(stateModelDef);
-      }
-      return this;
-    }
-
-    /**
-     * Set the paused status of the cluster
-     * @param isPaused true if paused, false otherwise
-     * @return Builder
-     */
-    public Builder pausedStatus(boolean isPaused) {
-      _isPaused = isPaused;
-      return this;
-    }
-
-    /**
-     * Set the user configuration
-     * @param userConfig user-specified properties
-     * @return Builder
-     */
-    public Builder userConfig(UserConfig userConfig) {
-      _userConfig = userConfig;
-      return this;
-    }
-
-    /**
-     * Create the cluster configuration
-     * @return ClusterConfig
-     */
-    public ClusterConfig build() {
-      return new ClusterConfig(_id, _resourceMap, _participantMap, _constraintMap, _stateModelMap,
-          _userConfig, _isPaused);
-    }
-
-    /**
-     * Get a valid instance of ClusterConstraints for a type
-     * @param type the type
-     * @return ClusterConstraints
-     */
-    private ClusterConstraints getConstraintsInstance(ConstraintType type) {
-      ClusterConstraints constraints = _constraintMap.get(type);
-      if (constraints == null) {
-        constraints = new ClusterConstraints(type);
-        _constraintMap.put(type, constraints);
-      }
-      return constraints;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ClusterId.java b/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
deleted file mode 100644
index 4e8b382..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
+++ /dev/null
@@ -1,57 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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.
- */
-
-/**
- * Identifies a cluster
- */
-public class ClusterId extends Id {
-  @JsonProperty("id")
-  final private String _id;
-
-  /**
-   * Create a cluster id
-   * @param id string representation of the id
-   */
-  @JsonCreator
-  public ClusterId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete cluster id for a string name
-   * @param clusterId string cluster identifier
-   * @return ClusterId
-   */
-  public static ClusterId from(String clusterId) {
-    if (clusterId == null) {
-      return null;
-    }
-    return new ClusterId(clusterId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java b/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
deleted file mode 100644
index 7da6714..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
+++ /dev/null
@@ -1,78 +0,0 @@
-package org.apache.helix.api;
-
-import org.apache.helix.model.Transition;
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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.
- */
-
-/**
- * Identifies a constraint item on the cluster
- */
-public class ConstraintId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a constraint id
-   * @param constraintId string representing the constraint id
-   */
-  @JsonCreator
-  public ConstraintId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a constraint id from a string
-   * @param constraintId string representing the constraint id
-   * @return ConstraintId
-   */
-  public static ConstraintId from(String constraintId) {
-    return new ConstraintId(constraintId);
-  }
-
-  /**
-   * Get a state constraint id based on the state model definition and state
-   * @param scope the scope of the constraint
-   * @param stateModelDefId the state model
-   * @param state the constrained state
-   * @return ConstraintId
-   */
-  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId, State state) {
-    return new ConstraintId(scope + "|" + stateModelDefId + "|" + state);
-  }
-
-  /**
-   * Get a state constraint id based on the state model definition and transition
-   * @param scope the scope of the constraint
-   * @param stateModelDefId the state model
-   * @param transition the constrained transition
-   * @return ConstraintId
-   */
-  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId,
-      Transition transition) {
-    return new ConstraintId(scope + "|" + stateModelDefId + "|" + transition);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Controller.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Controller.java b/helix-core/src/main/java/org/apache/helix/api/Controller.java
index c47d603..33e85ed 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Controller.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Controller.java
@@ -19,6 +19,7 @@ package org.apache.helix.api;
  * under the License.
  */
 
+import org.apache.helix.api.id.ControllerId;
 import org.apache.helix.model.LiveInstance;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ControllerAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ControllerAccessor.java b/helix-core/src/main/java/org/apache/helix/api/ControllerAccessor.java
deleted file mode 100644
index b835a4c..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ControllerAccessor.java
+++ /dev/null
@@ -1,47 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.HelixDataAccessor;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.model.LiveInstance;
-
-public class ControllerAccessor {
-  private final HelixDataAccessor _accessor;
-  private final PropertyKey.Builder _keyBuilder;
-
-  public ControllerAccessor(HelixDataAccessor accessor) {
-    _accessor = accessor;
-    _keyBuilder = accessor.keyBuilder();
-  }
-
-  /**
-   * Read the leader controller if it is live
-   * @return Controller snapshot, or null
-   */
-  public Controller readLeader() {
-    LiveInstance leader = _accessor.getProperty(_keyBuilder.controllerLeader());
-    if (leader != null) {
-      ControllerId leaderId = ControllerId.from(leader.getId());
-      return new Controller(leaderId, leader, true);
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ControllerId.java b/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
deleted file mode 100644
index e55f37a..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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.
- */
-
-/**
- * Identifies Helix nodes that take on the CONTROLLER role
- */
-public class ControllerId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a controller id
-   * @param id string representation of a controller id
-   */
-  @JsonCreator
-  public ControllerId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a ControllerId from a string
-   * @param controllerId string representing the id
-   * @return ControllerId
-   */
-  public static ControllerId from(String controllerId) {
-    return new ControllerId(controllerId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Id.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Id.java b/helix-core/src/main/java/org/apache/helix/api/Id.java
deleted file mode 100644
index ce5d2e4..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/Id.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * 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.
- */
-
-/**
- * Generic identifier for Helix constructs
- */
-public abstract class Id implements Comparable<Id> {
-  public abstract String stringify();
-
-  @Override
-  public String toString() {
-    return stringify();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that instanceof Id) {
-      return this.stringify().equals(((Id) that).stringify());
-    } else if (that instanceof String) {
-      return this.stringify().equals(that);
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return this.stringify().hashCode();
-  }
-
-  @Override
-  public int compareTo(Id that) {
-    if (that instanceof Id) {
-      return this.stringify().compareTo(that.stringify());
-    }
-    return -1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/MessageId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/MessageId.java b/helix-core/src/main/java/org/apache/helix/api/MessageId.java
deleted file mode 100644
index c5d4002..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/MessageId.java
+++ /dev/null
@@ -1,54 +0,0 @@
-package org.apache.helix.api;
-
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/*
- * 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 MessageId extends Id {
-  @JsonProperty("id")
-  private final String _id;
-
-  /**
-   * Create a message id
-   * @param id string representation of a message id
-   */
-  @JsonCreator
-  public MessageId(@JsonProperty("id") String id) {
-    _id = id;
-  }
-
-  @Override
-  public String stringify() {
-    return _id;
-  }
-
-  /**
-   * Get a concrete message id
-   * @param messageId string message identifier
-   * @return MsgId
-   */
-  public static MessageId from(String messageId) {
-    if (messageId == null) {
-      return null;
-    }
-    return new MessageId(messageId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java b/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
deleted file mode 100644
index 675b144..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
+++ /dev/null
@@ -1,227 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.HelixProperty;
-import org.apache.helix.ZNRecord;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
-
-/*
- * 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.
- */
-
-/**
- * Generic configuration of Helix components prefixed with a namespace
- */
-public class NamespacedConfig extends ZNRecord {
-  private static final char PREFIX_CHAR = '!';
-  private final String _prefix;
-
-  /**
-   * Instantiate a NamespacedConfig. It is intended for use only by entities that can be identified
-   * @param scope scope object
-   */
-  public NamespacedConfig(Scope<?> scope, String prefix) {
-    super(scope.getScopedId().stringify());
-    _prefix = prefix + PREFIX_CHAR;
-  }
-
-  /**
-   * Instantiate a NamespacedConfig from an existing HelixProperty
-   * @param property property wrapping a configuration
-   */
-  public NamespacedConfig(HelixProperty property, String prefix) {
-    super(property.getRecord());
-    _prefix = prefix + PREFIX_CHAR;
-    filterNonPrefixedFields();
-  }
-
-  /**
-   * Instantiate a NamespacedConfig as a copy of another NamedspacedConfig
-   * @param config populated NamespacedConfig
-   */
-  public NamespacedConfig(NamespacedConfig config) {
-    super(config.getId());
-    _prefix = config.getPrefix() + PREFIX_CHAR;
-    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
-      setRawPayload(config.getRawPayload());
-      setPayloadSerializer(config.getPayloadSerializer());
-    }
-    super.setSimpleFields(config.getPrefixedSimpleFields());
-    super.setListFields(config.getPrefixedListFields());
-    super.setMapFields(config.getPrefixedMapFields());
-  }
-
-  @Override
-  public void setMapField(String k, Map<String, String> v) {
-    super.setMapField(_prefix + k, v);
-  }
-
-  @Override
-  public Map<String, String> getMapField(String k) {
-    return super.getMapField(_prefix + k);
-  }
-
-  @Override
-  public void setMapFields(Map<String, Map<String, String>> mapFields) {
-    for (String k : mapFields.keySet()) {
-      super.setMapField(_prefix + k, mapFields.get(k));
-    }
-  }
-
-  /**
-   * Returns an immutable map of map fields
-   */
-  @Override
-  public Map<String, Map<String, String>> getMapFields() {
-    return convertToPrefixlessMap(super.getMapFields(), _prefix);
-  }
-
-  @Override
-  public void setListField(String k, List<String> v) {
-    super.setListField(_prefix + k, v);
-  }
-
-  @Override
-  public List<String> getListField(String k) {
-    return super.getListField(_prefix + k);
-  }
-
-  @Override
-  public void setListFields(Map<String, List<String>> listFields) {
-    for (String k : listFields.keySet()) {
-      super.setListField(_prefix + k, listFields.get(k));
-    }
-  }
-
-  /**
-   * Returns an immutable map of list fields
-   */
-  @Override
-  public Map<String, List<String>> getListFields() {
-    return convertToPrefixlessMap(super.getListFields(), _prefix);
-  }
-
-  @Override
-  public void setSimpleField(String k, String v) {
-    super.setSimpleField(_prefix + k, v);
-  }
-
-  @Override
-  public String getSimpleField(String k) {
-    return super.getSimpleField(_prefix + k);
-  }
-
-  @Override
-  public void setSimpleFields(Map<String, String> simpleFields) {
-    for (String k : simpleFields.keySet()) {
-      super.setSimpleField(_prefix + k, simpleFields.get(k));
-    }
-  }
-
-  /**
-   * Returns an immutable map of simple fields
-   */
-  @Override
-  public Map<String, String> getSimpleFields() {
-    return convertToPrefixlessMap(super.getSimpleFields(), _prefix);
-  }
-
-  /**
-   * Get the prefix used to distinguish these config properties
-   * @return string prefix, not including the underscore
-   */
-  public String getPrefix() {
-    return _prefix.substring(0, _prefix.indexOf(PREFIX_CHAR));
-  }
-
-  /**
-   * Remove all fields from this config that are not prefixed
-   */
-  private void filterNonPrefixedFields() {
-    // filter out any configuration that isn't user-defined
-    Predicate<String> keyFilter = new Predicate<String>() {
-      @Override
-      public boolean apply(String key) {
-        return key.contains(_prefix);
-      }
-    };
-    super.setMapFields(Maps.filterKeys(super.getMapFields(), keyFilter));
-    super.setListFields(Maps.filterKeys(super.getListFields(), keyFilter));
-    super.setSimpleFields(Maps.filterKeys(super.getSimpleFields(), keyFilter));
-  }
-
-  /**
-   * Get all map fields with prefixed keys
-   * @return prefixed map fields
-   */
-  private Map<String, Map<String, String>> getPrefixedMapFields() {
-    return super.getMapFields();
-  }
-
-  /**
-   * Get all list fields with prefixed keys
-   * @return prefixed list fields
-   */
-  private Map<String, List<String>> getPrefixedListFields() {
-    return super.getListFields();
-  }
-
-  /**
-   * Get all simple fields with prefixed keys
-   * @return prefixed simple fields
-   */
-  private Map<String, String> getPrefixedSimpleFields() {
-    return super.getSimpleFields();
-  }
-
-  /**
-   * Add user configuration to an existing helix property.
-   * @param property the property to update
-   * @param config the user config
-   */
-  public static void addConfigToProperty(HelixProperty property, NamespacedConfig config) {
-    ZNRecord record = property.getRecord();
-    record.getMapFields().putAll(config.getPrefixedMapFields());
-    record.getListFields().putAll(config.getPrefixedListFields());
-    record.getSimpleFields().putAll(config.getPrefixedSimpleFields());
-    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
-      record.setPayloadSerializer(config.getPayloadSerializer());
-      record.setRawPayload(config.getRawPayload());
-    }
-  }
-
-  /**
-   * Get a copy of a map with the key prefix stripped. The resulting map is immutable
-   * @param rawMap map of key, value pairs where the key is prefixed
-   * @return map of key, value pairs where the key is not prefixed
-   */
-  private static <T> Map<String, T> convertToPrefixlessMap(Map<String, T> rawMap, String prefix) {
-    Map<String, T> convertedMap = new HashMap<String, T>();
-    for (String rawKey : rawMap.keySet()) {
-      String k = rawKey.substring(prefix.length());
-      convertedMap.put(k, rawMap.get(rawKey));
-    }
-    return ImmutableMap.copyOf(convertedMap);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/Participant.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Participant.java b/helix-core/src/main/java/org/apache/helix/api/Participant.java
index 8b02f0e..0e0de9d 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Participant.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Participant.java
@@ -22,6 +22,12 @@ package org.apache.helix.api;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.api.config.ParticipantConfig;
+import org.apache.helix.api.config.UserConfig;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;