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/10/16 01:51:38 UTC

[1/9] [HELIX-209] Backward compatible function naming in the model package

Updated Branches:
  refs/heads/helix-logical-model 2d5007734 -> 9f229c80c


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 6e7db19..d0a57f1 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
@@ -255,7 +255,7 @@ public class StatusUpdateUtil {
    */
   ZNRecord createMessageLogRecord(Message message) {
     ZNRecord result = new ZNRecord(getStatusUpdateRecordName(message));
-    String mapFieldKey = "MESSAGE " + message.getMsgId();
+    String mapFieldKey = "MESSAGE " + message.getMessageId();
     result.setMapField(mapFieldKey, new TreeMap<String, String>());
 
     // Store all the simple fields of the message in the new ZNRecord's map
@@ -291,7 +291,7 @@ public class StatusUpdateUtil {
     contentMap.put("Message state", message.getMsgState().toString());
     contentMap.put("AdditionalInfo", additionalInfo);
     contentMap.put("Class", classInfo.toString());
-    contentMap.put("MSG_ID", message.getMsgId().stringify());
+    contentMap.put("MSG_ID", message.getMessageId().stringify());
 
     DateFormat formatter = new SimpleDateFormat("yyyyMMdd-HHmmss.SSSSSS");
     String time = formatter.format(new Date());
@@ -305,8 +305,8 @@ public class StatusUpdateUtil {
 
   String getRecordIdForMessage(Message message) {
     if (message.getMsgType().equals(MessageType.STATE_TRANSITION)) {
-      return message.getPartitionId() + " Trans:" + message.getFromState().toString().charAt(0)
-          + "->" + message.getToState().toString().charAt(0) + "  " + UUID.randomUUID().toString();
+      return message.getPartitionId() + " Trans:" + message.getTypedFromState().toString().charAt(0)
+          + "->" + message.getTypedToState().toString().charAt(0) + "  " + UUID.randomUUID().toString();
     } else {
       return message.getMsgType() + " " + UUID.randomUUID().toString();
     }
@@ -376,16 +376,16 @@ public class StatusUpdateUtil {
     String instanceName = message.getTgtName();
     String statusUpdateSubPath = getStatusUpdateSubPath(message);
     String statusUpdateKey = getStatusUpdateKey(message);
-    SessionId sessionId = message.getExecutionSessionId();
+    SessionId sessionId = message.getTypedExecutionSessionId();
     if (sessionId == null) {
-      sessionId = message.getTgtSessionId();
+      sessionId = message.getTypedTgtSessionId();
     }
     if (sessionId == null) {
       sessionId = SessionId.from("*");
     }
 
     Builder keyBuilder = accessor.keyBuilder();
-    if (!_recordedMessages.containsKey(message.getMsgId().stringify())) {
+    if (!_recordedMessages.containsKey(message.getMessageId().stringify())) {
       // TODO instanceName of a controller might be any string
       if (instanceName.equalsIgnoreCase("Controller")) {
         accessor.updateProperty(
@@ -409,7 +409,7 @@ public class StatusUpdateUtil {
         accessor.updateProperty(propertyKey, new StatusUpdate(statusUpdateRecord));
 
       }
-      _recordedMessages.put(message.getMsgId().stringify(), message.getMsgId().stringify());
+      _recordedMessages.put(message.getMessageId().stringify(), message.getMessageId().stringify());
     }
 
     if (instanceName.equalsIgnoreCase("Controller")) {
@@ -439,7 +439,7 @@ public class StatusUpdateUtil {
     if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.toString())) {
       return message.getPartitionId().stringify();
     }
-    return message.getMsgId().stringify();
+    return message.getMessageId().stringify();
   }
 
   /**
@@ -455,9 +455,9 @@ public class StatusUpdateUtil {
 
   String getStatusUpdateRecordName(Message message) {
     if (message.getMsgType().equalsIgnoreCase(MessageType.STATE_TRANSITION.toString())) {
-      return message.getTgtSessionId() + "__" + message.getResourceId();
+      return message.getTypedTgtSessionId() + "__" + message.getResourceId();
     }
-    return message.getMsgId().stringify();
+    return message.getMessageId().stringify();
   }
 
   /**
@@ -473,9 +473,9 @@ public class StatusUpdateUtil {
     String instanceName = message.getTgtName();
     String statusUpdateSubPath = getStatusUpdateSubPath(message);
     String statusUpdateKey = getStatusUpdateKey(message);
-    SessionId sessionId = message.getExecutionSessionId();
+    SessionId sessionId = message.getTypedExecutionSessionId();
     if (sessionId == null) {
-      sessionId = message.getTgtSessionId();
+      sessionId = message.getTypedTgtSessionId();
     }
     if (sessionId == null) {
       sessionId = SessionId.from("*");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 74d1d53..a3b16e5 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskExecutor.java
@@ -49,7 +49,7 @@ public class TestHelixTaskExecutor {
     MessageId msgId = MessageId.from("TestMessageId");
     Message message = new Message(MessageType.TASK_REPLY, msgId);
 
-    message.setMsgId(msgId);
+    message.setMessageId(msgId);
     message.setSrcName("cm-instance-0");
     message.setTgtName("cm-instance-1");
     message.setTgtSessionId(SessionId.from("1234"));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 07de562..43b4407 100644
--- a/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
+++ b/helix-core/src/test/java/org/apache/helix/TestHelixTaskHandler.java
@@ -55,7 +55,7 @@ public class TestHelixTaskHandler {
     message.setFromState(State.from("Offline"));
     message.setToState(State.from("Slave"));
     message.setPartitionId(PartitionId.from("TestDB_0"));
-    message.setMsgId(MessageId.from("Some unique message id"));
+    message.setMessageId(MessageId.from("Some unique message id"));
     message.setResourceId(ResourceId.from("TestDB"));
     message.setTgtName("localhost");
     message.setStateModelDef(StateModelDefId.from("MasterSlave"));
@@ -94,7 +94,7 @@ public class TestHelixTaskHandler {
     message.setFromState(State.from("Offline"));
     message.setToState(State.from("Slave"));
     message.setPartitionId(PartitionId.from("TestDB_0"));
-    message.setMsgId(MessageId.from("Some unique message id"));
+    message.setMessageId(MessageId.from("Some unique message id"));
     message.setResourceId(ResourceId.from("TestDB"));
     message.setTgtName("localhost");
     message.setStateModelDef(StateModelDefId.from("MasterSlave"));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 e5b4452..b626cf2 100644
--- a/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/ZkUnitTestBase.java
@@ -181,7 +181,7 @@ public class ZkUnitTestBase {
     Builder keyBuilder = accessor.keyBuilder();
 
     IdealState idealState = accessor.getProperty(keyBuilder.idealState(resource));
-    for (PartitionId partitionId : idealState.getPartitionSet()) {
+    for (PartitionId partitionId : idealState.getPartitionIdSet()) {
       if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
         AssertJUnit.assertEquals(repl, idealState.getPreferenceList(partitionId).size());
       } else if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 f5e22d9..c478bbb 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
@@ -89,7 +89,7 @@ public class TestNewStages extends ZkUnitTestBase {
       ResourceId resourceId = ResourceId.from("TestDB0");
       Assert.assertTrue(curStateMap.containsKey(resourceId));
       CurrentState curState = curStateMap.get(resourceId);
-      Map<PartitionId, State> partitionStateMap = curState.getPartitionStateMap();
+      Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
       Assert.assertEquals(partitionStateMap.size(), p);
     }
 
@@ -176,10 +176,10 @@ public class TestNewStages extends ZkUnitTestBase {
    * @param assignment the assignment to verify
    */
   private void verifySemiAutoRebalance(Resource resource, ResourceAssignment assignment) {
-    Assert.assertEquals(assignment.getMappedPartitions().size(), resource.getSubUnitSet().size());
+    Assert.assertEquals(assignment.getMappedPartitionIds().size(), resource.getSubUnitSet().size());
     SemiAutoRebalancerContext context =
         resource.getRebalancerConfig().getRebalancerContext(SemiAutoRebalancerContext.class);
-    for (PartitionId partitionId : assignment.getMappedPartitions()) {
+    for (PartitionId partitionId : assignment.getMappedPartitionIds()) {
       List<ParticipantId> preferenceList = context.getPreferenceList(partitionId);
       Map<ParticipantId, State> replicaMap = assignment.getReplicaMap(partitionId);
       Assert.assertEquals(replicaMap.size(), preferenceList.size());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 113e1af..33eff27 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
@@ -96,7 +96,7 @@ public class TestMsgSelectionStage {
             messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE"));
 
     Assert.assertEquals(selectedMsg.size(), 1);
-    Assert.assertEquals(selectedMsg.get(0).getMsgId(), MessageId.from("msgId_1"));
+    Assert.assertEquals(selectedMsg.get(0).getMessageId(), MessageId.from("msgId_1"));
     System.out.println("END testMasterXfer at " + new Date(System.currentTimeMillis()));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 c9e0f53..892bfa7 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
@@ -110,8 +110,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
             PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 1, "Should output 1 message: OFFLINE-SLAVE for node0");
     Message message = messages.get(0);
-    Assert.assertEquals(message.getFromState().toString(), "OFFLINE");
-    Assert.assertEquals(message.getToState().toString(), "SLAVE");
+    Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE");
+    Assert.assertEquals(message.getTypedToState().toString(), "SLAVE");
     Assert.assertEquals(message.getTgtName(), "localhost_0");
 
     // round2: updates node0 currentState to SLAVE but keep the
@@ -265,8 +265,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
             PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 1, "Should output 1 message: OFFLINE-SLAVE for node0");
     Message message = messages.get(0);
-    Assert.assertEquals(message.getFromState().toString(), "OFFLINE");
-    Assert.assertEquals(message.getToState().toString(), "SLAVE");
+    Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE");
+    Assert.assertEquals(message.getTypedToState().toString(), "SLAVE");
     Assert.assertEquals(message.getTgtName(), "localhost_0");
 
     // round2: drop resource, but keep the
@@ -284,8 +284,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
         "Should output only 1 message: OFFLINE->DROPPED for localhost_1");
 
     message = messages.get(0);
-    Assert.assertEquals(message.getFromState().toString(), "SLAVE");
-    Assert.assertEquals(message.getToState().toString(), "OFFLINE");
+    Assert.assertEquals(message.getTypedFromState().toString(), "SLAVE");
+    Assert.assertEquals(message.getTypedToState().toString(), "OFFLINE");
     Assert.assertEquals(message.getTgtName(), "localhost_1");
 
     // round3: remove O->S for localhost_0, controller should now send O->DROPPED to
@@ -302,8 +302,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     Assert.assertEquals(messages.size(), 1,
         "Should output 1 message: OFFLINE->DROPPED for localhost_0");
     message = messages.get(0);
-    Assert.assertEquals(message.getFromState().toString(), "OFFLINE");
-    Assert.assertEquals(message.getToState().toString(), "DROPPED");
+    Assert.assertEquals(message.getTypedFromState().toString(), "OFFLINE");
+    Assert.assertEquals(message.getTypedToState().toString(), "DROPPED");
     Assert.assertEquals(message.getTgtName(), "localhost_0");
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
@@ -368,8 +368,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
             PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 1, "Should output 1 message: SLAVE-MASTER for node1");
     Message message = messages.get(0);
-    Assert.assertEquals(message.getFromState().toString(), "SLAVE");
-    Assert.assertEquals(message.getToState().toString(), "MASTER");
+    Assert.assertEquals(message.getTypedFromState().toString(), "SLAVE");
+    Assert.assertEquals(message.getTypedToState().toString(), "MASTER");
     Assert.assertEquals(message.getTgtName(), "localhost_1");
 
     // round2: updates node0 currentState to SLAVE but keep the

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 20fb0c9..ab0e4a6 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
@@ -195,7 +195,7 @@ public class TestResourceComputationStage extends BaseStageTest {
         .getRebalancerContext(RebalancerContext.class).getStateModelDefId(),
         currentState.getStateModelDefId());
     AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getSubUnitSet().size(), currentState
-        .getPartitionStateMap().size());
+        .getTypedPartitionStateMap().size());
     AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getSubUnit(
         PartitionId.from("testResourceOld_0")));
     AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getSubUnit(

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 e4943f8..813cd71 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
@@ -225,7 +225,7 @@ public class TestNewAutoRebalanceStrategy {
       ClusterId clusterId = ClusterId.from("clusterId");
       ClusterConfig.Builder clusterConfigBuilder =
           new ClusterConfig.Builder(clusterId).addStateModelDefinition(_stateModelDef);
-      for (State state : _stateModelDef.getStatesPriorityList()) {
+      for (State state : _stateModelDef.getTypedStatesPriorityList()) {
         clusterConfigBuilder.addStateUpperBoundConstraint(Scope.cluster(clusterId),
             _stateModelDef.getStateModelDefId(), state,
             _stateModelDef.getNumParticipantsPerState(state));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
index a29d604..e0e1544 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAddDropAlert.java
@@ -74,8 +74,8 @@ public class TestAddDropAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
index c181e2c..f429b5f 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestExpandAlert.java
@@ -76,8 +76,8 @@ public class TestExpandAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
index 003232f..6d33df0 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleAlert.java
@@ -81,8 +81,8 @@ public class TestSimpleAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
index bca492e..3eb31ed 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestSimpleWildcardAlert.java
@@ -80,8 +80,8 @@ public class TestSimpleWildcardAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
index 9d7ac9a..2661560 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestStalenessAlert.java
@@ -74,8 +74,8 @@ public class TestStalenessAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
index b4ed5d2..5265ebb 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestWildcardAlert.java
@@ -150,8 +150,8 @@ public class TestWildcardAlert extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       HelixManager manager = context.getManager();
       HelixDataAccessor accessor = manager.getHelixDataAccessor();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String instance = message.getTgtName();
 
       if (fromState.toString().equalsIgnoreCase("SLAVE")

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
index 0731475..c74d654 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalance.java
@@ -270,7 +270,7 @@ public class TestAutoRebalance extends ZkStandAloneCMTestBaseWithPropertyServerC
       String stateModelDefName = idealState.getStateModelDefId().stringify();
       StateModelDefinition stateModelDef =
           accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName));
-      State masterValue = stateModelDef.getStatesPriorityList().get(0);
+      State masterValue = stateModelDef.getTypedStatesPriorityList().get(0);
       int replicas = Integer.parseInt(idealState.getReplicas());
 
       String instanceGroupTag = idealState.getInstanceGroupTag();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
index 19535ff..7228cef 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAutoRebalancePartitionLimit.java
@@ -102,11 +102,11 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
         Assert.assertTrue(result);
         ExternalView ev =
             manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
-        System.out.println(ev.getPartitionStringSet().size());
+        System.out.println(ev.getPartitionSet().size());
         if (i < 3) {
-          Assert.assertEquals(ev.getPartitionStringSet().size(), 25 * (i + 1));
+          Assert.assertEquals(ev.getPartitionSet().size(), 25 * (i + 1));
         } else {
-          Assert.assertEquals(ev.getPartitionStringSet().size(), 100);
+          Assert.assertEquals(ev.getPartitionSet().size(), 100);
         }
       }
     }
@@ -136,7 +136,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     ExternalView ev =
         manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
-    Assert.assertEquals(ev.getPartitionStringSet().size(), 100);
+    Assert.assertEquals(ev.getPartitionSet().size(), 100);
 
     instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + 1);
     _startCMResultMap.get(instanceName)._manager.disconnect();
@@ -149,7 +149,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
             CLUSTER_NAME, TEST_DB));
     Assert.assertTrue(result);
     ev = manager.getHelixDataAccessor().getProperty(accessor.keyBuilder().externalView(TEST_DB));
-    Assert.assertEquals(ev.getPartitionStringSet().size(), 75);
+    Assert.assertEquals(ev.getPartitionSet().size(), 75);
 
     // add 2 nodes
     for (int i = 0; i < 2; i++) {
@@ -233,7 +233,7 @@ public class TestAutoRebalancePartitionLimit extends ZkStandAloneCMTestBaseWithP
       String stateModelDefName = idealState.getStateModelDefId().stringify();
       StateModelDefinition stateModelDef =
           accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName));
-      State masterValue = stateModelDef.getStatesPriorityList().get(0);
+      State masterValue = stateModelDef.getTypedStatesPriorityList().get(0);
       Map<String, LiveInstance> liveInstanceMap =
           accessor.getChildValuesMap(keyBuilder.liveInstances());
       int replicas = Integer.parseInt(idealState.getReplicas());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
index eb3d83a..49c5576 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCleanupExternalView.java
@@ -98,10 +98,10 @@ public class TestCleanupExternalView extends ZkUnitTestBase {
 
     // System.out.println("remove current-state");
     LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance("localhost_12918"));
-    accessor.removeProperty(keyBuilder.currentState("localhost_12918", liveInstance.getSessionId()
+    accessor.removeProperty(keyBuilder.currentState("localhost_12918", liveInstance.getTypedSessionId()
         .stringify(), "TestDB0"));
     liveInstance = accessor.getProperty(keyBuilder.liveInstance("localhost_12919"));
-    accessor.removeProperty(keyBuilder.currentState("localhost_12919", liveInstance.getSessionId()
+    accessor.removeProperty(keyBuilder.currentState("localhost_12919", liveInstance.getTypedSessionId()
         .stringify(), "TestDB0"));
 
     // re-enable controller shall remove orphan external-view

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 09a0b1c..1b7c30c 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
@@ -78,7 +78,7 @@ public class TestCustomizedIdealStateRebalancer extends
       for (PartitionId partitionId : context.getPartitionSet()) {
         int nodeIndex = i % liveParticipants.size();
         Map<ParticipantId, State> replicaMap = new HashMap<ParticipantId, State>();
-        replicaMap.put(liveParticipants.get(nodeIndex), stateModelDef.getStatesPriorityList()
+        replicaMap.put(liveParticipants.get(nodeIndex), stateModelDef.getTypedStatesPriorityList()
             .get(0));
         resourceMapping.addReplicaMap(partitionId, replicaMap);
         i++;
@@ -113,12 +113,12 @@ public class TestCustomizedIdealStateRebalancer extends
         new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2));
-    Assert.assertEquals(ev.getPartitionStringSet().size(), 60);
-    for (String partition : ev.getPartitionStringSet()) {
+    Assert.assertEquals(ev.getPartitionSet().size(), 60);
+    for (String partition : ev.getPartitionSet()) {
       Assert.assertEquals(ev.getStateMap(partition).size(), 1);
     }
     IdealState is = accessor.getProperty(keyBuilder.idealState(db2));
-    for (PartitionId partition : is.getPartitionSet()) {
+    for (PartitionId partition : is.getPartitionIdSet()) {
       Assert.assertEquals(is.getPreferenceList(partition).size(), 0);
       Assert.assertEquals(is.getParticipantStateMap(partition).size(), 0);
     }
@@ -148,7 +148,7 @@ public class TestCustomizedIdealStateRebalancer extends
         String stateModelDefName = idealState.getStateModelDefId().stringify();
         StateModelDefinition stateModelDef =
             accessor.getProperty(keyBuilder.stateModelDef(stateModelDefName));
-        State masterValue = stateModelDef.getStatesPriorityList().get(0);
+        State masterValue = stateModelDef.getTypedStatesPriorityList().get(0);
         int replicas = Integer.parseInt(idealState.getReplicas());
         String instanceGroupTag = idealState.getInstanceGroupTag();
         int instances = 0;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
index 7f9d461..794a790 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDrop.java
@@ -347,7 +347,7 @@ public class TestDrop extends ZkIntegrationTestBase {
     ZKHelixDataAccessor accessor = new ZKHelixDataAccessor(clusterName, baseAccessor);
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView extView = accessor.getProperty(keyBuilder.externalView("schemata"));
-    Assert.assertEquals(extView.getPartitionStringSet().size(), 0,
+    Assert.assertEquals(extView.getPartitionSet().size(), 0,
         "schemata externalView should be empty but was \"" + extView + "\"");
 
     // clean up

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 ade7ca1..616f63b 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
@@ -52,8 +52,8 @@ public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
 
       String instance = message.getTgtName();
       PartitionId partitionId = message.getPartitionId();
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       if (instance.equals("localhost_12919") && partitionId.equals(PartitionId.from("TestDB0_0"))) {
         if (fromState.equals("SLAVE") && toState.equals("OFFLINE")) {
           slaveToOfflineCnt++;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
index 69b45da..4484386 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixInstanceTag.java
@@ -56,7 +56,7 @@ public class TestHelixInstanceTag extends ZkStandAloneCMTestBase {
 
     ExternalView ev = accessor.getProperty(accessor.keyBuilder().externalView(DB2));
     Set<String> hosts = new HashSet<String>();
-    for (String p : ev.getPartitionStringSet()) {
+    for (String p : ev.getPartitionSet()) {
       for (String hostName : ev.getStateMap(p).keySet()) {
         InstanceConfig config =
             accessor.getProperty(accessor.keyBuilder().instanceConfig(hostName));
@@ -85,7 +85,7 @@ public class TestHelixInstanceTag extends ZkStandAloneCMTestBase {
 
     ev = accessor.getProperty(accessor.keyBuilder().externalView(DB3));
     hosts = new HashSet<String>();
-    for (String p : ev.getPartitionStringSet()) {
+    for (String p : ev.getPartitionSet()) {
       for (String hostName : ev.getStateMap(p).keySet()) {
         InstanceConfig config =
             accessor.getProperty(accessor.keyBuilder().instanceConfig(hostName));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 3613cb8..268d6d0 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
@@ -53,8 +53,8 @@ public class TestMessagePartitionStateMismatch extends ZkStandAloneCMTestBase {
         accessor.getChildValuesMap(accessor.keyBuilder().liveInstances());
 
     for (String instanceName : liveinstanceMap.keySet()) {
-      String sessionid = liveinstanceMap.get(instanceName).getSessionId().stringify();
-      for (String partition : ev.getPartitionStringSet()) {
+      String sessionid = liveinstanceMap.get(instanceName).getTypedSessionId().stringify();
+      for (String partition : ev.getPartitionSet()) {
         if (ev.getStateMap(partition).containsKey(instanceName)) {
           MessageId uuid = MessageId.from(UUID.randomUUID().toString());
           Message message = new Message(MessageType.STATE_TRANSITION, uuid);
@@ -85,7 +85,7 @@ public class TestMessagePartitionStateMismatch extends ZkStandAloneCMTestBase {
             message.setStateModelFactoryName("DEFAULT");
           }
           accessor.setProperty(
-              accessor.keyBuilder().message(instanceName, message.getMsgId().stringify()), message);
+              accessor.keyBuilder().message(instanceName, message.getMessageId().stringify()), message);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 a3d6237..9c6b4b7 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
@@ -69,7 +69,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
         HelixTaskResult result = new HelixTaskResult();
         result.setSuccess(true);
         Thread.sleep(1000);
-        System.out.println("TestMessagingHandler " + _message.getMsgId());
+        System.out.println("TestMessagingHandler " + _message.getMessageId());
         _processedMsgIds.add(_message.getRecord().getSimpleField("TestMessagingPara"));
         result.getTaskResultMap().put("ReplyMessage", "TestReplyMessage");
         return result;
@@ -94,7 +94,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
     msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
@@ -189,7 +189,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
 
     msg.setTgtSessionId(SessionId.from("*"));
@@ -253,7 +253,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
 
     msg.setTgtSessionId(SessionId.from("*"));
@@ -296,7 +296,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
 
     msg.setTgtSessionId(SessionId.from("*"));
@@ -365,7 +365,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
 
     msg.setTgtSessionId(SessionId.from("*"));
@@ -401,7 +401,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     }
     MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(MessageType.CONTROLLER_MSG, msgId);
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     msg.setSrcName(hostSrc);
 
     msg.setTgtSessionId(SessionId.from("*"));
@@ -425,7 +425,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(callback1.getMessageReplied().size() == 1);
 
     msgId = MessageId.from(UUID.randomUUID().toString());
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     cr.setPartition("TestDB_17");
     AsyncCallback callback2 = new MockAsyncCallback();
     messagesSent =
@@ -438,7 +438,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     AssertJUnit.assertTrue(callback2.getMessageReplied().size() == 1);
 
     msgId = MessageId.from(UUID.randomUUID().toString());
-    msg.setMsgId(msgId);
+    msg.setMessageId(msgId);
     cr.setPartitionState("SLAVE");
     AsyncCallback callback3 = new MockAsyncCallback();
     messagesSent =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
index d6ee8c6..9479cff 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetPartitionState.java
@@ -53,8 +53,8 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
     public void doTransition(Message message, NotificationContext context) {
       // System.err.println("doReset() invoked");
       super.doTransition(message, context);
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       if (fromState.toString().equals("ERROR") && toState.toString().equals("OFFLINE")) {
         _errToOfflineInvoked++;
       }
@@ -190,7 +190,7 @@ public class TestResetPartitionState extends ZkIntegrationTestBase {
     Builder keyBuilder = accessor.keyBuilder();
 
     LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instance));
-    accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getSessionId()
+    accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getTypedSessionId()
         .stringify(), resource, partition));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8af9f14..8558b18 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
@@ -117,7 +117,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
         HelixTaskResult result = new HelixTaskResult();
         result.setSuccess(true);
         // String tgtName = _message.getTgtName();
-        String messageId = _message.getMsgId().stringify();
+        String messageId = _message.getMessageId().stringify();
         String partitionId = _message.getPartitionId().stringify();
 
         result.getTaskResultMap().put("Message", messageId);
@@ -189,7 +189,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     Builder keyBuilder = helixDataAccessor.keyBuilder();
     helixDataAccessor.createProperty(
-        keyBuilder.controllerMessage(schedulerMessage.getMsgId().stringify()), schedulerMessage);
+        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify()), schedulerMessage);
 
     for (int i = 0; i < 30; i++) {
       Thread.sleep(2000);
@@ -201,7 +201,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     Assert.assertEquals(_PARTITIONS, _factory._results.size());
     PropertyKey controllerTaskStatus =
         keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMsgId().stringify());
+            .getMessageId().stringify());
 
     int messageResultCount = 0;
     for (int i = 0; i < 10; i++) {
@@ -417,19 +417,19 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     Builder keyBuilder = helixDataAccessor.keyBuilder();
     PropertyKey controllerMessageKey =
-        keyBuilder.controllerMessage(schedulerMessage.getMsgId().stringify());
+        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify());
     helixDataAccessor.setProperty(controllerMessageKey, schedulerMessage);
 
     Thread.sleep(3000);
 
     Assert.assertEquals(0, factory._results.size());
 
-    waitMessageUpdate("SentMessageCount", schedulerMessage.getMsgId().stringify(),
+    waitMessageUpdate("SentMessageCount", schedulerMessage.getMessageId().stringify(),
         helixDataAccessor);
     PropertyKey controllerTaskStatus =
         keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMsgId().stringify());
-    waitMessageUpdate("SentMessageCount", schedulerMessage.getMsgId().stringify(),
+            .getMessageId().stringify());
+    waitMessageUpdate("SentMessageCount", schedulerMessage.getMessageId().stringify(),
         helixDataAccessor);
     ZNRecord statusUpdate = helixDataAccessor.getProperty(controllerTaskStatus).getRecord();
     Assert.assertTrue(statusUpdate.getMapField("SentMessageCount").get("MessageCount").equals("0"));
@@ -518,7 +518,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
       sw = new StringWriter();
       mapper.writeValue(sw, cr);
-      schedulerMessage.setMsgId(MessageId.from(UUID.randomUUID().toString()));
+      schedulerMessage.setMessageId(MessageId.from(UUID.randomUUID().toString()));
       crString = sw.toString();
       schedulerMessage.getRecord().setSimpleField("Criteria", crString);
       manager.getMessagingService().sendAndWait(cr2, schedulerMessage, callback, -1);
@@ -647,7 +647,7 @@ public class TestSchedulerMessage extends ZkStandAloneCMTestBaseWithPropertyServ
 
       sw = new StringWriter();
       mapper.writeValue(sw, cr);
-      schedulerMessage.setMsgId(MessageId.from(UUID.randomUUID().toString()));
+      schedulerMessage.setMessageId(MessageId.from(UUID.randomUUID().toString()));
 
       // need to use a different name for scheduler_task_queue task resource
       schedulerMessage.getRecord().setSimpleField(

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 aa91589..b887fe7 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
@@ -89,7 +89,7 @@ public class TestSchedulerMsgContraints extends ZkStandAloneCMTestBaseWithProper
         _latch.await();
         HelixTaskResult result = new HelixTaskResult();
         result.setSuccess(true);
-        result.getTaskResultMap().put("Message", _message.getMsgId().stringify());
+        result.getTaskResultMap().put("Message", _message.getMessageId().stringify());
         String destName = _message.getTgtName();
         synchronized (_results) {
           if (!_results.containsKey(_message.getPartitionId().stringify())) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 d922843..bbaa18d 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
@@ -66,7 +66,7 @@ public class TestSchedulerMsgUsingQueue extends ZkStandAloneCMTestBaseWithProper
         HelixTaskResult result = new HelixTaskResult();
         result.setSuccess(true);
         // String tgtName = _message.getTgtName();
-        String messageId = _message.getMsgId().stringify();
+        String messageId = _message.getMessageId().stringify();
         String partitionId = _message.getPartitionId().stringify();
 
         result.getTaskResultMap().put("Message", messageId);
@@ -140,7 +140,7 @@ public class TestSchedulerMsgUsingQueue extends ZkStandAloneCMTestBaseWithProper
     HelixDataAccessor helixDataAccessor = manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = helixDataAccessor.keyBuilder();
     helixDataAccessor.createProperty(
-        keyBuilder.controllerMessage(schedulerMessage.getMsgId().stringify()), schedulerMessage);
+        keyBuilder.controllerMessage(schedulerMessage.getMessageId().stringify()), schedulerMessage);
 
     for (int i = 0; i < 30; i++) {
       Thread.sleep(2000);
@@ -152,7 +152,7 @@ public class TestSchedulerMsgUsingQueue extends ZkStandAloneCMTestBaseWithProper
     Assert.assertEquals(_PARTITIONS, factory._results.size());
     PropertyKey controllerTaskStatus =
         keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), schedulerMessage
-            .getMsgId().stringify());
+            .getMessageId().stringify());
 
     int messageResultCount = 0;
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 451972a..aff40b3 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
@@ -179,7 +179,7 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
       String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
       SleepStateModelFactory factory = new SleepStateModelFactory(1000);
       factories.put(instanceName, factory);
-      for (PartitionId p : idealState.getPartitionSet()) {
+      for (PartitionId p : idealState.getPartitionIdSet()) {
         if (idealState.getPreferenceList(p).get(0).equals(ParticipantId.from(instanceName))) {
           factory.addPartition(p.stringify());
         }
@@ -201,7 +201,7 @@ public class TestStateTransitionTimeout extends ZkStandAloneCMTestBase {
 
     Builder kb = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(kb.externalView(TEST_DB));
-    for (PartitionId p : idealState.getPartitionSet()) {
+    for (PartitionId p : idealState.getPartitionIdSet()) {
       ParticipantId idealMaster = idealState.getPreferenceList(p).get(0);
       Assert.assertTrue(ev.getStateMap(p).get(idealMaster).equals(State.from("ERROR")));
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
index f59ca1d..4b92670 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestStatusUpdate.java
@@ -49,7 +49,7 @@ public class TestStatusUpdate extends ZkStandAloneCMTestBase {
 
     for (ExternalView extView : extViews) {
       String resourceName = extView.getResourceName();
-      Set<String> partitionSet = extView.getPartitionStringSet();
+      Set<String> partitionSet = extView.getPartitionSet();
       for (String partition : partitionSet) {
         Map<String, String> stateMap = extView.getStateMap(partition);
         for (String instance : stateMap.keySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
index e19a3d5..c099232 100644
--- a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkClusterManager.java
@@ -188,7 +188,7 @@ public class TestZkClusterManager extends ZkUnitTestBase {
     Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5);
-    Assert.assertFalse(liveInstance.getSessionId().stringify().equals("value"));
+    Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value"));
     Assert.assertFalse(liveInstance.getLiveInstance().equals("value"));
 
     // //////////////////////////////////
@@ -204,9 +204,9 @@ public class TestZkClusterManager extends ZkUnitTestBase {
     Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5);
-    Assert.assertFalse(liveInstance.getSessionId().stringify().equals("value"));
+    Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value"));
     Assert.assertFalse(liveInstance.getLiveInstance().equals("value"));
-    String sessionId = liveInstance.getSessionId().stringify();
+    String sessionId = liveInstance.getTypedSessionId().stringify();
 
     ZkTestHelper.expireSession(manager2.getZkClient());
     Thread.sleep(1000);
@@ -215,9 +215,9 @@ public class TestZkClusterManager extends ZkUnitTestBase {
     Assert.assertTrue(liveInstance.getRecord().getListFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getMapFields().size() == 1);
     Assert.assertTrue(liveInstance.getRecord().getSimpleFields().size() == 5);
-    Assert.assertFalse(liveInstance.getSessionId().stringify().equals("value"));
+    Assert.assertFalse(liveInstance.getTypedSessionId().stringify().equals("value"));
     Assert.assertFalse(liveInstance.getLiveInstance().equals("value"));
-    Assert.assertFalse(sessionId.equals(liveInstance.getSessionId().stringify()));
+    Assert.assertFalse(sessionId.equals(liveInstance.getTypedSessionId().stringify()));
 
     System.out.println("END " + className + ".testLiveInstanceInfoProvider() at "
         + new Date(System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8183e2c..da686fe 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
@@ -57,7 +57,7 @@ public class TestAsyncCallbackSvc {
     @Override
     public void onReplyMessage(Message message) {
       // TODO Auto-generated method stub
-      _repliedMessageId.add(message.getMsgId());
+      _repliedMessageId.add(message.getMessageId());
     }
 
   }
@@ -75,14 +75,14 @@ public class TestAsyncCallbackSvc {
     try {
       MessageHandler aHandler = svc.createHandler(msg, changeContext);
     } catch (HelixException e) {
-      AssertJUnit.assertTrue(e.getMessage().indexOf(msg.getMsgId().stringify()) != -1);
+      AssertJUnit.assertTrue(e.getMessage().indexOf(msg.getMessageId().stringify()) != -1);
     }
     Message msg2 = new Message("RandomType", MessageId.from(UUID.randomUUID().toString()));
     msg2.setTgtSessionId(SessionId.from(manager.getSessionId()));
     try {
       MessageHandler aHandler = svc.createHandler(msg2, changeContext);
     } catch (HelixException e) {
-      AssertJUnit.assertTrue(e.getMessage().indexOf(msg2.getMsgId().stringify()) != -1);
+      AssertJUnit.assertTrue(e.getMessage().indexOf(msg2.getMessageId().stringify()) != -1);
     }
     Message msg3 = new Message(svc.getMessageType(), MessageId.from(UUID.randomUUID().toString()));
     msg3.setTgtSessionId(SessionId.from(manager.getSessionId()));
@@ -90,7 +90,7 @@ public class TestAsyncCallbackSvc {
     try {
       MessageHandler aHandler = svc.createHandler(msg3, changeContext);
     } catch (HelixException e) {
-      AssertJUnit.assertTrue(e.getMessage().indexOf(msg3.getMsgId().stringify()) != -1);
+      AssertJUnit.assertTrue(e.getMessage().indexOf(msg3.getMessageId().stringify()) != -1);
     }
 
     TestAsyncCallback callback = new TestAsyncCallback();
@@ -111,6 +111,6 @@ public class TestAsyncCallbackSvc {
     aHandler.handleMessage();
 
     AssertJUnit.assertTrue(callback.isDone());
-    AssertJUnit.assertTrue(callback._repliedMessageId.contains(msg.getMsgId()));
+    AssertJUnit.assertTrue(callback._repliedMessageId.contains(msg.getMessageId()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 3e6ff47..43818f7 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
@@ -58,7 +58,7 @@ public class TestHelixTaskExecutor {
       @Override
       public HelixTaskResult handleMessage() throws InterruptedException {
         HelixTaskResult result = new HelixTaskResult();
-        _processedMsgIds.put(_message.getMsgId().stringify(), _message.getMsgId().stringify());
+        _processedMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify());
         Thread.sleep(100);
         result.setSuccess(true);
         return result;
@@ -125,14 +125,14 @@ public class TestHelixTaskExecutor {
         if (_message.getRecord().getSimpleFields().containsKey("Cancelcount")) {
           sleepTimes = 10;
         }
-        _processingMsgIds.put(_message.getMsgId().stringify(), _message.getMsgId().stringify());
+        _processingMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify());
         try {
           for (int i = 0; i < sleepTimes; i++) {
             Thread.sleep(100);
           }
         } catch (InterruptedException e) {
           _interrupted = true;
-          _timedOutMsgIds.put(_message.getMsgId().stringify(), "");
+          _timedOutMsgIds.put(_message.getMessageId().stringify(), "");
           result.setInterrupted(true);
           if (!_message.getRecord().getSimpleFields().containsKey("Cancelcount")) {
             _message.getRecord().setSimpleField("Cancelcount", "1");
@@ -142,7 +142,7 @@ public class TestHelixTaskExecutor {
           }
           throw e;
         }
-        _processedMsgIds.put(_message.getMsgId().stringify(), _message.getMsgId().stringify());
+        _processedMsgIds.put(_message.getMessageId().stringify(), _message.getMessageId().stringify());
         result.setSuccess(true);
         return result;
       }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 61e727c..4ddaac4 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
@@ -65,7 +65,7 @@ public class MockController {
       String partitionKey, int partitionId) throws InterruptedException, JsonGenerationException,
       JsonMappingException, IOException {
     Message message = new Message(MessageType.STATE_TRANSITION, msgId);
-    message.setMsgId(msgId);
+    message.setMessageId(msgId);
     message.setSrcName(srcName);
     message.setTgtName(instanceName);
     message.setMsgState(MessageState.NEW);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 0a744c2..a2e1414 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
@@ -48,8 +48,8 @@ public class ErrTransition extends MockTransition {
 
   @Override
   public void doTransition(Message message, NotificationContext context) {
-    State fromState = message.getFromState();
-    State toState = message.getToState();
+    State fromState = message.getTypedFromState();
+    State toState = message.getTypedToState();
     PartitionId partition = message.getPartitionId();
 
     String key = (fromState + "-" + toState).toUpperCase();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 223ae53..a2c7601 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
@@ -654,7 +654,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     IdealState dbIs = accessor.getProperty(accessor.keyBuilder().idealState("db_11"));
     Set<ParticipantId> hosts = new HashSet<ParticipantId>();
-    for (PartitionId p : dbIs.getPartitionSet()) {
+    for (PartitionId p : dbIs.getPartitionIdSet()) {
       for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) {
         InstanceConfig config =
             accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify()));
@@ -676,7 +676,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
 
     dbIs = accessor.getProperty(accessor.keyBuilder().idealState("db_11"));
     hosts = new HashSet<ParticipantId>();
-    for (PartitionId p : dbIs.getPartitionSet()) {
+    for (PartitionId p : dbIs.getPartitionIdSet()) {
       for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) {
         InstanceConfig config =
             accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify()));
@@ -704,7 +704,7 @@ public class TestHelixAdminCli extends ZkIntegrationTestBase {
     ClusterSetup.processCommandLineArgs(command.split("\\s+"));
     dbIs = accessor.getProperty(accessor.keyBuilder().idealState("db_11"));
     hosts = new HashSet<ParticipantId>();
-    for (PartitionId p : dbIs.getPartitionSet()) {
+    for (PartitionId p : dbIs.getPartitionIdSet()) {
       for (ParticipantId participantId : dbIs.getParticipantStateMap(p).keySet()) {
         InstanceConfig config =
             accessor.getProperty(accessor.keyBuilder().instanceConfig(participantId.stringify()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
----------------------------------------------------------------------
diff --git a/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
index b278c39..71d1412 100644
--- a/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
+++ b/helix-examples/src/main/java/org/apache/helix/examples/MasterSlaveStateModelFactory.java
@@ -74,8 +74,8 @@ public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel>
 
     public void onBecomeSlaveFromOffline(Message message, NotificationContext context) {
 
-      System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to "
-          + message.getToState() + " for " + partitionName);
+      System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to "
+          + message.getTypedToState() + " for " + partitionName);
       sleep();
     }
 
@@ -88,22 +88,22 @@ public class MasterSlaveStateModelFactory extends StateModelFactory<StateModel>
     }
 
     public void onBecomeSlaveFromMaster(Message message, NotificationContext context) {
-      System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to "
-          + message.getToState() + " for " + partitionName);
+      System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to "
+          + message.getTypedToState() + " for " + partitionName);
       sleep();
 
     }
 
     public void onBecomeMasterFromSlave(Message message, NotificationContext context) {
-      System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to "
-          + message.getToState() + " for " + partitionName);
+      System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to "
+          + message.getTypedToState() + " for " + partitionName);
       sleep();
 
     }
 
     public void onBecomeOfflineFromSlave(Message message, NotificationContext context) {
-      System.out.println(_instanceName + " transitioning from " + message.getFromState() + " to "
-          + message.getToState() + " for " + partitionName);
+      System.out.println(_instanceName + " transitioning from " + message.getTypedFromState() + " to "
+          + message.getTypedToState() + " for " + partitionName);
       sleep();
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 11d4fa4..2f3a677 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
@@ -212,7 +212,7 @@ public class Quickstart {
   private static void printState(String msg) {
     System.out.println("CLUSTER STATE: " + msg);
     ExternalView resourceExternalView = admin.getResourceExternalView(CLUSTER_NAME, RESOURCE_NAME);
-    TreeSet<String> sortedSet = new TreeSet<String>(resourceExternalView.getPartitionStringSet());
+    TreeSet<String> sortedSet = new TreeSet<String>(resourceExternalView.getPartitionSet());
     StringBuilder sb = new StringBuilder("\t\t");
     for (int i = 0; i < NUM_NODES; i++) {
       sb.append(INSTANCE_CONFIG_LIST.get(i).getInstanceName()).append("\t");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/recipes/distributed-lock-manager/src/main/java/org/apache/helix/lockmanager/LockManagerDemo.java
----------------------------------------------------------------------
diff --git a/recipes/distributed-lock-manager/src/main/java/org/apache/helix/lockmanager/LockManagerDemo.java b/recipes/distributed-lock-manager/src/main/java/org/apache/helix/lockmanager/LockManagerDemo.java
index da913eb..b6c54db 100644
--- a/recipes/distributed-lock-manager/src/main/java/org/apache/helix/lockmanager/LockManagerDemo.java
+++ b/recipes/distributed-lock-manager/src/main/java/org/apache/helix/lockmanager/LockManagerDemo.java
@@ -115,7 +115,7 @@ public class LockManagerDemo {
   private static void printStatus(HelixAdmin admin, String cluster, String resource) {
     ExternalView externalView = admin.getResourceExternalView(cluster, resource);
     // System.out.println(externalView);
-    TreeSet<String> treeSet = new TreeSet<String>(externalView.getPartitionStringSet());
+    TreeSet<String> treeSet = new TreeSet<String>(externalView.getPartitionSet());
     System.out.println("lockName" + "\t" + "acquired By");
     System.out.println("======================================");
     for (String lockName : treeSet) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStoreStateModel.java
----------------------------------------------------------------------
diff --git a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStoreStateModel.java b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStoreStateModel.java
index 56c07f3..cce0277 100644
--- a/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStoreStateModel.java
+++ b/recipes/rsync-replicated-file-system/src/main/java/org/apache/helix/filestore/FileStoreStateModel.java
@@ -123,12 +123,12 @@ public class FileStoreStateModel extends StateModel {
   @Transition(from = "OFFLINE", to = "SLAVE")
   public void onBecomeSlaveFromOffline(Message message, NotificationContext context)
       throws Exception {
-    System.out.println(_serverId + " transitioning from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    System.out.println(_serverId + " transitioning from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
 
     replicator.start();
-    System.out.println(_serverId + " transitioned from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    System.out.println(_serverId + " transitioned from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
   }
 
   /**
@@ -142,8 +142,8 @@ public class FileStoreStateModel extends StateModel {
   public void onBecomeMasterFromSlave(final Message message, NotificationContext context)
       throws Exception {
     replicator.stop();
-    System.out.println(_serverId + " transitioning from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    System.out.println(_serverId + " transitioning from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
     ZkHelixPropertyStore<ZNRecord> helixPropertyStore =
         context.getManager().getHelixPropertyStore();
     String checkpointDirPath = instanceConfig.getRecord().getSimpleField("check_point_dir");
@@ -168,8 +168,8 @@ public class FileStoreStateModel extends StateModel {
     long now = System.currentTimeMillis();
     service = new FileSystemWatchService(fileStoreDir, now, generator);
     service.start();
-    System.out.println(_serverId + " transitioned from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    System.out.println(_serverId + " transitioned from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
   }
 
   /**
@@ -183,16 +183,16 @@ public class FileStoreStateModel extends StateModel {
   public void onBecomeSlaveFromMaster(Message message, NotificationContext context)
       throws Exception {
     service.stop();
-    LOG.info(_serverId + " transitioning from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    LOG.info(_serverId + " transitioning from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
     replicator.start();
   }
 
   @Transition(from = "SLAVE", to = "OFFLINE")
   public void onBecomeOfflineFromSlave(Message message, NotificationContext context) {
     replicator.stop();
-    LOG.info(_serverId + " transitioning from " + message.getFromState() + " to "
-        + message.getToState() + " for " + _partition);
+    LOG.info(_serverId + " transitioning from " + message.getTypedFromState() + " to "
+        + message.getTypedToState() + " for " + _partition);
   }
 
   public void onBecomeDroppedFromOffline(Message message, NotificationContext context) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/Task.java
----------------------------------------------------------------------
diff --git a/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/Task.java b/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/Task.java
index 6030186..0cc8bba 100644
--- a/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/Task.java
+++ b/recipes/task-execution/src/main/java/org/apache/helix/taskexecution/Task.java
@@ -80,7 +80,7 @@ public abstract class Task implements ExternalViewChangeListener {
   }
 
   private boolean isParentTaskDone(ExternalView ev) {
-    Set<String> partitionSet = ev.getPartitionStringSet();
+    Set<String> partitionSet = ev.getPartitionSet();
     if (partitionSet.isEmpty()) {
       return false;
     }


[8/9] [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancerContext.java
new file mode 100644
index 0000000..e1f1ac2
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancerContext.java
@@ -0,0 +1,176 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.api.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.IdealState.RebalanceMode;
+import org.apache.helix.model.StateModelDefinition;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+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.
+ */
+
+/**
+ * RebalancerContext for SEMI_AUTO rebalancer mode. It indicates the preferred locations of each
+ * partition replica. By default, it corresponds to {@link SemiAutoRebalancer}
+ */
+public final class SemiAutoRebalancerContext extends PartitionedRebalancerContext {
+  @JsonProperty("preferenceLists")
+  private Map<PartitionId, List<ParticipantId>> _preferenceLists;
+
+  /**
+   * Instantiate a SemiAutoRebalancerContext
+   */
+  public SemiAutoRebalancerContext() {
+    super(RebalanceMode.SEMI_AUTO);
+    setRebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class));
+    _preferenceLists = Maps.newHashMap();
+  }
+
+  /**
+   * Get the preference lists of all partitions of the resource
+   * @return map of partition id to list of participant ids
+   */
+  public Map<PartitionId, List<ParticipantId>> getPreferenceLists() {
+    return _preferenceLists;
+  }
+
+  /**
+   * Set the preference lists of all partitions of the resource
+   * @param preferenceLists
+   */
+  public void setPreferenceLists(Map<PartitionId, List<ParticipantId>> preferenceLists) {
+    _preferenceLists = preferenceLists;
+  }
+
+  /**
+   * Get the preference list of a partition
+   * @param partitionId the partition to look up
+   * @return list of participant ids
+   */
+  @JsonIgnore
+  public List<ParticipantId> getPreferenceList(PartitionId partitionId) {
+    return _preferenceLists.get(partitionId);
+  }
+
+  /**
+   * Generate preference lists based on a default cluster setup
+   * @param stateModelDef the state model definition to follow
+   * @param participantSet the set of participant ids to configure for
+   */
+  @Override
+  @JsonIgnore
+  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
+      Set<ParticipantId> participantSet) {
+    // compute default upper bounds
+    Map<State, String> upperBounds = Maps.newHashMap();
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
+      upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
+    }
+
+    // determine the current mapping
+    Map<PartitionId, Map<ParticipantId, State>> currentMapping = Maps.newHashMap();
+    for (PartitionId partitionId : getPartitionSet()) {
+      List<ParticipantId> preferenceList = getPreferenceList(partitionId);
+      if (preferenceList != null && !preferenceList.isEmpty()) {
+        Set<ParticipantId> disabledParticipants = Collections.emptySet();
+        Map<ParticipantId, State> emptyCurrentState = Collections.emptyMap();
+        Map<ParticipantId, State> initialMap =
+            ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
+                participantSet, stateModelDef, preferenceList, emptyCurrentState,
+                disabledParticipants);
+        currentMapping.put(partitionId, initialMap);
+      }
+    }
+
+    // determine the preference
+    LinkedHashMap<State, Integer> stateCounts =
+        ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(),
+            getReplicaCount());
+    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
+    List<ParticipantId> participantList = new ArrayList<ParticipantId>(participantSet);
+    List<PartitionId> partitionList = new ArrayList<PartitionId>(getPartitionSet());
+    AutoRebalanceStrategy strategy =
+        new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts,
+            getMaxPartitionsPerParticipant(), placementScheme);
+    Map<String, List<String>> rawPreferenceLists =
+        strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList)
+            .getListFields();
+    Map<PartitionId, List<ParticipantId>> preferenceLists =
+        Maps.newHashMap(IdealState.preferenceListsFromStringLists(rawPreferenceLists));
+    setPreferenceLists(preferenceLists);
+  }
+
+  /**
+   * Build a SemiAutoRebalancerContext. By default, it corresponds to {@link SemiAutoRebalancer}
+   */
+  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
+    private final Map<PartitionId, List<ParticipantId>> _preferenceLists;
+
+    /**
+     * Instantiate for a resource
+     * @param resourceId resource id
+     */
+    public Builder(ResourceId resourceId) {
+      super(resourceId);
+      super.rebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class));
+      _preferenceLists = Maps.newHashMap();
+    }
+
+    /**
+     * Add a preference list for a partition
+     * @param partitionId partition to set
+     * @param preferenceList ordered list of participants who can serve the partition
+     * @return Builder
+     */
+    public Builder preferenceList(PartitionId partitionId, List<ParticipantId> preferenceList) {
+      _preferenceLists.put(partitionId, preferenceList);
+      return self();
+    }
+
+    @Override
+    protected Builder self() {
+      return this;
+    }
+
+    @Override
+    public SemiAutoRebalancerContext build() {
+      SemiAutoRebalancerContext context = new SemiAutoRebalancerContext();
+      super.update(context);
+      context.setPreferenceLists(_preferenceLists);
+      return context;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/util/ConstraintBasedAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/util/ConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/util/ConstraintBasedAssignment.java
new file mode 100644
index 0000000..0199796
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/util/ConstraintBasedAssignment.java
@@ -0,0 +1,244 @@
+package org.apache.helix.api.rebalancer.util;
+
+/*
+ * 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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+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.Participant;
+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;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/**
+ * Collection of functions that will compute the best possible state based on the participants and
+ * the rebalancer configuration of a resource.
+ */
+public class ConstraintBasedAssignment {
+  private static Logger logger = Logger.getLogger(ConstraintBasedAssignment.class);
+
+  /**
+   * Get a set of disabled participants for a partition
+   * @param participantMap map of all participants
+   * @param partitionId the partition to check
+   * @return a set of all participants that are disabled for the partition
+   */
+  public static Set<ParticipantId> getDisabledParticipants(
+      final Map<ParticipantId, Participant> participantMap, final PartitionId partitionId) {
+    Set<ParticipantId> participantSet = new HashSet<ParticipantId>(participantMap.keySet());
+    Set<ParticipantId> disabledParticipantsForPartition =
+        Sets.filter(participantSet, new Predicate<ParticipantId>() {
+          @Override
+          public boolean apply(ParticipantId participantId) {
+            Participant participant = participantMap.get(participantId);
+            return !participant.isEnabled()
+                || participant.getDisabledPartitionIds().contains(partitionId);
+          }
+        });
+    return disabledParticipantsForPartition;
+  }
+
+  /**
+   * Get an ordered list of participants that can serve a partition
+   * @param cluster cluster snapshot
+   * @param partitionId the partition to look up
+   * @param config rebalancing constraints
+   * @return list with most preferred participants first
+   */
+  public static List<ParticipantId> getPreferenceList(Cluster cluster, PartitionId partitionId,
+      List<ParticipantId> prefList) {
+    if (prefList != null && prefList.size() == 1
+        && StateModelToken.ANY_LIVEINSTANCE.toString().equals(prefList.get(0).stringify())) {
+      prefList = new ArrayList<ParticipantId>(cluster.getLiveParticipantMap().keySet());
+      Collections.sort(prefList);
+    }
+    return prefList;
+  }
+
+  /**
+   * Get a map of state to upper bound constraint given a cluster
+   * @param stateModelDef the state model definition to check
+   * @param resourceId the resource that is constraint
+   * @param cluster the cluster the resource belongs to
+   * @return map of state to upper bound
+   */
+  public static Map<State, String> stateConstraints(StateModelDefinition stateModelDef,
+      ResourceId resourceId, ClusterConfig cluster) {
+    Map<State, String> stateMap = Maps.newHashMap();
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
+      String num =
+          cluster.getStateUpperBoundConstraint(Scope.resource(resourceId),
+              stateModelDef.getStateModelDefId(), state);
+      stateMap.put(state, num);
+    }
+    return stateMap;
+  }
+
+  /**
+   * compute best state for resource in SEMI_AUTO and FULL_AUTO modes
+   * @param upperBounds map of state to upper bound
+   * @param liveParticipantSet set of live participant ids
+   * @param stateModelDef
+   * @param participantPreferenceList
+   * @param currentStateMap
+   *          : participant->state for each partition
+   * @param disabledParticipantsForPartition
+   * @return
+   */
+  public static Map<ParticipantId, State> computeAutoBestStateForPartition(
+      Map<State, String> upperBounds, Set<ParticipantId> liveParticipantSet,
+      StateModelDefinition stateModelDef, List<ParticipantId> participantPreferenceList,
+      Map<ParticipantId, State> currentStateMap, Set<ParticipantId> disabledParticipantsForPartition) {
+    Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
+
+    // if the resource is deleted, instancePreferenceList will be empty and
+    // we should drop all resources.
+    if (currentStateMap != null) {
+      for (ParticipantId participantId : currentStateMap.keySet()) {
+        if ((participantPreferenceList == null || !participantPreferenceList
+            .contains(participantId)) && !disabledParticipantsForPartition.contains(participantId)) {
+          // if dropped and not disabled, transit to DROPPED
+          participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED));
+        } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get(
+            participantId).equals(State.from(HelixDefinedState.ERROR)))
+            && disabledParticipantsForPartition.contains(participantId)) {
+          // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
+          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
+        }
+      }
+    }
+
+    // resource is deleted
+    if (participantPreferenceList == null) {
+      return participantStateMap;
+    }
+
+    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
+    boolean assigned[] = new boolean[participantPreferenceList.size()];
+
+    for (State state : statesPriorityList) {
+      String num = upperBounds.get(state);
+      int stateCount = -1;
+      if ("N".equals(num)) {
+        Set<ParticipantId> liveAndEnabled = new HashSet<ParticipantId>(liveParticipantSet);
+        liveAndEnabled.removeAll(disabledParticipantsForPartition);
+        stateCount = liveAndEnabled.size();
+      } else if ("R".equals(num)) {
+        stateCount = participantPreferenceList.size();
+      } else {
+        try {
+          stateCount = Integer.parseInt(num);
+        } catch (Exception e) {
+          logger.error("Invalid count for state:" + state + " ,count=" + num);
+        }
+      }
+      if (stateCount > -1) {
+        int count = 0;
+        for (int i = 0; i < participantPreferenceList.size(); i++) {
+          ParticipantId participantId = participantPreferenceList.get(i);
+
+          boolean notInErrorState =
+              currentStateMap == null
+                  || currentStateMap.get(participantId) == null
+                  || !currentStateMap.get(participantId)
+                      .equals(State.from(HelixDefinedState.ERROR));
+
+          if (liveParticipantSet.contains(participantId) && !assigned[i] && notInErrorState
+              && !disabledParticipantsForPartition.contains(participantId)) {
+            participantStateMap.put(participantId, state);
+            count = count + 1;
+            assigned[i] = true;
+            if (count == stateCount) {
+              break;
+            }
+          }
+        }
+      }
+    }
+    return participantStateMap;
+  }
+
+  /**
+   * Get the number of replicas that should be in each state for a partition
+   * @param upperBounds map of state to upper bound
+   * @param stateModelDef StateModelDefinition object
+   * @param liveNodesNb number of live nodes
+   * @param total number of replicas
+   * @return state count map: state->count
+   */
+  public static LinkedHashMap<State, Integer> stateCount(Map<State, String> upperBounds,
+      StateModelDefinition stateModelDef, int liveNodesNb, int totalReplicas) {
+    LinkedHashMap<State, Integer> stateCountMap = new LinkedHashMap<State, Integer>();
+    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
+
+    int replicas = totalReplicas;
+    for (State state : statesPriorityList) {
+      String num = upperBounds.get(state);
+      if ("N".equals(num)) {
+        stateCountMap.put(state, liveNodesNb);
+      } else if ("R".equals(num)) {
+        // wait until we get the counts for all other states
+        continue;
+      } else {
+        int stateCount = -1;
+        try {
+          stateCount = Integer.parseInt(num);
+        } catch (Exception e) {
+          // LOG.error("Invalid count for state: " + state + ", count: " + num +
+          // ", use -1 instead");
+        }
+
+        if (stateCount > 0) {
+          stateCountMap.put(state, stateCount);
+          replicas -= stateCount;
+        }
+      }
+    }
+
+    // get state count for R
+    for (State state : statesPriorityList) {
+      String num = upperBounds.get(state);
+      if ("R".equals(num)) {
+        stateCountMap.put(state, replicas);
+        // should have at most one state using R
+        break;
+      }
+    }
+    return stateCountMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index b27398d..eec745e 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -45,16 +45,16 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.pipeline.Pipeline;
 import org.apache.helix.controller.pipeline.PipelineRegistry;
 import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.controller.stages.NewBestPossibleStateCalcStage;
-import org.apache.helix.controller.stages.NewCompatibilityCheckStage;
-import org.apache.helix.controller.stages.NewCurrentStateComputationStage;
-import org.apache.helix.controller.stages.NewExternalViewComputeStage;
-import org.apache.helix.controller.stages.NewMessageGenerationStage;
-import org.apache.helix.controller.stages.NewMessageSelectionStage;
-import org.apache.helix.controller.stages.NewMessageThrottleStage;
-import org.apache.helix.controller.stages.NewReadClusterDataStage;
-import org.apache.helix.controller.stages.NewResourceComputationStage;
-import org.apache.helix.controller.stages.NewTaskAssignmentStage;
+import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
+import org.apache.helix.controller.stages.CompatibilityCheckStage;
+import org.apache.helix.controller.stages.CurrentStateComputationStage;
+import org.apache.helix.controller.stages.ExternalViewComputeStage;
+import org.apache.helix.controller.stages.MessageGenerationStage;
+import org.apache.helix.controller.stages.MessageSelectionStage;
+import org.apache.helix.controller.stages.MessageThrottleStage;
+import org.apache.helix.controller.stages.ReadClusterDataStage;
+import org.apache.helix.controller.stages.ResourceComputationStage;
+import org.apache.helix.controller.stages.TaskAssignmentStage;
 import org.apache.helix.controller.stages.PersistAssignmentStage;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
@@ -175,23 +175,23 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
 
       // cluster data cache refresh
       Pipeline dataRefresh = new Pipeline();
-      dataRefresh.addStage(new NewReadClusterDataStage());
+      dataRefresh.addStage(new ReadClusterDataStage());
 
       // rebalance pipeline
       Pipeline rebalancePipeline = new Pipeline();
-      rebalancePipeline.addStage(new NewCompatibilityCheckStage());
-      rebalancePipeline.addStage(new NewResourceComputationStage());
-      rebalancePipeline.addStage(new NewCurrentStateComputationStage());
-      rebalancePipeline.addStage(new NewBestPossibleStateCalcStage());
+      rebalancePipeline.addStage(new CompatibilityCheckStage());
+      rebalancePipeline.addStage(new ResourceComputationStage());
+      rebalancePipeline.addStage(new CurrentStateComputationStage());
+      rebalancePipeline.addStage(new BestPossibleStateCalcStage());
       rebalancePipeline.addStage(new PersistAssignmentStage());
-      rebalancePipeline.addStage(new NewMessageGenerationStage());
-      rebalancePipeline.addStage(new NewMessageSelectionStage());
-      rebalancePipeline.addStage(new NewMessageThrottleStage());
-      rebalancePipeline.addStage(new NewTaskAssignmentStage());
+      rebalancePipeline.addStage(new MessageGenerationStage());
+      rebalancePipeline.addStage(new MessageSelectionStage());
+      rebalancePipeline.addStage(new MessageThrottleStage());
+      rebalancePipeline.addStage(new TaskAssignmentStage());
 
       // external view generation
       Pipeline externalViewPipeline = new Pipeline();
-      externalViewPipeline.addStage(new NewExternalViewComputeStage());
+      externalViewPipeline.addStage(new ExternalViewComputeStage());
 
       registry.register("idealStateChange", dataRefresh, rebalancePipeline);
       registry.register("currentStateChange", dataRefresh, rebalancePipeline, externalViewPipeline);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
index 5dd399f..26fc2ef 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
@@ -30,6 +30,8 @@ import org.apache.helix.model.ResourceAssignment;
  * Allows one to come up with custom implementation of a rebalancer.<br/>
  * This will be invoked on all changes that happen in the cluster.<br/>
  * Simply return the newIdealState for a resource in this method.<br/>
+ * <br/>
+ * Deprecated. Use {@link org.apache.helix.api.rebalancer.Rebalancer} instead.
  */
 @Deprecated
 public interface Rebalancer {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index bb2ab17..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/BasicRebalancerContext.java
+++ /dev/null
@@ -1,239 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.Set;
-
-import org.apache.helix.api.Partition;
-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;
-
-/*
- * 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.
- */
-
-/**
- * Abstract RebalancerContext that functions for generic subunits. Use a subclass that more
- * concretely defines the subunits.
- */
-public abstract class BasicRebalancerContext implements RebalancerContext {
-  private ResourceId _resourceId;
-  private StateModelDefId _stateModelDefId;
-  private StateModelFactoryId _stateModelFactoryId;
-  private String _participantGroupTag;
-  private Class<? extends ContextSerializer> _serializer;
-  private RebalancerRef _rebalancerRef;
-
-  /**
-   * Instantiate a basic rebalancer context
-   */
-  public BasicRebalancerContext() {
-    _serializer = DefaultContextSerializer.class;
-  }
-
-  @Override
-  public ResourceId getResourceId() {
-    return _resourceId;
-  }
-
-  /**
-   * Set the resource to rebalance
-   * @param resourceId resource id
-   */
-  public void setResourceId(ResourceId resourceId) {
-    _resourceId = resourceId;
-  }
-
-  @Override
-  public StateModelDefId getStateModelDefId() {
-    return _stateModelDefId;
-  }
-
-  /**
-   * Set the state model definition that the resource follows
-   * @param stateModelDefId state model definition id
-   */
-  public void setStateModelDefId(StateModelDefId stateModelDefId) {
-    _stateModelDefId = stateModelDefId;
-  }
-
-  @Override
-  public StateModelFactoryId getStateModelFactoryId() {
-    return _stateModelFactoryId;
-  }
-
-  /**
-   * Set the state model factory that the resource uses
-   * @param stateModelFactoryId state model factory id
-   */
-  public void setStateModelFactoryId(StateModelFactoryId stateModelFactoryId) {
-    _stateModelFactoryId = stateModelFactoryId;
-  }
-
-  @Override
-  public String getParticipantGroupTag() {
-    return _participantGroupTag;
-  }
-
-  /**
-   * Set a tag that participants must have in order to serve this resource
-   * @param participantGroupTag string group tag
-   */
-  public void setParticipantGroupTag(String participantGroupTag) {
-    _participantGroupTag = participantGroupTag;
-  }
-
-  /**
-   * Get the serializer. If none is provided, {@link DefaultContextSerializer} is used
-   */
-  @Override
-  public Class<? extends ContextSerializer> getSerializerClass() {
-    return _serializer;
-  }
-
-  /**
-   * Set the class that can serialize this context
-   * @param serializer serializer class that implements ContextSerializer
-   */
-  public void setSerializerClass(Class<? extends ContextSerializer> serializer) {
-    _serializer = serializer;
-  }
-
-  @Override
-  @JsonIgnore
-  public Set<? extends PartitionId> getSubUnitIdSet() {
-    return getSubUnitMap().keySet();
-  }
-
-  @Override
-  @JsonIgnore
-  public Partition getSubUnit(PartitionId subUnitId) {
-    return getSubUnitMap().get(subUnitId);
-  }
-
-  @Override
-  public RebalancerRef getRebalancerRef() {
-    return _rebalancerRef;
-  }
-
-  /**
-   * Set the reference to the class used to rebalance this resource
-   * @param rebalancerRef RebalancerRef instance
-   */
-  public void setRebalancerRef(RebalancerRef rebalancerRef) {
-    _rebalancerRef = rebalancerRef;
-  }
-
-  /**
-   * Abstract builder for the base rebalancer context
-   */
-  public static abstract class AbstractBuilder<T extends AbstractBuilder<T>> {
-    private final ResourceId _resourceId;
-    private StateModelDefId _stateModelDefId;
-    private StateModelFactoryId _stateModelFactoryId;
-    private String _participantGroupTag;
-    private Class<? extends ContextSerializer> _serializerClass;
-    private RebalancerRef _rebalancerRef;
-
-    /**
-     * Instantiate with a resource id
-     * @param resourceId resource id
-     */
-    public AbstractBuilder(ResourceId resourceId) {
-      _resourceId = resourceId;
-      _serializerClass = DefaultContextSerializer.class;
-    }
-
-    /**
-     * Set the state model definition that the resource should follow
-     * @param stateModelDefId state model definition id
-     * @return Builder
-     */
-    public T stateModelDefId(StateModelDefId stateModelDefId) {
-      _stateModelDefId = stateModelDefId;
-      return self();
-    }
-
-    /**
-     * Set the state model factory that the resource should use
-     * @param stateModelFactoryId state model factory id
-     * @return Builder
-     */
-    public T stateModelFactoryId(StateModelFactoryId stateModelFactoryId) {
-      _stateModelFactoryId = stateModelFactoryId;
-      return self();
-    }
-
-    /**
-     * Set the tag that all participants require in order to serve this resource
-     * @param participantGroupTag the tag
-     * @return Builder
-     */
-    public T participantGroupTag(String participantGroupTag) {
-      _participantGroupTag = participantGroupTag;
-      return self();
-    }
-
-    /**
-     * Set the serializer class for this rebalancer context
-     * @param serializerClass class that implements ContextSerializer
-     * @return Builder
-     */
-    public T serializerClass(Class<? extends ContextSerializer> serializerClass) {
-      _serializerClass = serializerClass;
-      return self();
-    }
-
-    /**
-     * Specify a custom class to use for rebalancing
-     * @param rebalancerRef RebalancerRef instance
-     * @return Builder
-     */
-    public T rebalancerRef(RebalancerRef rebalancerRef) {
-      _rebalancerRef = rebalancerRef;
-      return self();
-    }
-
-    /**
-     * Update an existing context with base fields
-     * @param context derived context
-     */
-    protected final void update(BasicRebalancerContext context) {
-      context.setResourceId(_resourceId);
-      context.setStateModelDefId(_stateModelDefId);
-      context.setStateModelFactoryId(_stateModelFactoryId);
-      context.setParticipantGroupTag(_participantGroupTag);
-      context.setSerializerClass(_serializerClass);
-      context.setRebalancerRef(_rebalancerRef);
-    }
-
-    /**
-     * Get a typed reference to "this" class. Final derived classes should simply return the this
-     * reference.
-     * @return this for the most specific type
-     */
-    protected abstract T self();
-
-    /**
-     * Get the rebalancer context from the built fields
-     * @return RebalancerContext
-     */
-    public abstract RebalancerContext build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java
deleted file mode 100644
index ef12a09..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ContextSerializer.java
+++ /dev/null
@@ -1,37 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-/*
- * 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 interface ContextSerializer {
-  /**
-   * Convert a RebalancerContext object instance to a String
-   * @param data instance of the rebalancer context type
-   * @return String representing the object
-   */
-  public <T> String serialize(final T data);
-
-  /**
-   * Convert raw bytes to a generic object instance
-   * @param clazz The class represented by the deserialized string
-   * @param string String representing the object
-   * @return instance of the generic type or null if the conversion failed
-   */
-  public <T> T deserialize(final Class<T> clazz, final String string);
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 00219af..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancer.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixDefinedState;
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-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;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-/*
- * 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 CustomRebalancer implements Rebalancer {
-
-  private static final Logger LOG = Logger.getLogger(CustomRebalancer.class);
-
-  @Override
-  public void init(HelixManager helixManager) {
-    // do nothing
-  }
-
-  @Override
-  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
-      Cluster cluster, ResourceCurrentState currentState) {
-    CustomRebalancerContext config =
-        rebalancerConfig.getRebalancerContext(CustomRebalancerContext.class);
-    StateModelDefinition stateModelDef =
-        cluster.getStateModelMap().get(config.getStateModelDefId());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing resource:" + config.getResourceId());
-    }
-    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
-    for (PartitionId partition : config.getPartitionSet()) {
-      Map<ParticipantId, State> currentStateMap =
-          currentState.getCurrentStateMap(config.getResourceId(), partition);
-      Set<ParticipantId> disabledInstancesForPartition =
-          NewConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
-              partition);
-      Map<ParticipantId, State> bestStateForPartition =
-          computeCustomizedBestStateForPartition(cluster.getLiveParticipantMap().keySet(),
-              stateModelDef, config.getPreferenceMap(partition), currentStateMap,
-              disabledInstancesForPartition);
-      partitionMapping.addReplicaMap(partition, bestStateForPartition);
-    }
-    return partitionMapping;
-  }
-
-  /**
-   * compute best state for resource in CUSTOMIZED rebalancer mode
-   * @param liveParticipantMap
-   * @param stateModelDef
-   * @param preferenceMap
-   * @param currentStateMap
-   * @param disabledParticipantsForPartition
-   * @return
-   */
-  private Map<ParticipantId, State> computeCustomizedBestStateForPartition(
-      Set<ParticipantId> liveParticipantSet, StateModelDefinition stateModelDef,
-      Map<ParticipantId, State> preferenceMap, Map<ParticipantId, State> currentStateMap,
-      Set<ParticipantId> disabledParticipantsForPartition) {
-    Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
-
-    // if the resource is deleted, idealStateMap will be null/empty and
-    // we should drop all resources.
-    if (currentStateMap != null) {
-      for (ParticipantId participantId : currentStateMap.keySet()) {
-        if ((preferenceMap == null || !preferenceMap.containsKey(participantId))
-            && !disabledParticipantsForPartition.contains(participantId)) {
-          // if dropped and not disabled, transit to DROPPED
-          participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED));
-        } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get(
-            participantId).equals(State.from(HelixDefinedState.ERROR)))
-            && disabledParticipantsForPartition.contains(participantId)) {
-          // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
-        }
-      }
-    }
-
-    // ideal state is deleted
-    if (preferenceMap == null) {
-      return participantStateMap;
-    }
-
-    for (ParticipantId participantId : preferenceMap.keySet()) {
-      boolean notInErrorState =
-          currentStateMap == null || currentStateMap.get(participantId) == null
-              || !currentStateMap.get(participantId).equals(State.from(HelixDefinedState.ERROR));
-
-      if (liveParticipantSet.contains(participantId) && notInErrorState
-          && !disabledParticipantsForPartition.contains(participantId)) {
-        participantStateMap.put(participantId, preferenceMap.get(participantId));
-      }
-    }
-
-    return participantStateMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 6e1485b..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.java
+++ /dev/null
@@ -1,161 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-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.controller.rebalancer.util.NewConstraintBasedAssignment;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.StateModelDefinition;
-import org.codehaus.jackson.annotate.JsonIgnore;
-
-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.
- */
-
-/**
- * RebalancerContext for a resource that should be rebalanced in CUSTOMIZED mode. By default, it
- * corresponds to {@link CustomRebalancer}
- */
-public class CustomRebalancerContext extends PartitionedRebalancerContext {
-  private Map<PartitionId, Map<ParticipantId, State>> _preferenceMaps;
-
-  /**
-   * Instantiate a CustomRebalancerContext
-   */
-  public CustomRebalancerContext() {
-    super(RebalanceMode.CUSTOMIZED);
-    setRebalancerRef(RebalancerRef.from(CustomRebalancer.class));
-    _preferenceMaps = Maps.newHashMap();
-  }
-
-  /**
-   * Get the preference maps of the partitions and replicas of the resource
-   * @return map of partition to participant and state
-   */
-  public Map<PartitionId, Map<ParticipantId, State>> getPreferenceMaps() {
-    return _preferenceMaps;
-  }
-
-  /**
-   * Set the preference maps of the partitions and replicas of the resource
-   * @param preferenceMaps map of partition to participant and state
-   */
-  public void setPreferenceMaps(Map<PartitionId, Map<ParticipantId, State>> preferenceMaps) {
-    _preferenceMaps = preferenceMaps;
-  }
-
-  /**
-   * Get the preference map of a partition
-   * @param partitionId the partition to look up
-   * @return map of participant to state
-   */
-  @JsonIgnore
-  public Map<ParticipantId, State> getPreferenceMap(PartitionId partitionId) {
-    return _preferenceMaps.get(partitionId);
-  }
-
-  /**
-   * Generate preference maps based on a default cluster setup
-   * @param stateModelDef the state model definition to follow
-   * @param participantSet the set of participant ids to configure for
-   */
-  @Override
-  @JsonIgnore
-  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
-      Set<ParticipantId> participantSet) {
-    // compute default upper bounds
-    Map<State, String> upperBounds = Maps.newHashMap();
-    for (State state : stateModelDef.getTypedStatesPriorityList()) {
-      upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
-    }
-
-    // determine the current mapping
-    Map<PartitionId, Map<ParticipantId, State>> currentMapping = getPreferenceMaps();
-
-    // determine the preference maps
-    LinkedHashMap<State, Integer> stateCounts =
-        NewConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(),
-            getReplicaCount());
-    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
-    List<ParticipantId> participantList = new ArrayList<ParticipantId>(participantSet);
-    List<PartitionId> partitionList = new ArrayList<PartitionId>(getPartitionSet());
-    AutoRebalanceStrategy strategy =
-        new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts,
-            getMaxPartitionsPerParticipant(), placementScheme);
-    Map<String, Map<String, String>> rawPreferenceMaps =
-        strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList)
-            .getMapFields();
-    Map<PartitionId, Map<ParticipantId, State>> preferenceMaps =
-        Maps.newHashMap(ResourceAssignment.replicaMapsFromStringMaps(rawPreferenceMaps));
-    setPreferenceMaps(preferenceMaps);
-  }
-
-  /**
-   * Build a CustomRebalancerContext. By default, it corresponds to {@link CustomRebalancer}
-   */
-  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
-    private final Map<PartitionId, Map<ParticipantId, State>> _preferenceMaps;
-
-    /**
-     * Instantiate for a resource
-     * @param resourceId resource id
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-      super.rebalancerRef(RebalancerRef.from(CustomRebalancer.class));
-      _preferenceMaps = Maps.newHashMap();
-    }
-
-    /**
-     * Add a preference map for a partition
-     * @param partitionId partition to set
-     * @param preferenceList map of participant id to state indicating where replicas are served
-     * @return Builder
-     */
-    public Builder preferenceMap(PartitionId partitionId, Map<ParticipantId, State> preferenceMap) {
-      _preferenceMaps.put(partitionId, preferenceMap);
-      return self();
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-
-    @Override
-    public CustomRebalancerContext build() {
-      CustomRebalancerContext context = new CustomRebalancerContext();
-      super.update(context);
-      context.setPreferenceMaps(_preferenceMaps);
-      return context;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java
deleted file mode 100644
index ecc93fb..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/DefaultContextSerializer.java
+++ /dev/null
@@ -1,83 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.io.ByteArrayInputStream;
-import java.io.StringWriter;
-
-import org.apache.helix.HelixException;
-import org.apache.log4j.Logger;
-import org.codehaus.jackson.map.DeserializationConfig;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.SerializationConfig;
-
-/*
- * 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.
- */
-
-/**
- * Default serializer implementation for RebalancerContexts. Uses the Jackson JSON library to
- * convert to and from strings
- */
-public class DefaultContextSerializer implements ContextSerializer {
-
-  private static Logger logger = Logger.getLogger(DefaultContextSerializer.class);
-
-  @Override
-  public <T> String serialize(final T data) {
-    if (data == null) {
-      return null;
-    }
-
-    ObjectMapper mapper = new ObjectMapper();
-    SerializationConfig serializationConfig = mapper.getSerializationConfig();
-    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
-    serializationConfig.set(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
-    serializationConfig.set(SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
-    StringWriter sw = new StringWriter();
-    try {
-      mapper.writeValue(sw, data);
-    } catch (Exception e) {
-      logger.error("Exception during payload data serialization.", e);
-      throw new HelixException(e);
-    }
-    return sw.toString();
-  }
-
-  @Override
-  public <T> T deserialize(final Class<T> clazz, final String string) {
-    if (string == null || string.length() == 0) {
-      return null;
-    }
-
-    ObjectMapper mapper = new ObjectMapper();
-    ByteArrayInputStream bais = new ByteArrayInputStream(string.getBytes());
-
-    DeserializationConfig deserializationConfig = mapper.getDeserializationConfig();
-    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_FIELDS, true);
-    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_SETTERS, true);
-    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_CREATORS, true);
-    deserializationConfig.set(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, true);
-    deserializationConfig.set(DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
-    try {
-      T payload = mapper.readValue(bais, clazz);
-      return payload;
-    } catch (Exception e) {
-      logger.error("Exception during deserialization of payload bytes: " + string, e);
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 521af5c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancer.java
+++ /dev/null
@@ -1,194 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-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.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;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-
-/*
- * 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 FullAutoRebalancer implements Rebalancer {
-  // These should be final, but are initialized in init rather than a constructor
-  private AutoRebalanceStrategy _algorithm;
-
-  private static Logger LOG = Logger.getLogger(FullAutoRebalancer.class);
-
-  @Override
-  public void init(HelixManager helixManager) {
-    // do nothing
-  }
-
-  @Override
-  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
-      Cluster cluster, ResourceCurrentState currentState) {
-    FullAutoRebalancerContext config =
-        rebalancerConfig.getRebalancerContext(FullAutoRebalancerContext.class);
-    StateModelDefinition stateModelDef =
-        cluster.getStateModelMap().get(config.getStateModelDefId());
-    // Compute a preference list based on the current ideal state
-    List<PartitionId> partitions = new ArrayList<PartitionId>(config.getPartitionSet());
-    Map<ParticipantId, Participant> liveParticipants = cluster.getLiveParticipantMap();
-    Map<ParticipantId, Participant> allParticipants = cluster.getParticipantMap();
-    int replicas = -1;
-    if (config.anyLiveParticipant()) {
-      replicas = liveParticipants.size();
-    } else {
-      replicas = config.getReplicaCount();
-    }
-
-    // count how many replicas should be in each state
-    Map<State, String> upperBounds =
-        NewConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(),
-            cluster.getConfig());
-    LinkedHashMap<State, Integer> stateCountMap =
-        NewConstraintBasedAssignment.stateCount(upperBounds, stateModelDef,
-            liveParticipants.size(), replicas);
-
-    // get the participant lists
-    List<ParticipantId> liveParticipantList =
-        new ArrayList<ParticipantId>(liveParticipants.keySet());
-    List<ParticipantId> allParticipantList =
-        new ArrayList<ParticipantId>(cluster.getParticipantMap().keySet());
-
-    // compute the current mapping from the current state
-    Map<PartitionId, Map<ParticipantId, State>> currentMapping =
-        currentMapping(config, currentState, stateCountMap);
-
-    // If there are nodes tagged with resource, use only those nodes
-    Set<ParticipantId> taggedNodes = new HashSet<ParticipantId>();
-    if (config.getParticipantGroupTag() != null) {
-      for (ParticipantId participantId : liveParticipantList) {
-        if (liveParticipants.get(participantId).hasTag(config.getParticipantGroupTag())) {
-          taggedNodes.add(participantId);
-        }
-      }
-    }
-    if (taggedNodes.size() > 0) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("found the following instances with tag " + config.getResourceId() + " "
-            + taggedNodes);
-      }
-      liveParticipantList = new ArrayList<ParticipantId>(taggedNodes);
-    }
-
-    // determine which nodes the replicas should live on
-    int maxPartition = config.getMaxPartitionsPerParticipant();
-    if (LOG.isInfoEnabled()) {
-      LOG.info("currentMapping: " + currentMapping);
-      LOG.info("stateCountMap: " + stateCountMap);
-      LOG.info("liveNodes: " + liveParticipantList);
-      LOG.info("allNodes: " + allParticipantList);
-      LOG.info("maxPartition: " + maxPartition);
-    }
-    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
-    _algorithm =
-        new AutoRebalanceStrategy(config.getResourceId(), partitions, stateCountMap, maxPartition,
-            placementScheme);
-    ZNRecord newMapping =
-        _algorithm.typedComputePartitionAssignment(liveParticipantList, currentMapping,
-            allParticipantList);
-
-    if (LOG.isInfoEnabled()) {
-      LOG.info("newMapping: " + newMapping);
-    }
-
-    // compute a full partition mapping for the resource
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing resource:" + config.getResourceId());
-    }
-    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
-    for (PartitionId partition : partitions) {
-      Set<ParticipantId> disabledParticipantsForPartition =
-          NewConstraintBasedAssignment.getDisabledParticipants(allParticipants, partition);
-      List<String> rawPreferenceList = newMapping.getListField(partition.stringify());
-      if (rawPreferenceList == null) {
-        rawPreferenceList = Collections.emptyList();
-      }
-      List<ParticipantId> preferenceList =
-          Lists.transform(rawPreferenceList, new Function<String, ParticipantId>() {
-            @Override
-            public ParticipantId apply(String participantName) {
-              return ParticipantId.from(participantName);
-            }
-          });
-      preferenceList =
-          NewConstraintBasedAssignment.getPreferenceList(cluster, partition, preferenceList);
-      Map<ParticipantId, State> bestStateForPartition =
-          NewConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
-              liveParticipants.keySet(), stateModelDef, preferenceList,
-              currentState.getCurrentStateMap(config.getResourceId(), partition),
-              disabledParticipantsForPartition);
-      partitionMapping.addReplicaMap(partition, bestStateForPartition);
-    }
-    return partitionMapping;
-  }
-
-  private Map<PartitionId, Map<ParticipantId, State>> currentMapping(
-      FullAutoRebalancerContext config, ResourceCurrentState currentStateOutput,
-      Map<State, Integer> stateCountMap) {
-    Map<PartitionId, Map<ParticipantId, State>> map =
-        new HashMap<PartitionId, Map<ParticipantId, State>>();
-
-    for (PartitionId partition : config.getPartitionSet()) {
-      Map<ParticipantId, State> curStateMap =
-          currentStateOutput.getCurrentStateMap(config.getResourceId(), partition);
-      map.put(partition, new HashMap<ParticipantId, State>());
-      for (ParticipantId node : curStateMap.keySet()) {
-        State state = curStateMap.get(node);
-        if (stateCountMap.containsKey(state)) {
-          map.get(partition).put(node, state);
-        }
-      }
-
-      Map<ParticipantId, State> pendingStateMap =
-          currentStateOutput.getPendingStateMap(config.getResourceId(), partition);
-      for (ParticipantId node : pendingStateMap.keySet()) {
-        State state = pendingStateMap.get(node);
-        if (stateCountMap.containsKey(state)) {
-          map.get(partition).put(node, state);
-        }
-      }
-    }
-    return map;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 11a1b47..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.java
+++ /dev/null
@@ -1,61 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import org.apache.helix.api.id.ResourceId;
-import org.apache.helix.model.IdealState.RebalanceMode;
-
-/*
- * 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.
- */
-
-/**
- * RebalancerContext for FULL_AUTO rebalancing mode. By default, it corresponds to
- * {@link FullAutoRebalancer}
- */
-public class FullAutoRebalancerContext extends PartitionedRebalancerContext {
-  public FullAutoRebalancerContext() {
-    super(RebalanceMode.FULL_AUTO);
-    setRebalancerRef(RebalancerRef.from(FullAutoRebalancer.class));
-  }
-
-  /**
-   * Builder for a full auto rebalancer context. By default, it corresponds to
-   * {@link FullAutoRebalancer}
-   */
-  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
-    /**
-     * Instantiate with a resource
-     * @param resourceId resource id
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-      super.rebalancerRef(RebalancerRef.from(FullAutoRebalancer.class));
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-
-    @Override
-    public FullAutoRebalancerContext build() {
-      FullAutoRebalancerContext context = new FullAutoRebalancerContext();
-      super.update(context);
-      return context;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 768e40c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.java
+++ /dev/null
@@ -1,372 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixConstants.StateModelToken;
-import org.apache.helix.api.Partition;
-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.apache.helix.model.StateModelDefinition;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-
-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.
- */
-
-/**
- * RebalancerContext for a resource whose subunits are partitions. In addition, these partitions can
- * be replicated.
- */
-@JsonIgnoreProperties(ignoreUnknown = true)
-public class PartitionedRebalancerContext extends BasicRebalancerContext implements
-    ReplicatedRebalancerContext {
-  private Map<PartitionId, Partition> _partitionMap;
-  private boolean _anyLiveParticipant;
-  private int _replicaCount;
-  private int _maxPartitionsPerParticipant;
-  private final RebalanceMode _rebalanceMode;
-
-  /**
-   * Instantiate a DataRebalancerContext
-   */
-  public PartitionedRebalancerContext(RebalanceMode rebalanceMode) {
-    _partitionMap = Collections.emptyMap();
-    _replicaCount = 1;
-    _anyLiveParticipant = false;
-    _maxPartitionsPerParticipant = Integer.MAX_VALUE;
-    _rebalanceMode = rebalanceMode;
-  }
-
-  /**
-   * Get a map from partition id to partition
-   * @return partition map (mutable)
-   */
-  public Map<PartitionId, Partition> getPartitionMap() {
-    return _partitionMap;
-  }
-
-  /**
-   * Set a map of partition id to partition
-   * @param partitionMap partition map
-   */
-  public void setPartitionMap(Map<PartitionId, Partition> partitionMap) {
-    _partitionMap = Maps.newHashMap(partitionMap);
-  }
-
-  /**
-   * Get the set of partitions for this resource
-   * @return set of partition ids
-   */
-  @JsonIgnore
-  public Set<PartitionId> getPartitionSet() {
-    return _partitionMap.keySet();
-  }
-
-  /**
-   * Get a partition
-   * @param partitionId id of the partition to get
-   * @return Partition object, or null if not present
-   */
-  @JsonIgnore
-  public Partition getPartition(PartitionId partitionId) {
-    return _partitionMap.get(partitionId);
-  }
-
-  @Override
-  public boolean anyLiveParticipant() {
-    return _anyLiveParticipant;
-  }
-
-  /**
-   * Indicate if this resource should be assigned to any live participant
-   * @param anyLiveParticipant true if any live participant expected, false otherwise
-   */
-  public void setAnyLiveParticipant(boolean anyLiveParticipant) {
-    _anyLiveParticipant = anyLiveParticipant;
-  }
-
-  @Override
-  public int getReplicaCount() {
-    return _replicaCount;
-  }
-
-  /**
-   * Set the number of replicas that each partition should have
-   * @param replicaCount
-   */
-  public void setReplicaCount(int replicaCount) {
-    _replicaCount = replicaCount;
-  }
-
-  /**
-   * Get the maximum number of partitions that a participant can serve
-   * @return maximum number of partitions per participant
-   */
-  public int getMaxPartitionsPerParticipant() {
-    return _maxPartitionsPerParticipant;
-  }
-
-  /**
-   * Set the maximum number of partitions that a participant can serve
-   * @param maxPartitionsPerParticipant maximum number of partitions per participant
-   */
-  public void setMaxPartitionsPerParticipant(int maxPartitionsPerParticipant) {
-    _maxPartitionsPerParticipant = maxPartitionsPerParticipant;
-  }
-
-  /**
-   * Get the rebalancer mode of the resource
-   * @return RebalanceMode
-   */
-  public RebalanceMode getRebalanceMode() {
-    return _rebalanceMode;
-  }
-
-  @Override
-  @JsonIgnore
-  public Map<PartitionId, Partition> getSubUnitMap() {
-    return getPartitionMap();
-  }
-
-  /**
-   * Generate a default configuration given the state model and a participant.
-   * @param stateModelDef the state model definition to follow
-   * @param participantSet the set of participant ids to configure for
-   */
-  @JsonIgnore
-  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
-      Set<ParticipantId> participantSet) {
-    // the base context does not understand enough to know do to anything
-  }
-
-  /**
-   * Convert a physically-stored IdealState into a rebalancer context for a partitioned resource
-   * @param idealState populated IdealState
-   * @return PartitionedRebalancerContext
-   */
-  public static PartitionedRebalancerContext from(IdealState idealState) {
-    PartitionedRebalancerContext context;
-    switch (idealState.getRebalanceMode()) {
-    case FULL_AUTO:
-      FullAutoRebalancerContext.Builder fullAutoBuilder =
-          new FullAutoRebalancerContext.Builder(idealState.getResourceId());
-      populateContext(fullAutoBuilder, idealState);
-      context = fullAutoBuilder.build();
-      break;
-    case SEMI_AUTO:
-      SemiAutoRebalancerContext.Builder semiAutoBuilder =
-          new SemiAutoRebalancerContext.Builder(idealState.getResourceId());
-      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
-        semiAutoBuilder.preferenceList(partitionId, idealState.getPreferenceList(partitionId));
-      }
-      populateContext(semiAutoBuilder, idealState);
-      context = semiAutoBuilder.build();
-      break;
-    case CUSTOMIZED:
-      CustomRebalancerContext.Builder customBuilder =
-          new CustomRebalancerContext.Builder(idealState.getResourceId());
-      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
-        customBuilder.preferenceMap(partitionId, idealState.getParticipantStateMap(partitionId));
-      }
-      populateContext(customBuilder, idealState);
-      context = customBuilder.build();
-      break;
-    default:
-      Builder baseBuilder = new Builder(idealState.getResourceId());
-      populateContext(baseBuilder, idealState);
-      context = baseBuilder.build();
-      break;
-    }
-    return context;
-  }
-
-  /**
-   * Update a builder subclass with all the fields of the ideal state
-   * @param builder builder that extends AbstractBuilder
-   * @param idealState populated IdealState
-   */
-  private static <T extends AbstractBuilder<T>> void populateContext(T builder,
-      IdealState idealState) {
-    String replicas = idealState.getReplicas();
-    int replicaCount = 0;
-    boolean anyLiveParticipant = false;
-    if (replicas.equals(StateModelToken.ANY_LIVEINSTANCE.toString())) {
-      anyLiveParticipant = true;
-    } else {
-      replicaCount = Integer.parseInt(replicas);
-    }
-    if (idealState.getNumPartitions() > 0 && idealState.getPartitionIdSet().size() == 0) {
-      // backwards compatibility: partition sets were based on pref lists/maps previously
-      builder.addPartitions(idealState.getNumPartitions());
-    } else {
-      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
-        builder.addPartition(new Partition(partitionId));
-      }
-    }
-    builder.anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
-        .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
-        .participantGroupTag(idealState.getInstanceGroupTag())
-        .stateModelDefId(idealState.getStateModelDefId())
-        .stateModelFactoryId(idealState.getStateModelFactoryId());
-    RebalancerRef rebalancerRef = idealState.getRebalancerRef();
-    if (rebalancerRef != null) {
-      builder.rebalancerRef(rebalancerRef);
-    }
-  }
-
-  /**
-   * Builder for a basic data rebalancer context
-   */
-  public static final class Builder extends AbstractBuilder<Builder> {
-    /**
-     * Instantiate with a resource
-     * @param resourceId resource id
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-
-    @Override
-    public PartitionedRebalancerContext build() {
-      PartitionedRebalancerContext context =
-          new PartitionedRebalancerContext(RebalanceMode.USER_DEFINED);
-      super.update(context);
-      return context;
-    }
-  }
-
-  /**
-   * Abstract builder for a generic partitioned resource rebalancer context
-   */
-  public static abstract class AbstractBuilder<T extends BasicRebalancerContext.AbstractBuilder<T>>
-      extends BasicRebalancerContext.AbstractBuilder<T> {
-    private final ResourceId _resourceId;
-    private final Map<PartitionId, Partition> _partitionMap;
-    private boolean _anyLiveParticipant;
-    private int _replicaCount;
-    private int _maxPartitionsPerParticipant;
-
-    /**
-     * Instantiate with a resource
-     * @param resourceId resource id
-     */
-    public AbstractBuilder(ResourceId resourceId) {
-      super(resourceId);
-      _resourceId = resourceId;
-      _partitionMap = Maps.newHashMap();
-      _anyLiveParticipant = false;
-      _replicaCount = 1;
-      _maxPartitionsPerParticipant = Integer.MAX_VALUE;
-    }
-
-    /**
-     * Add a partition that the resource serves
-     * @param partition fully-qualified partition
-     * @return Builder
-     */
-    public T addPartition(Partition partition) {
-      _partitionMap.put(partition.getId(), partition);
-      return self();
-    }
-
-    /**
-     * Add a collection of partitions
-     * @param partitions any collection of Partition objects
-     * @return Builder
-     */
-    public T addPartitions(Collection<Partition> partitions) {
-      for (Partition partition : partitions) {
-        addPartition(partition);
-      }
-      return self();
-    }
-
-    /**
-     * Add a specified number of partitions with a default naming scheme, namely
-     * resourceId_partitionNumber where partitionNumber starts at 0
-     * @param partitionCount number of partitions to add
-     * @return Builder
-     */
-    public T addPartitions(int partitionCount) {
-      for (int i = 0; i < partitionCount; i++) {
-        addPartition(new Partition(PartitionId.from(_resourceId, Integer.toString(i))));
-      }
-      return self();
-    }
-
-    /**
-     * Set whether any live participant should be used in rebalancing
-     * @param anyLiveParticipant true if any live participant can be used, false otherwise
-     * @return Builder
-     */
-    public T anyLiveParticipant(boolean anyLiveParticipant) {
-      _anyLiveParticipant = anyLiveParticipant;
-      return self();
-    }
-
-    /**
-     * Set the number of replicas
-     * @param replicaCount number of replicas
-     * @return Builder
-     */
-    public T replicaCount(int replicaCount) {
-      _replicaCount = replicaCount;
-      return self();
-    }
-
-    /**
-     * Set the maximum number of partitions to assign to any participant
-     * @param maxPartitionsPerParticipant the maximum
-     * @return Builder
-     */
-    public T maxPartitionsPerParticipant(int maxPartitionsPerParticipant) {
-      _maxPartitionsPerParticipant = maxPartitionsPerParticipant;
-      return self();
-    }
-
-    /**
-     * Update a DataRebalancerContext with fields from this builder level
-     * @param context DataRebalancerContext
-     */
-    protected final void update(PartitionedRebalancerContext context) {
-      super.update(context);
-      // enforce at least one partition
-      if (_partitionMap.isEmpty()) {
-        addPartitions(1);
-      }
-      context.setPartitionMap(_partitionMap);
-      context.setAnyLiveParticipant(_anyLiveParticipant);
-      context.setMaxPartitionsPerParticipant(_maxPartitionsPerParticipant);
-      context.setReplicaCount(_replicaCount);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/Rebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/Rebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/Rebalancer.java
deleted file mode 100644
index 6e55e63..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/Rebalancer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-import org.apache.helix.controller.stages.ResourceCurrentState;
-import org.apache.helix.model.ResourceAssignment;
-
-/*
- * 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.
- */
-
-/**
- * Allows one to come up with custom implementation of a rebalancer.<br/>
- * This will be invoked on all changes that happen in the cluster.<br/>
- * Simply return the resource assignment for a resource in this method.<br/>
- */
-public interface Rebalancer {
-
-  public void init(HelixManager helixManager);
-
-  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster,
-      ResourceCurrentState currentState);
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 924b8a1..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerConfig.java
+++ /dev/null
@@ -1,177 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-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;
-
-/*
- * 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 for a resource rebalancer. This contains a RebalancerContext, which contains
- * information specific to each rebalancer.
- */
-public final class RebalancerConfig {
-  private enum Fields {
-    SERIALIZER_CLASS,
-    REBALANCER_CONTEXT,
-    REBALANCER_CONTEXT_CLASS
-  }
-
-  private static final Logger LOG = Logger.getLogger(RebalancerConfig.class);
-  private ContextSerializer _serializer;
-  private Rebalancer _rebalancer;
-  private final RebalancerContext _context;
-  private final NamespacedConfig _config;
-
-  /**
-   * Instantiate a RebalancerConfig
-   * @param context rebalancer context
-   * @param rebalancerRef reference to the rebalancer class that will be used
-   */
-  public RebalancerConfig(RebalancerContext context) {
-    _config =
-        new NamespacedConfig(Scope.resource(context.getResourceId()),
-            RebalancerConfig.class.getSimpleName());
-    _config.setSimpleField(Fields.SERIALIZER_CLASS.toString(), context.getSerializerClass()
-        .getName());
-    _config
-        .setSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString(), context.getClass().getName());
-    _context = context;
-    try {
-      _serializer = context.getSerializerClass().newInstance();
-      _config.setSimpleField(Fields.REBALANCER_CONTEXT.toString(), _serializer.serialize(context));
-    } catch (InstantiationException e) {
-      LOG.error("Error initializing the configuration", e);
-    } catch (IllegalAccessException e) {
-      LOG.error("Error initializing the configuration", e);
-    }
-  }
-
-  /**
-   * Instantiate from a physical ResourceConfiguration
-   * @param resourceConfiguration populated ResourceConfiguration
-   */
-  public RebalancerConfig(ResourceConfiguration resourceConfiguration) {
-    _config = new NamespacedConfig(resourceConfiguration, RebalancerConfig.class.getSimpleName());
-    _serializer = getSerializer();
-    _context = getContext();
-  }
-
-  /**
-   * Get the class that can serialize and deserialize the rebalancer context
-   * @return ContextSerializer
-   */
-  private ContextSerializer getSerializer() {
-    String serializerClassName = _config.getSimpleField(Fields.SERIALIZER_CLASS.toString());
-    if (serializerClassName != null) {
-      try {
-        return (ContextSerializer) HelixUtil.loadClass(getClass(), serializerClassName)
-            .newInstance();
-      } catch (InstantiationException e) {
-        LOG.error("Error getting the serializer", e);
-      } catch (IllegalAccessException e) {
-        LOG.error("Error getting the serializer", e);
-      } catch (ClassNotFoundException e) {
-        LOG.error("Error getting the serializer", e);
-      }
-    }
-    return null;
-  }
-
-  private RebalancerContext getContext() {
-    String className = _config.getSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString());
-    try {
-      Class<? extends RebalancerContext> contextClass =
-          HelixUtil.loadClass(getClass(), className).asSubclass(RebalancerContext.class);
-      String serialized = _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString());
-      return _serializer.deserialize(contextClass, serialized);
-    } catch (ClassNotFoundException e) {
-      LOG.error(className + " is not a valid class");
-    } catch (ClassCastException e) {
-      LOG.error(className + " does not implement RebalancerContext");
-    }
-    return null;
-  }
-
-  /**
-   * Get a rebalancer class instance
-   * @return Rebalancer
-   */
-  public Rebalancer getRebalancer() {
-    // cache the rebalancer to avoid loading and instantiating it excessively
-    if (_rebalancer == null) {
-      if (_context == null || _context.getRebalancerRef() == null) {
-        return null;
-      }
-      _rebalancer = _context.getRebalancerRef().getRebalancer();
-    }
-    return _rebalancer;
-  }
-
-  /**
-   * Get the instantiated RebalancerContext
-   * @param contextClass specific class of the RebalancerContext
-   * @return RebalancerContext subclass instance, or null if conversion is not possible
-   */
-  public <T extends RebalancerContext> T getRebalancerContext(Class<T> contextClass) {
-    try {
-      return contextClass.cast(_context);
-    } catch (ClassCastException e) {
-      LOG.info(contextClass + " is incompatible with context class: " + _context.getClass());
-    }
-    return null;
-  }
-
-  /**
-   * Get the rebalancer context serialized as a string
-   * @return string representing the context
-   */
-  public String getSerializedContext() {
-    return _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString());
-  }
-
-  /**
-   * Convert this to a namespaced config
-   * @return NamespacedConfig
-   */
-  public NamespacedConfig toNamespacedConfig() {
-    return _config;
-  }
-
-  /**
-   * Get a RebalancerConfig from a physical resource config
-   * @param resourceConfiguration physical resource config
-   * @return RebalancerConfig
-   */
-  public static RebalancerConfig from(ResourceConfiguration resourceConfiguration) {
-    return new RebalancerConfig(resourceConfiguration);
-  }
-
-  /**
-   * Get a RebalancerConfig from a RebalancerContext
-   * @param context instantiated RebalancerContext
-   * @return RebalancerConfig
-   */
-  public static RebalancerConfig from(RebalancerContext context) {
-    return new RebalancerConfig(context);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index ea35525..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerContext.java
+++ /dev/null
@@ -1,93 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.api.Partition;
-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
- * 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.
- */
-
-/**
- * Defines the state available to a rebalancer. The most common use case is to use a
- * {@link PartitionedRebalancerContext} or a subclass and set up a resource with it. A rebalancer
- * configuration, at a minimum, is aware of subunits of a resource, the state model to follow, and
- * how the configuration should be serialized.
- */
-public interface RebalancerContext {
-  /**
-   * Get a map of resource partition identifiers to partitions. A partition is a subunit of a
-   * resource, e.g. a subtask of a task
-   * @return map of (subunit id, subunit) pairs
-   */
-  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap();
-
-  /**
-   * Get the subunits of the resource (e.g. partitions)
-   * @return set of subunit ids
-   */
-  public Set<? extends PartitionId> getSubUnitIdSet();
-
-  /**
-   * Get a specific subunit
-   * @param subUnitId the id of the subunit
-   * @return SubUnit
-   */
-  public Partition getSubUnit(PartitionId subUnitId);
-
-  /**
-   * Get the resource to rebalance
-   * @return resource id
-   */
-  public ResourceId getResourceId();
-
-  /**
-   * Get the state model definition that the resource follows
-   * @return state model definition id
-   */
-  public StateModelDefId getStateModelDefId();
-
-  /**
-   * Get the state model factory of this resource
-   * @return state model factory id
-   */
-  public StateModelFactoryId getStateModelFactoryId();
-
-  /**
-   * Get the tag, if any, that participants must have in order to serve this resource
-   * @return participant group tag, or null
-   */
-  public String getParticipantGroupTag();
-
-  /**
-   * Get the serializer for this context
-   * @return ContextSerializer class object
-   */
-  public Class<? extends ContextSerializer> getSerializerClass();
-
-  /**
-   * Get a reference to the class used to rebalance this resource
-   * @return RebalancerRef
-   */
-  public RebalancerRef getRebalancerRef();
-}


[4/9] [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 33eff27..4b33d8c 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
@@ -40,7 +40,7 @@ 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.controller.stages.MessageSelectionStage.Bounds;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
 import org.testng.Assert;
@@ -92,7 +92,7 @@ public class TestMsgSelectionStage {
     stateTransitionPriorities.put("SLAVE-MASTER", 1);
 
     List<Message> selectedMsg =
-        new NewMessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates,
+        new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates,
             messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE"));
 
     Assert.assertEquals(selectedMsg.size(), 1);
@@ -145,7 +145,7 @@ public class TestMsgSelectionStage {
     stateTransitionPriorities.put("SLAVE-MASTER", 1);
 
     List<Message> selectedMsg =
-        new NewMessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates,
+        new MessageSelectionStage().selectMessages(liveInstances, currentStates, pendingStates,
             messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE"));
 
     Assert.assertEquals(selectedMsg.size(), 0);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 892bfa7..4129f66 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
@@ -84,17 +84,17 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     // cluster data cache refresh pipeline
     Pipeline dataRefresh = new Pipeline();
-    dataRefresh.addStage(new NewReadClusterDataStage());
+    dataRefresh.addStage(new ReadClusterDataStage());
 
     // rebalance pipeline
     Pipeline rebalancePipeline = new Pipeline();
-    rebalancePipeline.addStage(new NewResourceComputationStage());
-    rebalancePipeline.addStage(new NewCurrentStateComputationStage());
-    rebalancePipeline.addStage(new NewBestPossibleStateCalcStage());
-    rebalancePipeline.addStage(new NewMessageGenerationStage());
-    rebalancePipeline.addStage(new NewMessageSelectionStage());
-    rebalancePipeline.addStage(new NewMessageThrottleStage());
-    rebalancePipeline.addStage(new NewTaskAssignmentStage());
+    rebalancePipeline.addStage(new ResourceComputationStage());
+    rebalancePipeline.addStage(new CurrentStateComputationStage());
+    rebalancePipeline.addStage(new BestPossibleStateCalcStage());
+    rebalancePipeline.addStage(new MessageGenerationStage());
+    rebalancePipeline.addStage(new MessageSelectionStage());
+    rebalancePipeline.addStage(new MessageThrottleStage());
+    rebalancePipeline.addStage(new TaskAssignmentStage());
 
     // round1: set node0 currentState to OFFLINE and node1 currentState to OFFLINE
     setCurrentState(clusterName, "localhost_0", resourceName, resourceName + "_0", "session_0",
@@ -104,7 +104,7 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     runPipeline(event, dataRefresh);
     runPipeline(event, rebalancePipeline);
-    NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
+    MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
         msgSelOutput.getMessages(ResourceId.from(resourceName),
             PartitionId.from(resourceName + "_0"));
@@ -239,17 +239,17 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     // cluster data cache refresh pipeline
     Pipeline dataRefresh = new Pipeline();
-    dataRefresh.addStage(new NewReadClusterDataStage());
+    dataRefresh.addStage(new ReadClusterDataStage());
 
     // rebalance pipeline
     Pipeline rebalancePipeline = new Pipeline();
-    rebalancePipeline.addStage(new NewResourceComputationStage());
-    rebalancePipeline.addStage(new NewCurrentStateComputationStage());
-    rebalancePipeline.addStage(new NewBestPossibleStateCalcStage());
-    rebalancePipeline.addStage(new NewMessageGenerationStage());
-    rebalancePipeline.addStage(new NewMessageSelectionStage());
-    rebalancePipeline.addStage(new NewMessageThrottleStage());
-    rebalancePipeline.addStage(new NewTaskAssignmentStage());
+    rebalancePipeline.addStage(new ResourceComputationStage());
+    rebalancePipeline.addStage(new CurrentStateComputationStage());
+    rebalancePipeline.addStage(new BestPossibleStateCalcStage());
+    rebalancePipeline.addStage(new MessageGenerationStage());
+    rebalancePipeline.addStage(new MessageSelectionStage());
+    rebalancePipeline.addStage(new MessageThrottleStage());
+    rebalancePipeline.addStage(new TaskAssignmentStage());
 
     // round1: set node0 currentState to OFFLINE and node1 currentState to SLAVE
     setCurrentState(clusterName, "localhost_0", resourceName, resourceName + "_0", "session_0",
@@ -259,7 +259,7 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     runPipeline(event, dataRefresh);
     runPipeline(event, rebalancePipeline);
-    NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
+    MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
         msgSelOutput.getMessages(ResourceId.from(resourceName),
             PartitionId.from(resourceName + "_0"));
@@ -344,17 +344,17 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     // cluster data cache refresh pipeline
     Pipeline dataRefresh = new Pipeline();
-    dataRefresh.addStage(new NewReadClusterDataStage());
+    dataRefresh.addStage(new ReadClusterDataStage());
 
     // rebalance pipeline
     Pipeline rebalancePipeline = new Pipeline();
-    rebalancePipeline.addStage(new NewResourceComputationStage());
-    rebalancePipeline.addStage(new NewCurrentStateComputationStage());
-    rebalancePipeline.addStage(new NewBestPossibleStateCalcStage());
-    rebalancePipeline.addStage(new NewMessageGenerationStage());
-    rebalancePipeline.addStage(new NewMessageSelectionStage());
-    rebalancePipeline.addStage(new NewMessageThrottleStage());
-    rebalancePipeline.addStage(new NewTaskAssignmentStage());
+    rebalancePipeline.addStage(new ResourceComputationStage());
+    rebalancePipeline.addStage(new CurrentStateComputationStage());
+    rebalancePipeline.addStage(new BestPossibleStateCalcStage());
+    rebalancePipeline.addStage(new MessageGenerationStage());
+    rebalancePipeline.addStage(new MessageSelectionStage());
+    rebalancePipeline.addStage(new MessageThrottleStage());
+    rebalancePipeline.addStage(new TaskAssignmentStage());
 
     // round1: set node1 currentState to SLAVE
     setCurrentState(clusterName, "localhost_1", resourceName, resourceName + "_0", "session_1",
@@ -362,7 +362,7 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
 
     runPipeline(event, dataRefresh);
     runPipeline(event, rebalancePipeline);
-    NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
+    MessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
         msgSelOutput.getMessages(ResourceId.from(resourceName),
             PartitionId.from(resourceName + "_0"));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 ab0e4a6..0dba374 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
@@ -32,8 +32,8 @@ 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.api.rebalancer.RebalancerContext;
 import org.apache.helix.controller.pipeline.StageContext;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
 import org.apache.helix.controller.strategy.DefaultTwoStateStrategy;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
@@ -67,8 +67,8 @@ public class TestResourceComputationStage extends BaseStageTest {
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     Builder keyBuilder = accessor.keyBuilder();
     accessor.setProperty(keyBuilder.idealState(resourceName), idealState);
-    NewResourceComputationStage stage = new NewResourceComputationStage();
-    runStage(event, new NewReadClusterDataStage());
+    ResourceComputationStage stage = new ResourceComputationStage();
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
 
     Map<ResourceId, ResourceConfig> resource =
@@ -92,8 +92,8 @@ public class TestResourceComputationStage extends BaseStageTest {
         "testResource1", "testResource2"
     };
     List<IdealState> idealStates = setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO);
-    NewResourceComputationStage stage = new NewResourceComputationStage();
-    runStage(event, new NewReadClusterDataStage());
+    ResourceComputationStage stage = new ResourceComputationStage();
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
 
     Map<ResourceId, ResourceConfig> resourceMap =
@@ -166,8 +166,8 @@ public class TestResourceComputationStage extends BaseStageTest {
     accessor.setProperty(keyBuilder.currentState(instanceName, sessionId, oldResource),
         currentState);
 
-    NewResourceComputationStage stage = new NewResourceComputationStage();
-    runStage(event, new NewReadClusterDataStage());
+    ResourceComputationStage stage = new ResourceComputationStage();
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
 
     Map<ResourceId, ResourceConfig> resourceMap =
@@ -208,7 +208,7 @@ public class TestResourceComputationStage extends BaseStageTest {
   @Test
   public void testNull() {
     ClusterEvent event = new ClusterEvent("sampleEvent");
-    NewResourceComputationStage stage = new NewResourceComputationStage();
+    ResourceComputationStage stage = new ResourceComputationStage();
     StageContext context = new StageContext();
     stage.init(context);
     stage.preProcess();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 8de39ab..50d5ad8 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
@@ -41,7 +41,7 @@ import org.apache.helix.api.id.ClusterId;
 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.rebalancer.util.NewConstraintBasedAssignment;
+import org.apache.helix.api.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.StateModelDefinition;
@@ -228,10 +228,10 @@ public class TestAutoRebalanceStrategy {
         Map<ParticipantId, State> currentStateMap =
             IdealState.participantStateMapFromStringMap(rawCurStateMap);
         Map<State, String> upperBounds =
-            NewConstraintBasedAssignment.stateConstraints(_stateModelDef,
+            ConstraintBasedAssignment.stateConstraints(_stateModelDef,
                 ResourceId.from(RESOURCE_NAME), cluster);
         Map<ParticipantId, State> assignment =
-            NewConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
+            ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
                 liveParticipantSet, _stateModelDef, preferenceList, currentStateMap,
                 disabledParticipantsForPartition);
         mapResult.put(partition, IdealState.stringMapFromParticipantStateMap(assignment));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 813cd71..8620383 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
@@ -46,7 +46,7 @@ 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.api.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
@@ -260,10 +260,10 @@ public class TestNewAutoRebalanceStrategy {
         Map<ParticipantId, State> replicaMap =
             ResourceAssignment.replicaMapFromStringMap(_currentMapping.get(partition));
         Map<State, String> upperBounds =
-            NewConstraintBasedAssignment.stateConstraints(_stateModelDef,
+            ConstraintBasedAssignment.stateConstraints(_stateModelDef,
                 ResourceId.from(RESOURCE_NAME), clusterConfig);
         Map<ParticipantId, State> assignment =
-            NewConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
+            ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
                 liveParticipantMap.keySet(), _stateModelDef, participantPreferenceList, replicaMap,
                 disabledParticipantsForPartition);
         mapResult.put(partitionId, assignment);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 1b7c30c..688846e 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
@@ -32,9 +32,9 @@ import org.apache.helix.api.Cluster;
 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;
+import org.apache.helix.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.Rebalancer;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 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/9f229c80/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 7215707..df81000 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
@@ -21,7 +21,7 @@ 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.api.rebalancer.FullAutoRebalancerContext;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;


[9/9] git commit: [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
[HELIX-209] Shuffling around rebalancer code to allow for compatibility


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

Branch: refs/heads/helix-logical-model
Commit: 9f229c80ca0411c2a1efdf79300115a907751cd6
Parents: 6b57486
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Tue Oct 15 16:49:46 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Tue Oct 15 16:51:14 2013 -0700

----------------------------------------------------------------------
 .../java/org/apache/helix/api/Resource.java     |   4 +-
 .../api/accessor/AtomicResourceAccessor.java    |   2 +-
 .../helix/api/accessor/ClusterAccessor.java     |   4 +-
 .../helix/api/accessor/ParticipantAccessor.java |   4 +-
 .../helix/api/accessor/ResourceAccessor.java    |  10 +-
 .../apache/helix/api/config/ResourceConfig.java |   4 +-
 .../api/rebalancer/BasicRebalancerContext.java  | 239 ++++++++++++
 .../helix/api/rebalancer/ContextSerializer.java |  37 ++
 .../helix/api/rebalancer/CustomRebalancer.java  | 123 ++++++
 .../api/rebalancer/CustomRebalancerContext.java | 161 ++++++++
 .../rebalancer/DefaultContextSerializer.java    |  83 +++++
 .../api/rebalancer/FullAutoRebalancer.java      | 194 ++++++++++
 .../rebalancer/FullAutoRebalancerContext.java   |  61 +++
 .../PartitionedRebalancerContext.java           | 372 +++++++++++++++++++
 .../apache/helix/api/rebalancer/Rebalancer.java |  38 ++
 .../helix/api/rebalancer/RebalancerConfig.java  | 177 +++++++++
 .../helix/api/rebalancer/RebalancerContext.java |  93 +++++
 .../helix/api/rebalancer/RebalancerRef.java     |  94 +++++
 .../rebalancer/ReplicatedRebalancerContext.java |  40 ++
 .../api/rebalancer/SemiAutoRebalancer.java      |  81 ++++
 .../rebalancer/SemiAutoRebalancerContext.java   | 176 +++++++++
 .../util/ConstraintBasedAssignment.java         | 244 ++++++++++++
 .../controller/GenericHelixController.java      |  40 +-
 .../helix/controller/rebalancer/Rebalancer.java |   2 +
 .../context/BasicRebalancerContext.java         | 239 ------------
 .../rebalancer/context/ContextSerializer.java   |  37 --
 .../rebalancer/context/CustomRebalancer.java    | 123 ------
 .../context/CustomRebalancerContext.java        | 161 --------
 .../context/DefaultContextSerializer.java       |  83 -----
 .../rebalancer/context/FullAutoRebalancer.java  | 194 ----------
 .../context/FullAutoRebalancerContext.java      |  61 ---
 .../context/PartitionedRebalancerContext.java   | 372 -------------------
 .../rebalancer/context/Rebalancer.java          |  38 --
 .../rebalancer/context/RebalancerConfig.java    | 177 ---------
 .../rebalancer/context/RebalancerContext.java   |  93 -----
 .../rebalancer/context/RebalancerRef.java       |  94 -----
 .../context/ReplicatedRebalancerContext.java    |  40 --
 .../rebalancer/context/SemiAutoRebalancer.java  |  81 ----
 .../context/SemiAutoRebalancerContext.java      | 176 ---------
 .../util/ConstraintBasedAssignment.java         |   4 +-
 .../util/NewConstraintBasedAssignment.java      | 244 ------------
 .../stages/BestPossibleStateCalcStage.java      | 143 +++----
 .../stages/BestPossibleStateOutput.java         |  91 ++---
 .../stages/CompatibilityCheckStage.java         |  17 +-
 .../stages/CurrentStateComputationStage.java    |  91 ++---
 .../stages/ExternalViewComputeStage.java        | 154 ++++----
 .../stages/MessageGenerationPhase.java          | 215 -----------
 .../stages/MessageGenerationStage.java          | 213 +++++++++++
 .../helix/controller/stages/MessageOutput.java  |  79 ++++
 .../stages/MessageSelectionStage.java           | 129 ++++---
 .../stages/MessageSelectionStageOutput.java     |  60 ---
 .../controller/stages/MessageThrottleStage.java |  41 +-
 .../stages/MessageThrottleStageOutput.java      |  53 ---
 .../stages/NewBestPossibleStateCalcStage.java   | 142 -------
 .../stages/NewBestPossibleStateOutput.java      |  49 ---
 .../stages/NewCompatibilityCheckStage.java      |  68 ----
 .../stages/NewCurrentStateComputationStage.java | 142 -------
 .../stages/NewExternalViewComputeStage.java     | 281 --------------
 .../stages/NewMessageGenerationStage.java       | 213 -----------
 .../controller/stages/NewMessageOutput.java     |  79 ----
 .../stages/NewMessageSelectionStage.java        | 317 ----------------
 .../stages/NewMessageThrottleStage.java         | 198 ----------
 .../stages/NewReadClusterDataStage.java         |  73 ----
 .../stages/NewResourceComputationStage.java     | 138 -------
 .../stages/NewTaskAssignmentStage.java          | 151 --------
 .../stages/PersistAssignmentStage.java          |   2 +-
 .../controller/stages/ReadClusterDataStage.java |  50 +--
 .../controller/stages/ReadHealthDataStage.java  |   2 -
 .../stages/ResourceComputationStage.java        | 164 ++++----
 .../controller/stages/TaskAssignmentStage.java  |  57 +--
 .../java/org/apache/helix/model/IdealState.java |   2 +-
 .../helix/tools/ClusterStateVerifier.java       |  20 +-
 .../org/apache/helix/tools/NewClusterSetup.java |   8 +-
 .../org/apache/helix/api/TestNewStages.java     |  10 +-
 .../org/apache/helix/api/TestUpdateConfig.java  |   4 +-
 .../context/TestSerializeRebalancerContext.java |   3 +
 .../helix/controller/stages/BaseStageTest.java  |   4 +-
 .../TestBestPossibleCalcStageCompatibility.java |  12 +-
 .../stages/TestBestPossibleStateCalcStage.java  |   6 +-
 .../stages/TestCompatibilityCheckStage.java     |  10 +-
 .../TestCurrentStateComputationStage.java       |  12 +-
 .../stages/TestMessageThrottleStage.java        |  20 +-
 .../stages/TestMsgSelectionStage.java           |   6 +-
 .../stages/TestRebalancePipeline.java           |  54 +--
 .../stages/TestResourceComputationStage.java    |  16 +-
 .../strategy/TestAutoRebalanceStrategy.java     |   6 +-
 .../strategy/TestNewAutoRebalanceStrategy.java  |   6 +-
 .../TestCustomizedIdealStateRebalancer.java     |   6 +-
 .../apache/helix/examples/NewModelExample.java  |   2 +-
 89 files changed, 3139 insertions(+), 4980 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 79a1e09..a505aeb 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
@@ -30,8 +30,8 @@ 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.api.id.StateModelDefId;
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
index 6d69981..74b6bdb 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/AtomicResourceAccessor.java
@@ -6,7 +6,7 @@ import org.apache.helix.api.Scope;
 import org.apache.helix.api.config.ResourceConfig;
 import org.apache.helix.api.id.ClusterId;
 import org.apache.helix.api.id.ResourceId;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.lock.HelixLock;
 import org.apache.helix.lock.HelixLockable;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
index 85b8432..ed6c844 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
@@ -51,8 +51,8 @@ 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.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.model.Alerts;
 import org.apache.helix.model.ClusterConfiguration;
 import org.apache.helix.model.ClusterConstraints;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
index 83dd53e..ff24ee9 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
@@ -51,8 +51,8 @@ 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.PartitionedRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
index 517c8c4..b959f17 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
@@ -38,11 +38,11 @@ import org.apache.helix.api.config.UserConfig;
 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.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.api.rebalancer.CustomRebalancerContext;
+import org.apache.helix.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
+import org.apache.helix.api.rebalancer.SemiAutoRebalancerContext;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.IdealState.RebalanceMode;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
index 38d48ab..ec1d8f6 100644
--- 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
@@ -7,8 +7,8 @@ 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 org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/BasicRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/BasicRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/BasicRebalancerContext.java
new file mode 100644
index 0000000..b346cda
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/BasicRebalancerContext.java
@@ -0,0 +1,239 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.Set;
+
+import org.apache.helix.api.Partition;
+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;
+
+/*
+ * 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.
+ */
+
+/**
+ * Abstract RebalancerContext that functions for generic subunits. Use a subclass that more
+ * concretely defines the subunits.
+ */
+public abstract class BasicRebalancerContext implements RebalancerContext {
+  private ResourceId _resourceId;
+  private StateModelDefId _stateModelDefId;
+  private StateModelFactoryId _stateModelFactoryId;
+  private String _participantGroupTag;
+  private Class<? extends ContextSerializer> _serializer;
+  private RebalancerRef _rebalancerRef;
+
+  /**
+   * Instantiate a basic rebalancer context
+   */
+  public BasicRebalancerContext() {
+    _serializer = DefaultContextSerializer.class;
+  }
+
+  @Override
+  public ResourceId getResourceId() {
+    return _resourceId;
+  }
+
+  /**
+   * Set the resource to rebalance
+   * @param resourceId resource id
+   */
+  public void setResourceId(ResourceId resourceId) {
+    _resourceId = resourceId;
+  }
+
+  @Override
+  public StateModelDefId getStateModelDefId() {
+    return _stateModelDefId;
+  }
+
+  /**
+   * Set the state model definition that the resource follows
+   * @param stateModelDefId state model definition id
+   */
+  public void setStateModelDefId(StateModelDefId stateModelDefId) {
+    _stateModelDefId = stateModelDefId;
+  }
+
+  @Override
+  public StateModelFactoryId getStateModelFactoryId() {
+    return _stateModelFactoryId;
+  }
+
+  /**
+   * Set the state model factory that the resource uses
+   * @param stateModelFactoryId state model factory id
+   */
+  public void setStateModelFactoryId(StateModelFactoryId stateModelFactoryId) {
+    _stateModelFactoryId = stateModelFactoryId;
+  }
+
+  @Override
+  public String getParticipantGroupTag() {
+    return _participantGroupTag;
+  }
+
+  /**
+   * Set a tag that participants must have in order to serve this resource
+   * @param participantGroupTag string group tag
+   */
+  public void setParticipantGroupTag(String participantGroupTag) {
+    _participantGroupTag = participantGroupTag;
+  }
+
+  /**
+   * Get the serializer. If none is provided, {@link DefaultContextSerializer} is used
+   */
+  @Override
+  public Class<? extends ContextSerializer> getSerializerClass() {
+    return _serializer;
+  }
+
+  /**
+   * Set the class that can serialize this context
+   * @param serializer serializer class that implements ContextSerializer
+   */
+  public void setSerializerClass(Class<? extends ContextSerializer> serializer) {
+    _serializer = serializer;
+  }
+
+  @Override
+  @JsonIgnore
+  public Set<? extends PartitionId> getSubUnitIdSet() {
+    return getSubUnitMap().keySet();
+  }
+
+  @Override
+  @JsonIgnore
+  public Partition getSubUnit(PartitionId subUnitId) {
+    return getSubUnitMap().get(subUnitId);
+  }
+
+  @Override
+  public RebalancerRef getRebalancerRef() {
+    return _rebalancerRef;
+  }
+
+  /**
+   * Set the reference to the class used to rebalance this resource
+   * @param rebalancerRef RebalancerRef instance
+   */
+  public void setRebalancerRef(RebalancerRef rebalancerRef) {
+    _rebalancerRef = rebalancerRef;
+  }
+
+  /**
+   * Abstract builder for the base rebalancer context
+   */
+  public static abstract class AbstractBuilder<T extends AbstractBuilder<T>> {
+    private final ResourceId _resourceId;
+    private StateModelDefId _stateModelDefId;
+    private StateModelFactoryId _stateModelFactoryId;
+    private String _participantGroupTag;
+    private Class<? extends ContextSerializer> _serializerClass;
+    private RebalancerRef _rebalancerRef;
+
+    /**
+     * Instantiate with a resource id
+     * @param resourceId resource id
+     */
+    public AbstractBuilder(ResourceId resourceId) {
+      _resourceId = resourceId;
+      _serializerClass = DefaultContextSerializer.class;
+    }
+
+    /**
+     * Set the state model definition that the resource should follow
+     * @param stateModelDefId state model definition id
+     * @return Builder
+     */
+    public T stateModelDefId(StateModelDefId stateModelDefId) {
+      _stateModelDefId = stateModelDefId;
+      return self();
+    }
+
+    /**
+     * Set the state model factory that the resource should use
+     * @param stateModelFactoryId state model factory id
+     * @return Builder
+     */
+    public T stateModelFactoryId(StateModelFactoryId stateModelFactoryId) {
+      _stateModelFactoryId = stateModelFactoryId;
+      return self();
+    }
+
+    /**
+     * Set the tag that all participants require in order to serve this resource
+     * @param participantGroupTag the tag
+     * @return Builder
+     */
+    public T participantGroupTag(String participantGroupTag) {
+      _participantGroupTag = participantGroupTag;
+      return self();
+    }
+
+    /**
+     * Set the serializer class for this rebalancer context
+     * @param serializerClass class that implements ContextSerializer
+     * @return Builder
+     */
+    public T serializerClass(Class<? extends ContextSerializer> serializerClass) {
+      _serializerClass = serializerClass;
+      return self();
+    }
+
+    /**
+     * Specify a custom class to use for rebalancing
+     * @param rebalancerRef RebalancerRef instance
+     * @return Builder
+     */
+    public T rebalancerRef(RebalancerRef rebalancerRef) {
+      _rebalancerRef = rebalancerRef;
+      return self();
+    }
+
+    /**
+     * Update an existing context with base fields
+     * @param context derived context
+     */
+    protected final void update(BasicRebalancerContext context) {
+      context.setResourceId(_resourceId);
+      context.setStateModelDefId(_stateModelDefId);
+      context.setStateModelFactoryId(_stateModelFactoryId);
+      context.setParticipantGroupTag(_participantGroupTag);
+      context.setSerializerClass(_serializerClass);
+      context.setRebalancerRef(_rebalancerRef);
+    }
+
+    /**
+     * Get a typed reference to "this" class. Final derived classes should simply return the this
+     * reference.
+     * @return this for the most specific type
+     */
+    protected abstract T self();
+
+    /**
+     * Get the rebalancer context from the built fields
+     * @return RebalancerContext
+     */
+    public abstract RebalancerContext build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/ContextSerializer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/ContextSerializer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/ContextSerializer.java
new file mode 100644
index 0000000..10e3ba9
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/ContextSerializer.java
@@ -0,0 +1,37 @@
+package org.apache.helix.api.rebalancer;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public interface ContextSerializer {
+  /**
+   * Convert a RebalancerContext object instance to a String
+   * @param data instance of the rebalancer context type
+   * @return String representing the object
+   */
+  public <T> String serialize(final T data);
+
+  /**
+   * Convert raw bytes to a generic object instance
+   * @param clazz The class represented by the deserialized string
+   * @param string String representing the object
+   * @return instance of the generic type or null if the conversion failed
+   */
+  public <T> T deserialize(final Class<T> clazz, final String string);
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancer.java
new file mode 100644
index 0000000..c5e4e80
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancer.java
@@ -0,0 +1,123 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
+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.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.stages.ResourceCurrentState;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+/*
+ * 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 CustomRebalancer implements Rebalancer {
+
+  private static final Logger LOG = Logger.getLogger(CustomRebalancer.class);
+
+  @Override
+  public void init(HelixManager helixManager) {
+    // do nothing
+  }
+
+  @Override
+  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
+      Cluster cluster, ResourceCurrentState currentState) {
+    CustomRebalancerContext config =
+        rebalancerConfig.getRebalancerContext(CustomRebalancerContext.class);
+    StateModelDefinition stateModelDef =
+        cluster.getStateModelMap().get(config.getStateModelDefId());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing resource:" + config.getResourceId());
+    }
+    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
+    for (PartitionId partition : config.getPartitionSet()) {
+      Map<ParticipantId, State> currentStateMap =
+          currentState.getCurrentStateMap(config.getResourceId(), partition);
+      Set<ParticipantId> disabledInstancesForPartition =
+          ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
+              partition);
+      Map<ParticipantId, State> bestStateForPartition =
+          computeCustomizedBestStateForPartition(cluster.getLiveParticipantMap().keySet(),
+              stateModelDef, config.getPreferenceMap(partition), currentStateMap,
+              disabledInstancesForPartition);
+      partitionMapping.addReplicaMap(partition, bestStateForPartition);
+    }
+    return partitionMapping;
+  }
+
+  /**
+   * compute best state for resource in CUSTOMIZED rebalancer mode
+   * @param liveParticipantMap
+   * @param stateModelDef
+   * @param preferenceMap
+   * @param currentStateMap
+   * @param disabledParticipantsForPartition
+   * @return
+   */
+  private Map<ParticipantId, State> computeCustomizedBestStateForPartition(
+      Set<ParticipantId> liveParticipantSet, StateModelDefinition stateModelDef,
+      Map<ParticipantId, State> preferenceMap, Map<ParticipantId, State> currentStateMap,
+      Set<ParticipantId> disabledParticipantsForPartition) {
+    Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
+
+    // if the resource is deleted, idealStateMap will be null/empty and
+    // we should drop all resources.
+    if (currentStateMap != null) {
+      for (ParticipantId participantId : currentStateMap.keySet()) {
+        if ((preferenceMap == null || !preferenceMap.containsKey(participantId))
+            && !disabledParticipantsForPartition.contains(participantId)) {
+          // if dropped and not disabled, transit to DROPPED
+          participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED));
+        } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get(
+            participantId).equals(State.from(HelixDefinedState.ERROR)))
+            && disabledParticipantsForPartition.contains(participantId)) {
+          // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
+          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
+        }
+      }
+    }
+
+    // ideal state is deleted
+    if (preferenceMap == null) {
+      return participantStateMap;
+    }
+
+    for (ParticipantId participantId : preferenceMap.keySet()) {
+      boolean notInErrorState =
+          currentStateMap == null || currentStateMap.get(participantId) == null
+              || !currentStateMap.get(participantId).equals(State.from(HelixDefinedState.ERROR));
+
+      if (liveParticipantSet.contains(participantId) && notInErrorState
+          && !disabledParticipantsForPartition.contains(participantId)) {
+        participantStateMap.put(participantId, preferenceMap.get(participantId));
+      }
+    }
+
+    return participantStateMap;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancerContext.java
new file mode 100644
index 0000000..9d5b7e8
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/CustomRebalancerContext.java
@@ -0,0 +1,161 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.api.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
+import org.apache.helix.model.IdealState.RebalanceMode;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.StateModelDefinition;
+import org.codehaus.jackson.annotate.JsonIgnore;
+
+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.
+ */
+
+/**
+ * RebalancerContext for a resource that should be rebalanced in CUSTOMIZED mode. By default, it
+ * corresponds to {@link CustomRebalancer}
+ */
+public class CustomRebalancerContext extends PartitionedRebalancerContext {
+  private Map<PartitionId, Map<ParticipantId, State>> _preferenceMaps;
+
+  /**
+   * Instantiate a CustomRebalancerContext
+   */
+  public CustomRebalancerContext() {
+    super(RebalanceMode.CUSTOMIZED);
+    setRebalancerRef(RebalancerRef.from(CustomRebalancer.class));
+    _preferenceMaps = Maps.newHashMap();
+  }
+
+  /**
+   * Get the preference maps of the partitions and replicas of the resource
+   * @return map of partition to participant and state
+   */
+  public Map<PartitionId, Map<ParticipantId, State>> getPreferenceMaps() {
+    return _preferenceMaps;
+  }
+
+  /**
+   * Set the preference maps of the partitions and replicas of the resource
+   * @param preferenceMaps map of partition to participant and state
+   */
+  public void setPreferenceMaps(Map<PartitionId, Map<ParticipantId, State>> preferenceMaps) {
+    _preferenceMaps = preferenceMaps;
+  }
+
+  /**
+   * Get the preference map of a partition
+   * @param partitionId the partition to look up
+   * @return map of participant to state
+   */
+  @JsonIgnore
+  public Map<ParticipantId, State> getPreferenceMap(PartitionId partitionId) {
+    return _preferenceMaps.get(partitionId);
+  }
+
+  /**
+   * Generate preference maps based on a default cluster setup
+   * @param stateModelDef the state model definition to follow
+   * @param participantSet the set of participant ids to configure for
+   */
+  @Override
+  @JsonIgnore
+  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
+      Set<ParticipantId> participantSet) {
+    // compute default upper bounds
+    Map<State, String> upperBounds = Maps.newHashMap();
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
+      upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
+    }
+
+    // determine the current mapping
+    Map<PartitionId, Map<ParticipantId, State>> currentMapping = getPreferenceMaps();
+
+    // determine the preference maps
+    LinkedHashMap<State, Integer> stateCounts =
+        ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(),
+            getReplicaCount());
+    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
+    List<ParticipantId> participantList = new ArrayList<ParticipantId>(participantSet);
+    List<PartitionId> partitionList = new ArrayList<PartitionId>(getPartitionSet());
+    AutoRebalanceStrategy strategy =
+        new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts,
+            getMaxPartitionsPerParticipant(), placementScheme);
+    Map<String, Map<String, String>> rawPreferenceMaps =
+        strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList)
+            .getMapFields();
+    Map<PartitionId, Map<ParticipantId, State>> preferenceMaps =
+        Maps.newHashMap(ResourceAssignment.replicaMapsFromStringMaps(rawPreferenceMaps));
+    setPreferenceMaps(preferenceMaps);
+  }
+
+  /**
+   * Build a CustomRebalancerContext. By default, it corresponds to {@link CustomRebalancer}
+   */
+  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
+    private final Map<PartitionId, Map<ParticipantId, State>> _preferenceMaps;
+
+    /**
+     * Instantiate for a resource
+     * @param resourceId resource id
+     */
+    public Builder(ResourceId resourceId) {
+      super(resourceId);
+      super.rebalancerRef(RebalancerRef.from(CustomRebalancer.class));
+      _preferenceMaps = Maps.newHashMap();
+    }
+
+    /**
+     * Add a preference map for a partition
+     * @param partitionId partition to set
+     * @param preferenceList map of participant id to state indicating where replicas are served
+     * @return Builder
+     */
+    public Builder preferenceMap(PartitionId partitionId, Map<ParticipantId, State> preferenceMap) {
+      _preferenceMaps.put(partitionId, preferenceMap);
+      return self();
+    }
+
+    @Override
+    protected Builder self() {
+      return this;
+    }
+
+    @Override
+    public CustomRebalancerContext build() {
+      CustomRebalancerContext context = new CustomRebalancerContext();
+      super.update(context);
+      context.setPreferenceMaps(_preferenceMaps);
+      return context;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/DefaultContextSerializer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/DefaultContextSerializer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/DefaultContextSerializer.java
new file mode 100644
index 0000000..64e1f8b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/DefaultContextSerializer.java
@@ -0,0 +1,83 @@
+package org.apache.helix.api.rebalancer;
+
+import java.io.ByteArrayInputStream;
+import java.io.StringWriter;
+
+import org.apache.helix.HelixException;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+
+/*
+ * 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.
+ */
+
+/**
+ * Default serializer implementation for RebalancerContexts. Uses the Jackson JSON library to
+ * convert to and from strings
+ */
+public class DefaultContextSerializer implements ContextSerializer {
+
+  private static Logger logger = Logger.getLogger(DefaultContextSerializer.class);
+
+  @Override
+  public <T> String serialize(final T data) {
+    if (data == null) {
+      return null;
+    }
+
+    ObjectMapper mapper = new ObjectMapper();
+    SerializationConfig serializationConfig = mapper.getSerializationConfig();
+    serializationConfig.set(SerializationConfig.Feature.INDENT_OUTPUT, true);
+    serializationConfig.set(SerializationConfig.Feature.AUTO_DETECT_FIELDS, true);
+    serializationConfig.set(SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+    StringWriter sw = new StringWriter();
+    try {
+      mapper.writeValue(sw, data);
+    } catch (Exception e) {
+      logger.error("Exception during payload data serialization.", e);
+      throw new HelixException(e);
+    }
+    return sw.toString();
+  }
+
+  @Override
+  public <T> T deserialize(final Class<T> clazz, final String string) {
+    if (string == null || string.length() == 0) {
+      return null;
+    }
+
+    ObjectMapper mapper = new ObjectMapper();
+    ByteArrayInputStream bais = new ByteArrayInputStream(string.getBytes());
+
+    DeserializationConfig deserializationConfig = mapper.getDeserializationConfig();
+    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_FIELDS, true);
+    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_SETTERS, true);
+    deserializationConfig.set(DeserializationConfig.Feature.AUTO_DETECT_CREATORS, true);
+    deserializationConfig.set(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, true);
+    deserializationConfig.set(DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+    try {
+      T payload = mapper.readValue(bais, clazz);
+      return payload;
+    } catch (Exception e) {
+      logger.error("Exception during deserialization of payload bytes: " + string, e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancer.java
new file mode 100644
index 0000000..f5a5abe
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancer.java
@@ -0,0 +1,194 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.State;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.stages.ResourceCurrentState;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
+import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
+/*
+ * 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 FullAutoRebalancer implements Rebalancer {
+  // These should be final, but are initialized in init rather than a constructor
+  private AutoRebalanceStrategy _algorithm;
+
+  private static Logger LOG = Logger.getLogger(FullAutoRebalancer.class);
+
+  @Override
+  public void init(HelixManager helixManager) {
+    // do nothing
+  }
+
+  @Override
+  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
+      Cluster cluster, ResourceCurrentState currentState) {
+    FullAutoRebalancerContext config =
+        rebalancerConfig.getRebalancerContext(FullAutoRebalancerContext.class);
+    StateModelDefinition stateModelDef =
+        cluster.getStateModelMap().get(config.getStateModelDefId());
+    // Compute a preference list based on the current ideal state
+    List<PartitionId> partitions = new ArrayList<PartitionId>(config.getPartitionSet());
+    Map<ParticipantId, Participant> liveParticipants = cluster.getLiveParticipantMap();
+    Map<ParticipantId, Participant> allParticipants = cluster.getParticipantMap();
+    int replicas = -1;
+    if (config.anyLiveParticipant()) {
+      replicas = liveParticipants.size();
+    } else {
+      replicas = config.getReplicaCount();
+    }
+
+    // count how many replicas should be in each state
+    Map<State, String> upperBounds =
+        ConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(),
+            cluster.getConfig());
+    LinkedHashMap<State, Integer> stateCountMap =
+        ConstraintBasedAssignment.stateCount(upperBounds, stateModelDef,
+            liveParticipants.size(), replicas);
+
+    // get the participant lists
+    List<ParticipantId> liveParticipantList =
+        new ArrayList<ParticipantId>(liveParticipants.keySet());
+    List<ParticipantId> allParticipantList =
+        new ArrayList<ParticipantId>(cluster.getParticipantMap().keySet());
+
+    // compute the current mapping from the current state
+    Map<PartitionId, Map<ParticipantId, State>> currentMapping =
+        currentMapping(config, currentState, stateCountMap);
+
+    // If there are nodes tagged with resource, use only those nodes
+    Set<ParticipantId> taggedNodes = new HashSet<ParticipantId>();
+    if (config.getParticipantGroupTag() != null) {
+      for (ParticipantId participantId : liveParticipantList) {
+        if (liveParticipants.get(participantId).hasTag(config.getParticipantGroupTag())) {
+          taggedNodes.add(participantId);
+        }
+      }
+    }
+    if (taggedNodes.size() > 0) {
+      if (LOG.isInfoEnabled()) {
+        LOG.info("found the following instances with tag " + config.getResourceId() + " "
+            + taggedNodes);
+      }
+      liveParticipantList = new ArrayList<ParticipantId>(taggedNodes);
+    }
+
+    // determine which nodes the replicas should live on
+    int maxPartition = config.getMaxPartitionsPerParticipant();
+    if (LOG.isInfoEnabled()) {
+      LOG.info("currentMapping: " + currentMapping);
+      LOG.info("stateCountMap: " + stateCountMap);
+      LOG.info("liveNodes: " + liveParticipantList);
+      LOG.info("allNodes: " + allParticipantList);
+      LOG.info("maxPartition: " + maxPartition);
+    }
+    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
+    _algorithm =
+        new AutoRebalanceStrategy(config.getResourceId(), partitions, stateCountMap, maxPartition,
+            placementScheme);
+    ZNRecord newMapping =
+        _algorithm.typedComputePartitionAssignment(liveParticipantList, currentMapping,
+            allParticipantList);
+
+    if (LOG.isInfoEnabled()) {
+      LOG.info("newMapping: " + newMapping);
+    }
+
+    // compute a full partition mapping for the resource
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing resource:" + config.getResourceId());
+    }
+    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
+    for (PartitionId partition : partitions) {
+      Set<ParticipantId> disabledParticipantsForPartition =
+          ConstraintBasedAssignment.getDisabledParticipants(allParticipants, partition);
+      List<String> rawPreferenceList = newMapping.getListField(partition.stringify());
+      if (rawPreferenceList == null) {
+        rawPreferenceList = Collections.emptyList();
+      }
+      List<ParticipantId> preferenceList =
+          Lists.transform(rawPreferenceList, new Function<String, ParticipantId>() {
+            @Override
+            public ParticipantId apply(String participantName) {
+              return ParticipantId.from(participantName);
+            }
+          });
+      preferenceList =
+          ConstraintBasedAssignment.getPreferenceList(cluster, partition, preferenceList);
+      Map<ParticipantId, State> bestStateForPartition =
+          ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
+              liveParticipants.keySet(), stateModelDef, preferenceList,
+              currentState.getCurrentStateMap(config.getResourceId(), partition),
+              disabledParticipantsForPartition);
+      partitionMapping.addReplicaMap(partition, bestStateForPartition);
+    }
+    return partitionMapping;
+  }
+
+  private Map<PartitionId, Map<ParticipantId, State>> currentMapping(
+      FullAutoRebalancerContext config, ResourceCurrentState currentStateOutput,
+      Map<State, Integer> stateCountMap) {
+    Map<PartitionId, Map<ParticipantId, State>> map =
+        new HashMap<PartitionId, Map<ParticipantId, State>>();
+
+    for (PartitionId partition : config.getPartitionSet()) {
+      Map<ParticipantId, State> curStateMap =
+          currentStateOutput.getCurrentStateMap(config.getResourceId(), partition);
+      map.put(partition, new HashMap<ParticipantId, State>());
+      for (ParticipantId node : curStateMap.keySet()) {
+        State state = curStateMap.get(node);
+        if (stateCountMap.containsKey(state)) {
+          map.get(partition).put(node, state);
+        }
+      }
+
+      Map<ParticipantId, State> pendingStateMap =
+          currentStateOutput.getPendingStateMap(config.getResourceId(), partition);
+      for (ParticipantId node : pendingStateMap.keySet()) {
+        State state = pendingStateMap.get(node);
+        if (stateCountMap.containsKey(state)) {
+          map.get(partition).put(node, state);
+        }
+      }
+    }
+    return map;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancerContext.java
new file mode 100644
index 0000000..6a3fe10
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/FullAutoRebalancerContext.java
@@ -0,0 +1,61 @@
+package org.apache.helix.api.rebalancer;
+
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.model.IdealState.RebalanceMode;
+
+/*
+ * 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.
+ */
+
+/**
+ * RebalancerContext for FULL_AUTO rebalancing mode. By default, it corresponds to
+ * {@link FullAutoRebalancer}
+ */
+public class FullAutoRebalancerContext extends PartitionedRebalancerContext {
+  public FullAutoRebalancerContext() {
+    super(RebalanceMode.FULL_AUTO);
+    setRebalancerRef(RebalancerRef.from(FullAutoRebalancer.class));
+  }
+
+  /**
+   * Builder for a full auto rebalancer context. By default, it corresponds to
+   * {@link FullAutoRebalancer}
+   */
+  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
+    /**
+     * Instantiate with a resource
+     * @param resourceId resource id
+     */
+    public Builder(ResourceId resourceId) {
+      super(resourceId);
+      super.rebalancerRef(RebalancerRef.from(FullAutoRebalancer.class));
+    }
+
+    @Override
+    protected Builder self() {
+      return this;
+    }
+
+    @Override
+    public FullAutoRebalancerContext build() {
+      FullAutoRebalancerContext context = new FullAutoRebalancerContext();
+      super.update(context);
+      return context;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/PartitionedRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/PartitionedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/PartitionedRebalancerContext.java
new file mode 100644
index 0000000..311683d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/PartitionedRebalancerContext.java
@@ -0,0 +1,372 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixConstants.StateModelToken;
+import org.apache.helix.api.Partition;
+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.apache.helix.model.StateModelDefinition;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+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.
+ */
+
+/**
+ * RebalancerContext for a resource whose subunits are partitions. In addition, these partitions can
+ * be replicated.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class PartitionedRebalancerContext extends BasicRebalancerContext implements
+    ReplicatedRebalancerContext {
+  private Map<PartitionId, Partition> _partitionMap;
+  private boolean _anyLiveParticipant;
+  private int _replicaCount;
+  private int _maxPartitionsPerParticipant;
+  private final RebalanceMode _rebalanceMode;
+
+  /**
+   * Instantiate a DataRebalancerContext
+   */
+  public PartitionedRebalancerContext(RebalanceMode rebalanceMode) {
+    _partitionMap = Collections.emptyMap();
+    _replicaCount = 1;
+    _anyLiveParticipant = false;
+    _maxPartitionsPerParticipant = Integer.MAX_VALUE;
+    _rebalanceMode = rebalanceMode;
+  }
+
+  /**
+   * Get a map from partition id to partition
+   * @return partition map (mutable)
+   */
+  public Map<PartitionId, Partition> getPartitionMap() {
+    return _partitionMap;
+  }
+
+  /**
+   * Set a map of partition id to partition
+   * @param partitionMap partition map
+   */
+  public void setPartitionMap(Map<PartitionId, Partition> partitionMap) {
+    _partitionMap = Maps.newHashMap(partitionMap);
+  }
+
+  /**
+   * Get the set of partitions for this resource
+   * @return set of partition ids
+   */
+  @JsonIgnore
+  public Set<PartitionId> getPartitionSet() {
+    return _partitionMap.keySet();
+  }
+
+  /**
+   * Get a partition
+   * @param partitionId id of the partition to get
+   * @return Partition object, or null if not present
+   */
+  @JsonIgnore
+  public Partition getPartition(PartitionId partitionId) {
+    return _partitionMap.get(partitionId);
+  }
+
+  @Override
+  public boolean anyLiveParticipant() {
+    return _anyLiveParticipant;
+  }
+
+  /**
+   * Indicate if this resource should be assigned to any live participant
+   * @param anyLiveParticipant true if any live participant expected, false otherwise
+   */
+  public void setAnyLiveParticipant(boolean anyLiveParticipant) {
+    _anyLiveParticipant = anyLiveParticipant;
+  }
+
+  @Override
+  public int getReplicaCount() {
+    return _replicaCount;
+  }
+
+  /**
+   * Set the number of replicas that each partition should have
+   * @param replicaCount
+   */
+  public void setReplicaCount(int replicaCount) {
+    _replicaCount = replicaCount;
+  }
+
+  /**
+   * Get the maximum number of partitions that a participant can serve
+   * @return maximum number of partitions per participant
+   */
+  public int getMaxPartitionsPerParticipant() {
+    return _maxPartitionsPerParticipant;
+  }
+
+  /**
+   * Set the maximum number of partitions that a participant can serve
+   * @param maxPartitionsPerParticipant maximum number of partitions per participant
+   */
+  public void setMaxPartitionsPerParticipant(int maxPartitionsPerParticipant) {
+    _maxPartitionsPerParticipant = maxPartitionsPerParticipant;
+  }
+
+  /**
+   * Get the rebalancer mode of the resource
+   * @return RebalanceMode
+   */
+  public RebalanceMode getRebalanceMode() {
+    return _rebalanceMode;
+  }
+
+  @Override
+  @JsonIgnore
+  public Map<PartitionId, Partition> getSubUnitMap() {
+    return getPartitionMap();
+  }
+
+  /**
+   * Generate a default configuration given the state model and a participant.
+   * @param stateModelDef the state model definition to follow
+   * @param participantSet the set of participant ids to configure for
+   */
+  @JsonIgnore
+  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
+      Set<ParticipantId> participantSet) {
+    // the base context does not understand enough to know do to anything
+  }
+
+  /**
+   * Convert a physically-stored IdealState into a rebalancer context for a partitioned resource
+   * @param idealState populated IdealState
+   * @return PartitionedRebalancerContext
+   */
+  public static PartitionedRebalancerContext from(IdealState idealState) {
+    PartitionedRebalancerContext context;
+    switch (idealState.getRebalanceMode()) {
+    case FULL_AUTO:
+      FullAutoRebalancerContext.Builder fullAutoBuilder =
+          new FullAutoRebalancerContext.Builder(idealState.getResourceId());
+      populateContext(fullAutoBuilder, idealState);
+      context = fullAutoBuilder.build();
+      break;
+    case SEMI_AUTO:
+      SemiAutoRebalancerContext.Builder semiAutoBuilder =
+          new SemiAutoRebalancerContext.Builder(idealState.getResourceId());
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
+        semiAutoBuilder.preferenceList(partitionId, idealState.getPreferenceList(partitionId));
+      }
+      populateContext(semiAutoBuilder, idealState);
+      context = semiAutoBuilder.build();
+      break;
+    case CUSTOMIZED:
+      CustomRebalancerContext.Builder customBuilder =
+          new CustomRebalancerContext.Builder(idealState.getResourceId());
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
+        customBuilder.preferenceMap(partitionId, idealState.getParticipantStateMap(partitionId));
+      }
+      populateContext(customBuilder, idealState);
+      context = customBuilder.build();
+      break;
+    default:
+      Builder baseBuilder = new Builder(idealState.getResourceId());
+      populateContext(baseBuilder, idealState);
+      context = baseBuilder.build();
+      break;
+    }
+    return context;
+  }
+
+  /**
+   * Update a builder subclass with all the fields of the ideal state
+   * @param builder builder that extends AbstractBuilder
+   * @param idealState populated IdealState
+   */
+  private static <T extends AbstractBuilder<T>> void populateContext(T builder,
+      IdealState idealState) {
+    String replicas = idealState.getReplicas();
+    int replicaCount = 0;
+    boolean anyLiveParticipant = false;
+    if (replicas.equals(StateModelToken.ANY_LIVEINSTANCE.toString())) {
+      anyLiveParticipant = true;
+    } else {
+      replicaCount = Integer.parseInt(replicas);
+    }
+    if (idealState.getNumPartitions() > 0 && idealState.getPartitionIdSet().size() == 0) {
+      // backwards compatibility: partition sets were based on pref lists/maps previously
+      builder.addPartitions(idealState.getNumPartitions());
+    } else {
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
+        builder.addPartition(new Partition(partitionId));
+      }
+    }
+    builder.anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
+        .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
+        .participantGroupTag(idealState.getInstanceGroupTag())
+        .stateModelDefId(idealState.getStateModelDefId())
+        .stateModelFactoryId(idealState.getStateModelFactoryId());
+    RebalancerRef rebalancerRef = idealState.getRebalancerRef();
+    if (rebalancerRef != null) {
+      builder.rebalancerRef(rebalancerRef);
+    }
+  }
+
+  /**
+   * Builder for a basic data rebalancer context
+   */
+  public static final class Builder extends AbstractBuilder<Builder> {
+    /**
+     * Instantiate with a resource
+     * @param resourceId resource id
+     */
+    public Builder(ResourceId resourceId) {
+      super(resourceId);
+    }
+
+    @Override
+    protected Builder self() {
+      return this;
+    }
+
+    @Override
+    public PartitionedRebalancerContext build() {
+      PartitionedRebalancerContext context =
+          new PartitionedRebalancerContext(RebalanceMode.USER_DEFINED);
+      super.update(context);
+      return context;
+    }
+  }
+
+  /**
+   * Abstract builder for a generic partitioned resource rebalancer context
+   */
+  public static abstract class AbstractBuilder<T extends BasicRebalancerContext.AbstractBuilder<T>>
+      extends BasicRebalancerContext.AbstractBuilder<T> {
+    private final ResourceId _resourceId;
+    private final Map<PartitionId, Partition> _partitionMap;
+    private boolean _anyLiveParticipant;
+    private int _replicaCount;
+    private int _maxPartitionsPerParticipant;
+
+    /**
+     * Instantiate with a resource
+     * @param resourceId resource id
+     */
+    public AbstractBuilder(ResourceId resourceId) {
+      super(resourceId);
+      _resourceId = resourceId;
+      _partitionMap = Maps.newHashMap();
+      _anyLiveParticipant = false;
+      _replicaCount = 1;
+      _maxPartitionsPerParticipant = Integer.MAX_VALUE;
+    }
+
+    /**
+     * Add a partition that the resource serves
+     * @param partition fully-qualified partition
+     * @return Builder
+     */
+    public T addPartition(Partition partition) {
+      _partitionMap.put(partition.getId(), partition);
+      return self();
+    }
+
+    /**
+     * Add a collection of partitions
+     * @param partitions any collection of Partition objects
+     * @return Builder
+     */
+    public T addPartitions(Collection<Partition> partitions) {
+      for (Partition partition : partitions) {
+        addPartition(partition);
+      }
+      return self();
+    }
+
+    /**
+     * Add a specified number of partitions with a default naming scheme, namely
+     * resourceId_partitionNumber where partitionNumber starts at 0
+     * @param partitionCount number of partitions to add
+     * @return Builder
+     */
+    public T addPartitions(int partitionCount) {
+      for (int i = 0; i < partitionCount; i++) {
+        addPartition(new Partition(PartitionId.from(_resourceId, Integer.toString(i))));
+      }
+      return self();
+    }
+
+    /**
+     * Set whether any live participant should be used in rebalancing
+     * @param anyLiveParticipant true if any live participant can be used, false otherwise
+     * @return Builder
+     */
+    public T anyLiveParticipant(boolean anyLiveParticipant) {
+      _anyLiveParticipant = anyLiveParticipant;
+      return self();
+    }
+
+    /**
+     * Set the number of replicas
+     * @param replicaCount number of replicas
+     * @return Builder
+     */
+    public T replicaCount(int replicaCount) {
+      _replicaCount = replicaCount;
+      return self();
+    }
+
+    /**
+     * Set the maximum number of partitions to assign to any participant
+     * @param maxPartitionsPerParticipant the maximum
+     * @return Builder
+     */
+    public T maxPartitionsPerParticipant(int maxPartitionsPerParticipant) {
+      _maxPartitionsPerParticipant = maxPartitionsPerParticipant;
+      return self();
+    }
+
+    /**
+     * Update a DataRebalancerContext with fields from this builder level
+     * @param context DataRebalancerContext
+     */
+    protected final void update(PartitionedRebalancerContext context) {
+      super.update(context);
+      // enforce at least one partition
+      if (_partitionMap.isEmpty()) {
+        addPartitions(1);
+      }
+      context.setPartitionMap(_partitionMap);
+      context.setAnyLiveParticipant(_anyLiveParticipant);
+      context.setMaxPartitionsPerParticipant(_maxPartitionsPerParticipant);
+      context.setReplicaCount(_replicaCount);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/Rebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/Rebalancer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/Rebalancer.java
new file mode 100644
index 0000000..e164920
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/Rebalancer.java
@@ -0,0 +1,38 @@
+package org.apache.helix.api.rebalancer;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
+import org.apache.helix.controller.stages.ResourceCurrentState;
+import org.apache.helix.model.ResourceAssignment;
+
+/*
+ * 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.
+ */
+
+/**
+ * Allows one to come up with custom implementation of a rebalancer.<br/>
+ * This will be invoked on all changes that happen in the cluster.<br/>
+ * Simply return the resource assignment for a resource in this method.<br/>
+ */
+public interface Rebalancer {
+
+  public void init(HelixManager helixManager);
+
+  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster,
+      ResourceCurrentState currentState);
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerConfig.java
new file mode 100644
index 0000000..2c57a29
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerConfig.java
@@ -0,0 +1,177 @@
+package org.apache.helix.api.rebalancer;
+
+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;
+
+/*
+ * 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 for a resource rebalancer. This contains a RebalancerContext, which contains
+ * information specific to each rebalancer.
+ */
+public final class RebalancerConfig {
+  private enum Fields {
+    SERIALIZER_CLASS,
+    REBALANCER_CONTEXT,
+    REBALANCER_CONTEXT_CLASS
+  }
+
+  private static final Logger LOG = Logger.getLogger(RebalancerConfig.class);
+  private ContextSerializer _serializer;
+  private Rebalancer _rebalancer;
+  private final RebalancerContext _context;
+  private final NamespacedConfig _config;
+
+  /**
+   * Instantiate a RebalancerConfig
+   * @param context rebalancer context
+   * @param rebalancerRef reference to the rebalancer class that will be used
+   */
+  public RebalancerConfig(RebalancerContext context) {
+    _config =
+        new NamespacedConfig(Scope.resource(context.getResourceId()),
+            RebalancerConfig.class.getSimpleName());
+    _config.setSimpleField(Fields.SERIALIZER_CLASS.toString(), context.getSerializerClass()
+        .getName());
+    _config
+        .setSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString(), context.getClass().getName());
+    _context = context;
+    try {
+      _serializer = context.getSerializerClass().newInstance();
+      _config.setSimpleField(Fields.REBALANCER_CONTEXT.toString(), _serializer.serialize(context));
+    } catch (InstantiationException e) {
+      LOG.error("Error initializing the configuration", e);
+    } catch (IllegalAccessException e) {
+      LOG.error("Error initializing the configuration", e);
+    }
+  }
+
+  /**
+   * Instantiate from a physical ResourceConfiguration
+   * @param resourceConfiguration populated ResourceConfiguration
+   */
+  public RebalancerConfig(ResourceConfiguration resourceConfiguration) {
+    _config = new NamespacedConfig(resourceConfiguration, RebalancerConfig.class.getSimpleName());
+    _serializer = getSerializer();
+    _context = getContext();
+  }
+
+  /**
+   * Get the class that can serialize and deserialize the rebalancer context
+   * @return ContextSerializer
+   */
+  private ContextSerializer getSerializer() {
+    String serializerClassName = _config.getSimpleField(Fields.SERIALIZER_CLASS.toString());
+    if (serializerClassName != null) {
+      try {
+        return (ContextSerializer) HelixUtil.loadClass(getClass(), serializerClassName)
+            .newInstance();
+      } catch (InstantiationException e) {
+        LOG.error("Error getting the serializer", e);
+      } catch (IllegalAccessException e) {
+        LOG.error("Error getting the serializer", e);
+      } catch (ClassNotFoundException e) {
+        LOG.error("Error getting the serializer", e);
+      }
+    }
+    return null;
+  }
+
+  private RebalancerContext getContext() {
+    String className = _config.getSimpleField(Fields.REBALANCER_CONTEXT_CLASS.toString());
+    try {
+      Class<? extends RebalancerContext> contextClass =
+          HelixUtil.loadClass(getClass(), className).asSubclass(RebalancerContext.class);
+      String serialized = _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString());
+      return _serializer.deserialize(contextClass, serialized);
+    } catch (ClassNotFoundException e) {
+      LOG.error(className + " is not a valid class");
+    } catch (ClassCastException e) {
+      LOG.error(className + " does not implement RebalancerContext");
+    }
+    return null;
+  }
+
+  /**
+   * Get a rebalancer class instance
+   * @return Rebalancer
+   */
+  public Rebalancer getRebalancer() {
+    // cache the rebalancer to avoid loading and instantiating it excessively
+    if (_rebalancer == null) {
+      if (_context == null || _context.getRebalancerRef() == null) {
+        return null;
+      }
+      _rebalancer = _context.getRebalancerRef().getRebalancer();
+    }
+    return _rebalancer;
+  }
+
+  /**
+   * Get the instantiated RebalancerContext
+   * @param contextClass specific class of the RebalancerContext
+   * @return RebalancerContext subclass instance, or null if conversion is not possible
+   */
+  public <T extends RebalancerContext> T getRebalancerContext(Class<T> contextClass) {
+    try {
+      return contextClass.cast(_context);
+    } catch (ClassCastException e) {
+      LOG.info(contextClass + " is incompatible with context class: " + _context.getClass());
+    }
+    return null;
+  }
+
+  /**
+   * Get the rebalancer context serialized as a string
+   * @return string representing the context
+   */
+  public String getSerializedContext() {
+    return _config.getSimpleField(Fields.REBALANCER_CONTEXT.toString());
+  }
+
+  /**
+   * Convert this to a namespaced config
+   * @return NamespacedConfig
+   */
+  public NamespacedConfig toNamespacedConfig() {
+    return _config;
+  }
+
+  /**
+   * Get a RebalancerConfig from a physical resource config
+   * @param resourceConfiguration physical resource config
+   * @return RebalancerConfig
+   */
+  public static RebalancerConfig from(ResourceConfiguration resourceConfiguration) {
+    return new RebalancerConfig(resourceConfiguration);
+  }
+
+  /**
+   * Get a RebalancerConfig from a RebalancerContext
+   * @param context instantiated RebalancerContext
+   * @return RebalancerConfig
+   */
+  public static RebalancerConfig from(RebalancerContext context) {
+    return new RebalancerConfig(context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerContext.java
new file mode 100644
index 0000000..80f6b06
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerContext.java
@@ -0,0 +1,93 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.api.Partition;
+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
+ * 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.
+ */
+
+/**
+ * Defines the state available to a rebalancer. The most common use case is to use a
+ * {@link PartitionedRebalancerContext} or a subclass and set up a resource with it. A rebalancer
+ * configuration, at a minimum, is aware of subunits of a resource, the state model to follow, and
+ * how the configuration should be serialized.
+ */
+public interface RebalancerContext {
+  /**
+   * Get a map of resource partition identifiers to partitions. A partition is a subunit of a
+   * resource, e.g. a subtask of a task
+   * @return map of (subunit id, subunit) pairs
+   */
+  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap();
+
+  /**
+   * Get the subunits of the resource (e.g. partitions)
+   * @return set of subunit ids
+   */
+  public Set<? extends PartitionId> getSubUnitIdSet();
+
+  /**
+   * Get a specific subunit
+   * @param subUnitId the id of the subunit
+   * @return SubUnit
+   */
+  public Partition getSubUnit(PartitionId subUnitId);
+
+  /**
+   * Get the resource to rebalance
+   * @return resource id
+   */
+  public ResourceId getResourceId();
+
+  /**
+   * Get the state model definition that the resource follows
+   * @return state model definition id
+   */
+  public StateModelDefId getStateModelDefId();
+
+  /**
+   * Get the state model factory of this resource
+   * @return state model factory id
+   */
+  public StateModelFactoryId getStateModelFactoryId();
+
+  /**
+   * Get the tag, if any, that participants must have in order to serve this resource
+   * @return participant group tag, or null
+   */
+  public String getParticipantGroupTag();
+
+  /**
+   * Get the serializer for this context
+   * @return ContextSerializer class object
+   */
+  public Class<? extends ContextSerializer> getSerializerClass();
+
+  /**
+   * Get a reference to the class used to rebalance this resource
+   * @return RebalancerRef
+   */
+  public RebalancerRef getRebalancerRef();
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerRef.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerRef.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerRef.java
new file mode 100644
index 0000000..83ae589
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/RebalancerRef.java
@@ -0,0 +1,94 @@
+package org.apache.helix.api.rebalancer;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import org.apache.helix.util.HelixUtil;
+import org.apache.log4j.Logger;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonIgnore;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * Reference to a class that extends {@link Rebalancer}. It loads the class automatically.
+ */
+public class RebalancerRef {
+  private static final Logger LOG = Logger.getLogger(RebalancerRef.class);
+
+  @JsonProperty("rebalancerClassName")
+  private final String _rebalancerClassName;
+
+  @JsonCreator
+  private RebalancerRef(@JsonProperty("rebalancerClassName") String rebalancerClassName) {
+    _rebalancerClassName = rebalancerClassName;
+  }
+
+  /**
+   * Get an instantiated Rebalancer
+   * @return Rebalancer or null if instantiation failed
+   */
+  @JsonIgnore
+  public Rebalancer getRebalancer() {
+    try {
+      return (Rebalancer) (HelixUtil.loadClass(getClass(), _rebalancerClassName).newInstance());
+    } catch (Exception e) {
+      LOG.warn("Exception while invoking custom rebalancer class:" + _rebalancerClassName, e);
+    }
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return _rebalancerClassName;
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that instanceof RebalancerRef) {
+      return this.toString().equals(((RebalancerRef) that).toString());
+    } else if (that instanceof String) {
+      return this.toString().equals(that);
+    }
+    return false;
+  }
+
+  /**
+   * Get a rebalancer class reference
+   * @param rebalancerClassName name of the class
+   * @return RebalancerRef or null if name is null
+   */
+  public static RebalancerRef from(String rebalancerClassName) {
+    if (rebalancerClassName == null) {
+      return null;
+    }
+    return new RebalancerRef(rebalancerClassName);
+  }
+
+  /**
+   * Get a RebalancerRef from a class object
+   * @param rebalancerClass class that implements Rebalancer
+   * @return RebalancerRef
+   */
+  public static RebalancerRef from(Class<? extends Rebalancer> rebalancerClass) {
+    if (rebalancerClass == null) {
+      return null;
+    }
+    return RebalancerRef.from(rebalancerClass.getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/ReplicatedRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/ReplicatedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/ReplicatedRebalancerContext.java
new file mode 100644
index 0000000..4e98ce7
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/ReplicatedRebalancerContext.java
@@ -0,0 +1,40 @@
+package org.apache.helix.api.rebalancer;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Methods specifying a rebalancer context that allows replicas. For instance, a rebalancer context
+ * with partitions may accept state model definitions that support multiple replicas per partition,
+ * and it's possible that the policy is that each live participant in the system should have a
+ * replica.
+ */
+public interface ReplicatedRebalancerContext extends RebalancerContext {
+  /**
+   * Check if this resource should be assigned to any live participant
+   * @return true if any live participant expected, false otherwise
+   */
+  public boolean anyLiveParticipant();
+
+  /**
+   * Get the number of replicas that each resource subunit should have
+   * @return replica count
+   */
+  public int getReplicaCount();
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancer.java
new file mode 100644
index 0000000..80ab256
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/rebalancer/SemiAutoRebalancer.java
@@ -0,0 +1,81 @@
+package org.apache.helix.api.rebalancer;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
+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.rebalancer.util.ConstraintBasedAssignment;
+import org.apache.helix.controller.stages.ResourceCurrentState;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+/*
+ * 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.
+ */
+
+/**
+ * Rebalancer for the SEMI_AUTO mode. It expects a RebalancerConfig that understands the preferred
+ * locations of each partition replica
+ */
+public class SemiAutoRebalancer implements Rebalancer {
+  private static final Logger LOG = Logger.getLogger(SemiAutoRebalancer.class);
+
+  @Override
+  public void init(HelixManager helixManager) {
+    // do nothing
+  }
+
+  @Override
+  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
+      Cluster cluster, ResourceCurrentState currentState) {
+    SemiAutoRebalancerContext config =
+        rebalancerConfig.getRebalancerContext(SemiAutoRebalancerContext.class);
+    StateModelDefinition stateModelDef =
+        cluster.getStateModelMap().get(config.getStateModelDefId());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing resource:" + config.getResourceId());
+    }
+    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
+    for (PartitionId partition : config.getPartitionSet()) {
+      Map<ParticipantId, State> currentStateMap =
+          currentState.getCurrentStateMap(config.getResourceId(), partition);
+      Set<ParticipantId> disabledInstancesForPartition =
+          ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
+              partition);
+      List<ParticipantId> preferenceList =
+          ConstraintBasedAssignment.getPreferenceList(cluster, partition,
+              config.getPreferenceList(partition));
+      Map<State, String> upperBounds =
+          ConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(),
+              cluster.getConfig());
+      Map<ParticipantId, State> bestStateForPartition =
+          ConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, cluster
+              .getLiveParticipantMap().keySet(), stateModelDef, preferenceList, currentStateMap,
+              disabledInstancesForPartition);
+      partitionMapping.addReplicaMap(partition, bestStateForPartition);
+    }
+    return partitionMapping;
+  }
+
+}


[2/9] [HELIX-209] Backward compatible function naming in the model package

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 75d564f..59bae9f 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
@@ -334,7 +334,7 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
 
     // check partition is in ERROR state
-    SessionId sessionId = liveInstance.getSessionId();
+    SessionId sessionId = liveInstance.getTypedSessionId();
     CurrentState curState =
         accessor.getProperty(keyBuilder.currentState(instanceName, sessionId.stringify(),
             resourceName));
@@ -358,7 +358,7 @@ public class ZKHelixAdmin implements HelixAdmin {
     List<Message> messages = accessor.getChildValues(keyBuilder.messages(instanceName));
     for (Message message : messages) {
       if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())
-          || !sessionId.equals(message.getTgtSessionId())
+          || !sessionId.equals(message.getTypedTgtSessionId())
           || !resourceName.equals(message.getResourceId().stringify())
           || !resetPartitionNames.contains(message.getPartitionId().stringify())) {
         continue;
@@ -391,7 +391,7 @@ public class ZKHelixAdmin implements HelixAdmin {
       message.setTgtSessionId(sessionId);
       message.setStateModelDef(stateModelDef);
       message.setFromState(State.from(HelixDefinedState.ERROR.toString()));
-      message.setToState(stateModel.getInitialState());
+      message.setToState(stateModel.getTypedInitialState());
       message.setStateModelFactoryId(idealState.getStateModelFactoryId());
 
       resetMessages.add(message);
@@ -993,7 +993,7 @@ public class ZKHelixAdmin implements HelixAdmin {
     }
     // StateModelDefinition def = new StateModelDefinition(stateModDef);
 
-    List<String> statePriorityList = stateModDef.getStatesPriorityStringList();
+    List<String> statePriorityList = stateModDef.getStatesPriorityList();
 
     String masterStateValue = null;
     String slaveStateValue = null;
@@ -1151,7 +1151,7 @@ public class ZKHelixAdmin implements HelixAdmin {
   @Override
   public void rebalance(String clusterName, IdealState currentIdealState, List<String> instanceNames) {
     Set<String> activeInstances = new HashSet<String>();
-    for (PartitionId partition : currentIdealState.getPartitionSet()) {
+    for (PartitionId partition : currentIdealState.getPartitionIdSet()) {
       activeInstances.addAll(IdealState.stringListFromPreferenceList(currentIdealState
           .getPreferenceList(partition)));
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
index 7d37b68..afd35e6 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -837,7 +837,7 @@ public class ZKHelixManager implements HelixManager {
             keyBuilder.currentState(_instanceName, _sessionId, lastCurState.getResourceName())
                 .getPath();
         _helixAccessor.getBaseDataAccessor().update(curStatePath,
-            new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialStateString(), lastCurState),
+            new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialState(), lastCurState),
             AccessOption.PERSISTENT);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java b/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java
index f9743a4..9fea0c8 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/AsyncCallback.java
@@ -71,7 +71,7 @@ public abstract class AsyncCallback {
   }
 
   public synchronized final void onReply(Message message) {
-    _logger.info("OnReply msg " + message.getMsgId());
+    _logger.info("OnReply msg " + message.getMessageId());
     if (!isDone()) {
       _messageReplied.add(message);
       try {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 a207b0c..73b69a8 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
@@ -161,7 +161,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
           List<LiveInstance> liveInstances = accessor.getChildValues(keyBuilder.liveInstances());
 
           for (LiveInstance liveInstance : liveInstances) {
-            sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getSessionId()
+            sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getTypedSessionId()
                 .stringify());
           }
         }
@@ -194,7 +194,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
     List<Message> messages = new ArrayList<Message>();
     MessageId id = MessageId.from(UUID.randomUUID().toString());
     Message newMessage = new Message(message.getRecord(), id);
-    newMessage.setMsgId(id);
+    newMessage.setMessageId(id);
     newMessage.setSrcName(_manager.getInstanceName());
     newMessage.setTgtName("Controller");
     messages.add(newMessage);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
index 46c595d..17fc67d 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
@@ -47,14 +47,14 @@ public class AsyncCallbackService implements MessageHandlerFactory {
   void verifyMessage(Message message) {
     if (!message.getMsgType().toString().equalsIgnoreCase(MessageType.TASK_REPLY.toString())) {
       String errorMsg =
-          "Unexpected msg type for message " + message.getMsgId() + " type:" + message.getMsgType()
+          "Unexpected msg type for message " + message.getMessageId() + " type:" + message.getMsgType()
               + " Expected : " + MessageType.TASK_REPLY;
       _logger.error(errorMsg);
       throw new HelixException(errorMsg);
     }
     String correlationId = message.getCorrelationId();
     if (correlationId == null) {
-      String errorMsg = "Message " + message.getMsgId() + " does not have correlation id";
+      String errorMsg = "Message " + message.getMessageId() + " does not have correlation id";
       _logger.error(errorMsg);
       throw new HelixException(errorMsg);
     }
@@ -62,13 +62,13 @@ public class AsyncCallbackService implements MessageHandlerFactory {
     if (!_callbackMap.containsKey(correlationId)) {
       String errorMsg =
           "Message "
-              + message.getMsgId()
+              + message.getMessageId()
               + " does not have correponding callback. Probably timed out already. Correlation id: "
               + correlationId;
       _logger.error(errorMsg);
       throw new HelixException(errorMsg);
     }
-    _logger.info("Verified reply message " + message.getMsgId() + " correlation:" + correlationId);
+    _logger.info("Verified reply message " + message.getMessageId() + " correlation:" + correlationId);
   }
 
   @Override
@@ -101,7 +101,7 @@ public class AsyncCallbackService implements MessageHandlerFactory {
       verifyMessage(_message);
       HelixTaskResult result = new HelixTaskResult();
       assert (_correlationId.equalsIgnoreCase(_message.getCorrelationId()));
-      _logger.info("invoking reply message " + _message.getMsgId() + ", correlationid:"
+      _logger.info("invoking reply message " + _message.getMessageId() + ", correlationid:"
           + _correlationId);
 
       AsyncCallback callback = _callbackMap.get(_correlationId);
@@ -118,7 +118,7 @@ public class AsyncCallbackService implements MessageHandlerFactory {
 
     @Override
     public void onError(Exception e, ErrorCode code, ErrorType type) {
-      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e);
+      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 c6eaa65..8381f4a 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
@@ -96,7 +96,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
 
     PartitionId partitionId = _message.getPartitionId();
-    State fromState = _message.getFromState();
+    State fromState = _message.getTypedFromState();
 
     // Verify the fromState and current state of the stateModel
     String state = _currentStateDelta.getState(partitionId.stringify());
@@ -122,7 +122,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
 
     PartitionId partitionId = _message.getPartitionId();
     ResourceId resource = _message.getResourceId();
-    SessionId sessionId = _message.getTgtSessionId();
+    SessionId sessionId = _message.getTypedTgtSessionId();
     String instanceName = _manager.getInstanceName();
 
     HelixDataAccessor accessor = _manager.getHelixDataAccessor();
@@ -135,15 +135,15 @@ public class HelixStateTransitionHandler extends MessageHandler {
     // new session
     // sessionId might change when we update the state model state.
     // for zk current state it is OK as we have the per-session current state node
-    if (!_message.getTgtSessionId().stringify().equals(_manager.getSessionId())) {
+    if (!_message.getTypedTgtSessionId().stringify().equals(_manager.getSessionId())) {
       logger.warn("Session id has changed. Skip postExecutionMessage. Old session "
-          + _message.getExecutionSessionId() + " , new session : " + _manager.getSessionId());
+          + _message.getTypedExecutionSessionId() + " , new session : " + _manager.getSessionId());
       return;
     }
 
     if (taskResult.isSuccess()) {
       // String fromState = message.getFromState();
-      State toState = _message.getToState();
+      State toState = _message.getTypedToState();
       _currentStateDelta.setState(partitionId, toState);
 
       if (toState.toString().equalsIgnoreCase(HelixDefinedState.DROPPED.toString())) {
@@ -181,7 +181,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
             // state in this case
             logger
                 .error("State transition interrupted but not timeout. Not updating state. Partition : "
-                    + _message.getPartitionId() + " MsgId : " + _message.getMsgId());
+                    + _message.getPartitionId() + " MsgId : " + _message.getMessageId());
             return;
           }
         }
@@ -190,7 +190,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
         _stateModel.updateState(HelixDefinedState.ERROR.toString());
 
         // if we have errors transit from ERROR state, disable the partition
-        if (_message.getFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
+        if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
           disablePartition();
         }
       }
@@ -229,7 +229,7 @@ public class HelixStateTransitionHandler extends MessageHandler {
     HelixAdmin admin = _manager.getClusterManagmentTool();
     admin.enablePartition(false, clusterName, instanceName, resourceId.stringify(),
         Arrays.asList(partitionId.stringify()));
-    logger.info("error in transit from ERROR to " + _message.getToState() + " for partition: "
+    logger.info("error in transit from ERROR to " + _message.getTypedToState() + " for partition: "
         + partitionId + ". disable it on " + instanceName);
 
   }
@@ -288,8 +288,8 @@ public class HelixStateTransitionHandler extends MessageHandler {
 
     // by default, we invoke state transition function in state model
     Method methodToInvoke = null;
-    State fromState = message.getFromState();
-    State toState = message.getToState();
+    State fromState = message.getTypedFromState();
+    State toState = message.getTypedToState();
     methodToInvoke =
         _transitionMethodFinder.getMethodForTransition(_stateModel.getClass(),
             fromState.toString(), toState.toString(), new Class[] {
@@ -335,10 +335,10 @@ public class HelixStateTransitionHandler extends MessageHandler {
         _stateModel.updateState(HelixDefinedState.ERROR.toString());
 
         // if transit from ERROR state, disable the partition
-        if (_message.getFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
+        if (_message.getTypedFromState().toString().equalsIgnoreCase(HelixDefinedState.ERROR.toString())) {
           disablePartition();
         }
-        accessor.updateProperty(keyBuilder.currentState(instanceName, _message.getTgtSessionId()
+        accessor.updateProperty(keyBuilder.currentState(instanceName, _message.getTypedTgtSessionId()
             .stringify(), resourceId.stringify()), currentStateDelta);
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
index 39da1aa..3bcc260 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTask.java
@@ -90,14 +90,14 @@ public class HelixTask implements MessageTask {
 
       _statusUpdateUtil.logError(_message, HelixTask.class, e,
           "State transition interrupted, timeout:" + _isTimeout, accessor);
-      logger.info("Message " + _message.getMsgId() + " is interrupted");
+      logger.info("Message " + _message.getMessageId() + " is interrupted");
     } catch (Exception e) {
       taskResult = new HelixTaskResult();
       taskResult.setException(e);
       taskResult.setMessage(e.getMessage());
 
       String errorMessage =
-          "Exception while executing a message. " + e + " msgId: " + _message.getMsgId()
+          "Exception while executing a message. " + e + " msgId: " + _message.getMessageId()
               + " type: " + _message.getMsgType();
       logger.error(errorMessage, e);
       _statusUpdateUtil.logError(_message, HelixTask.class, e, errorMessage, accessor);
@@ -111,17 +111,17 @@ public class HelixTask implements MessageTask {
       if (taskResult.isSuccess()) {
         _statusUpdateUtil.logInfo(_message, _handler.getClass(),
             "Message handling task completed successfully", accessor);
-        logger.info("Message " + _message.getMsgId() + " completed.");
+        logger.info("Message " + _message.getMessageId() + " completed.");
       } else {
         type = ErrorType.INTERNAL;
 
         if (taskResult.isInterrupted()) {
-          logger.info("Message " + _message.getMsgId() + " is interrupted");
+          logger.info("Message " + _message.getMessageId() + " is interrupted");
           code = _isTimeout ? ErrorCode.TIMEOUT : ErrorCode.CANCEL;
           if (_isTimeout) {
             int retryCount = _message.getRetryCount();
             logger.info("Message timeout, retry count: " + retryCount + " msgId:"
-                + _message.getMsgId());
+                + _message.getMessageId());
             _statusUpdateUtil.logInfo(_message, _handler.getClass(),
                 "Message handling task timeout, retryCount:" + retryCount, accessor);
             // Notify the handler that timeout happens, and the number of retries left
@@ -158,12 +158,12 @@ public class HelixTask implements MessageTask {
       code = ErrorCode.ERROR;
 
       String errorMessage =
-          "Exception after executing a message, msgId: " + _message.getMsgId() + e;
+          "Exception after executing a message, msgId: " + _message.getMessageId() + e;
       logger.error(errorMessage, e);
       _statusUpdateUtil.logError(_message, HelixTask.class, errorMessage, accessor);
     } finally {
       long end = System.currentTimeMillis();
-      logger.info("msg: " + _message.getMsgId() + " handling task completed, results:"
+      logger.info("msg: " + _message.getMessageId() + " handling task completed, results:"
           + taskResult.isSuccess() + ", at: " + end + ", took:" + (end - start));
 
       // Notify the handler about any error happened in the handling procedure, so that
@@ -182,9 +182,9 @@ public class HelixTask implements MessageTask {
     Builder keyBuilder = accessor.keyBuilder();
     if (message.getTgtName().equalsIgnoreCase("controller")) {
       // TODO: removeProperty returns boolean
-      accessor.removeProperty(keyBuilder.controllerMessage(message.getMsgId().stringify()));
+      accessor.removeProperty(keyBuilder.controllerMessage(message.getMessageId().stringify()));
     } else {
-      accessor.removeProperty(keyBuilder.message(_manager.getInstanceName(), message.getMsgId()
+      accessor.removeProperty(keyBuilder.message(_manager.getInstanceName(), message.getMessageId()
           .stringify()));
     }
   }
@@ -208,11 +208,11 @@ public class HelixTask implements MessageTask {
       if (message.getSrcInstanceType() == InstanceType.PARTICIPANT) {
         Builder keyBuilder = accessor.keyBuilder();
         accessor.setProperty(
-            keyBuilder.message(message.getMsgSrc(), replyMessage.getMsgId().stringify()),
+            keyBuilder.message(message.getMsgSrc(), replyMessage.getMessageId().stringify()),
             replyMessage);
       } else if (message.getSrcInstanceType() == InstanceType.CONTROLLER) {
         Builder keyBuilder = accessor.keyBuilder();
-        accessor.setProperty(keyBuilder.controllerMessage(replyMessage.getMsgId().stringify()),
+        accessor.setProperty(keyBuilder.controllerMessage(replyMessage.getMessageId().stringify()),
             replyMessage);
       }
       _statusUpdateUtil.logInfo(message, HelixTask.class,
@@ -232,8 +232,8 @@ public class HelixTask implements MessageTask {
       long totalDelay = now - msgReadTime;
       long executionDelay = now - msgExecutionStartTime;
       if (totalDelay > 0 && executionDelay > 0) {
-        State fromState = message.getFromState();
-        State toState = message.getToState();
+        State fromState = message.getTypedFromState();
+        State toState = message.getTypedToState();
         String transition = fromState + "--" + toState;
 
         StateTransitionContext cxt =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8cf1aa7..8da53ea 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
@@ -432,14 +432,14 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         continue;
       }
 
-      SessionId tgtSessionId = message.getTgtSessionId();
+      SessionId tgtSessionId = message.getTypedTgtSessionId();
 
       // sessionId mismatch normally means message comes from expired session, just remove it
       if (!sessionId.equals(tgtSessionId.toString()) && !tgtSessionId.toString().equals("*")) {
         String warningMessage =
             "SessionId does NOT match. expected sessionId: " + sessionId
                 + ", tgtSessionId in message: " + tgtSessionId + ", messageId: "
-                + message.getMsgId();
+                + message.getMessageId();
         LOG.warn(warningMessage);
         accessor.removeProperty(message.getKey(keyBuilder, instanceName));
         _statusUpdateUtil.logWarning(message, HelixStateMachineEngine.class, warningMessage,
@@ -454,7 +454,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         // We will read the message again if there is a new message but we
         // check for the status and ignore if its already read
         if (LOG.isTraceEnabled()) {
-          LOG.trace("Message already read. msgId: " + message.getMsgId());
+          LOG.trace("Message already read. msgId: " + message.getMessageId());
         }
         continue;
       }
@@ -467,9 +467,9 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
         }
         handlers.add(createHandler);
       } catch (Exception e) {
-        LOG.error("Failed to create message handler for " + message.getMsgId(), e);
+        LOG.error("Failed to create message handler for " + message.getMessageId(), e);
         String error =
-            "Failed to create message handler for " + message.getMsgId() + ", exception: " + e;
+            "Failed to create message handler for " + message.getMessageId() + ", exception: " + e;
 
         _statusUpdateUtil.logError(message, HelixStateMachineEngine.class, e, error, accessor);
 
@@ -547,7 +547,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
     // the corresponding MessageHandlerFactory is registered
     if (handlerFactory == null) {
       LOG.warn("Fail to find message handler factory for type: " + msgType + " msgId: "
-          + message.getMsgId());
+          + message.getMessageId());
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java
index e1b4f0f..17fc041 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageTimeoutTask.java
@@ -40,7 +40,7 @@ public class MessageTimeoutTask extends TimerTask {
     Message message = _task.getMessage();
     // NotificationContext context = _task.getNotificationContext();
     // System.out.println("msg: " + message.getMsgId() + " timeouot.");
-    LOG.warn("Message time out, canceling. id:" + message.getMsgId() + " timeout : "
+    LOG.warn("Message time out, canceling. id:" + message.getMessageId() + " timeout : "
         + message.getExecutionTimeout());
     _task.onTimeout();
     _executor.cancelTask(_task);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java b/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java
index d90ec1a..d5ee44c 100644
--- a/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java
+++ b/helix-core/src/main/java/org/apache/helix/model/AlertStatus.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.id.SessionId;
 import org.apache.helix.model.Alerts.AlertsProperty;
 
 /**
@@ -67,6 +68,16 @@ public class AlertStatus extends HelixProperty {
   }
 
   /**
+   * Set the session that the alerts correspond to
+   * @param sessionId the session for which alerts occurred
+   */
+  public void setSessionId(SessionId sessionId) {
+    if (sessionId != null) {
+      setSessionId(sessionId.stringify());
+    }
+  }
+
+  /**
    * Get the session that these alerts correspond to
    * @return session identifier
    */
@@ -75,6 +86,14 @@ public class AlertStatus extends HelixProperty {
   }
 
   /**
+   * Get the session that the alerts correspond to
+   * @return session identifier
+   */
+  public SessionId getTypedSessionId() {
+    return SessionId.from(getSessionId());
+  }
+
+  /**
    * Get the instance that these alerts correspond to
    * @return name of the instance
    */

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/model/Alerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Alerts.java b/helix-core/src/main/java/org/apache/helix/model/Alerts.java
index 506e3d5..f30f39c 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Alerts.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Alerts.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.id.SessionId;
 
 /**
  * Describe alerts and corresponding metrics. An alert is triggered when cluster health
@@ -78,6 +79,16 @@ public class Alerts extends HelixProperty {
   }
 
   /**
+   * Set the session that the alerts correspond to
+   * @param sessionId the session for which alerts occurred
+   */
+  public void setSessionId(SessionId sessionId) {
+    if (sessionId != null) {
+      setSessionId(sessionId.stringify());
+    }
+  }
+
+  /**
    * Get the session that the alerts correspond to
    * @return session identifier
    */
@@ -86,6 +97,14 @@ public class Alerts extends HelixProperty {
   }
 
   /**
+   * Get the session that the alerts correspond to
+   * @return session identifier
+   */
+  public SessionId getTypedSessionId() {
+    return SessionId.from(getSessionId());
+  }
+
+  /**
    * Get the instance that the alerts correspond to
    * @return the name of the instance
    */

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8e37b18..daefe6e 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
@@ -126,11 +126,20 @@ public class ClusterConstraints extends HelixProperty {
   }
 
   /**
+   * add the constraint, overwrite existing one if constraint with same constraint-id already exists
+   * @param constraintId unique constraint identifier
+   * @param item the constraint as a {@link ConstraintItem}
+   */
+  public void addConstraintItem(String constraintId, ConstraintItem item) {
+    addConstraintItem(ConstraintId.from(constraintId), item);
+  }
+
+  /**
    * Add multiple constraint items.
    * @param items (constraint identifier, {@link ConstrantItem}) pairs
    */
-  public void addConstraintItems(Map<ConstraintId, ConstraintItem> items) {
-    for (ConstraintId constraintId : items.keySet()) {
+  public void addConstraintItems(Map<String, ConstraintItem> items) {
+    for (String constraintId : items.keySet()) {
       addConstraintItem(constraintId, items.get(constraintId));
     }
   }
@@ -145,6 +154,14 @@ public class ClusterConstraints extends HelixProperty {
   }
 
   /**
+   * remove a constraint-item
+   * @param constraintId unique constraint identifier
+   */
+  public void removeConstraintItem(String constraintId) {
+    removeConstraintItem(ConstraintId.from(constraintId));
+  }
+
+  /**
    * get a constraint-item
    * @param constraintId unique constraint identifier
    * @return {@link ConstraintItem} or null if not present
@@ -154,6 +171,15 @@ public class ClusterConstraints extends HelixProperty {
   }
 
   /**
+   * get a constraint-item
+   * @param constraintId unique constraint identifier
+   * @return {@link ConstraintItem} or null if not present
+   */
+  public ConstraintItem getConstraintItem(String constraintId) {
+    return getConstraintItem(ConstraintId.from(constraintId));
+  }
+
+  /**
    * return a set of constraints that match the attribute pairs
    * @param attributes (constraint scope, constraint string) pairs
    * @return a set of {@link ConstraintItem}s with matching attributes
@@ -186,9 +212,9 @@ public class ClusterConstraints extends HelixProperty {
     String msgType = msg.getMsgType();
     attributes.put(ConstraintAttribute.MESSAGE_TYPE, msgType);
     if (MessageType.STATE_TRANSITION.toString().equals(msgType)) {
-      if (msg.getFromState() != null && msg.getToState() != null) {
+      if (msg.getTypedFromState() != null && msg.getTypedToState() != null) {
         attributes.put(ConstraintAttribute.TRANSITION,
-            Transition.from(msg.getFromState(), msg.getToState()).toString());
+            Transition.from(msg.getTypedFromState(), msg.getTypedToState()).toString());
       }
       if (msg.getResourceId() != null) {
         attributes.put(ConstraintAttribute.RESOURCE, msg.getResourceId().stringify());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 2fe37ce..5c9bcbc 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
@@ -95,7 +95,7 @@ public class CurrentState extends HelixProperty {
    * Get the partitions on this instance and the state that each partition is currently in.
    * @return (partition, state) pairs
    */
-  public Map<String, String> getPartitionStateStringMap() {
+  public Map<String, String> getPartitionStateMap() {
     Map<String, String> map = new HashMap<String, String>();
     Map<String, Map<String, String>> mapFields = _record.getMapFields();
     for (String partitionName : mapFields.keySet()) {
@@ -111,7 +111,7 @@ public class CurrentState extends HelixProperty {
    * Get the partitions on this instance and the state that each partition is currently in
    * @return (partition id, state) pairs
    */
-  public Map<PartitionId, State> getPartitionStateMap() {
+  public Map<PartitionId, State> getTypedPartitionStateMap() {
     Map<PartitionId, State> map = new HashMap<PartitionId, State>();
     for (String partitionName : _record.getMapFields().keySet()) {
       Map<String, String> stateMap = _record.getMapField(partitionName);
@@ -127,8 +127,16 @@ public class CurrentState extends HelixProperty {
    * Get the session that this current state corresponds to
    * @return session identifier
    */
-  public SessionId getSessionId() {
-    return SessionId.from(_record.getSimpleField(CurrentStateProperty.SESSION_ID.toString()));
+  public SessionId getTypedSessionId() {
+    return SessionId.from(getSessionId());
+  }
+
+  /**
+   * Get the session that this current state corresponds to
+   * @return session identifier
+   */
+  public String getSessionId() {
+    return _record.getSimpleField(CurrentStateProperty.SESSION_ID.toString());
   }
 
   /**
@@ -136,7 +144,15 @@ public class CurrentState extends HelixProperty {
    * @param sessionId session identifier
    */
   public void setSessionId(SessionId sessionId) {
-    _record.setSimpleField(CurrentStateProperty.SESSION_ID.toString(), sessionId.stringify());
+    setSessionId(sessionId.stringify());
+  }
+
+  /**
+   * Set the session that this current state corresponds to
+   * @param sessionId session identifier
+   */
+  public void setSessionId(String sessionId) {
+    _record.setSimpleField(CurrentStateProperty.SESSION_ID.toString(), sessionId);
   }
 
   /**
@@ -197,7 +213,7 @@ public class CurrentState extends HelixProperty {
 
   /**
    * Set the state that a partition is currently in on this instance
-   * @param partitionName the name of the partition
+   * @param partitionId the id of the partition
    * @param state the state of the partition
    */
   public void setState(PartitionId partitionId, State state) {
@@ -210,6 +226,19 @@ public class CurrentState extends HelixProperty {
   }
 
   /**
+   * Set the state that a partition is currently in on this instance
+   * @param partitionName the name of the partition
+   * @param state the state of the partition
+   */
+  public void setState(String partitionName, String state) {
+    Map<String, Map<String, String>> mapFields = _record.getMapFields();
+    if (mapFields.get(partitionName) == null) {
+      mapFields.put(partitionName, new TreeMap<String, String>());
+    }
+    mapFields.get(partitionName).put(CurrentStateProperty.CURRENT_STATE.toString(), state);
+  }
+
+  /**
    * Set the state model factory
    * @param factoryName the name of the factory
    */

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 15a22ca..0f7b6db 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
@@ -114,7 +114,7 @@ public class ExternalView extends HelixProperty {
    * Get all the partitions of the resource
    * @return a set of partition names
    */
-  public Set<String> getPartitionStringSet() {
+  public Set<String> getPartitionSet() {
     return _record.getMapFields().keySet();
   }
 
@@ -122,9 +122,9 @@ public class ExternalView extends HelixProperty {
    * Get all the partitions of the resource
    * @return a set of partition ids
    */
-  public Set<PartitionId> getPartitionSet() {
+  public Set<PartitionId> getPartitionIdSet() {
     Set<PartitionId> partitionSet = Sets.newHashSet();
-    for (String partitionName : getPartitionStringSet()) {
+    for (String partitionName : getPartitionSet()) {
       partitionSet.add(PartitionId.from(partitionName));
     }
     return partitionSet;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8f579ec..7d84258 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
@@ -126,7 +126,6 @@ public class IdealState extends HelixProperty {
    * Get the associated resource
    * @return the name of the resource
    */
-  @Deprecated
   public String getResourceName() {
     return _record.getId();
   }
@@ -173,7 +172,6 @@ public class IdealState extends HelixProperty {
    * Define a custom rebalancer that implements {@link Rebalancer}
    * @param rebalancerClassName the name of the custom rebalancing class
    */
-  @Deprecated
   public void setRebalancerClassName(String rebalancerClassName) {
     _record
         .setSimpleField(IdealStateProperty.REBALANCER_CLASS_NAME.toString(), rebalancerClassName);
@@ -183,7 +181,6 @@ public class IdealState extends HelixProperty {
    * Get the name of the user-defined rebalancer associated with this resource
    * @return the rebalancer class name, or null if none is being used
    */
-  @Deprecated
   public String getRebalancerClassName() {
     return _record.getSimpleField(IdealStateProperty.REBALANCER_CLASS_NAME.toString());
   }
@@ -277,8 +274,7 @@ public class IdealState extends HelixProperty {
    * Get all of the partitions
    * @return a set of partition names
    */
-  @Deprecated
-  public Set<String> getPartitionStringSet() {
+  public Set<String> getPartitionSet() {
     if (getRebalanceMode() == RebalanceMode.SEMI_AUTO
         || getRebalanceMode() == RebalanceMode.FULL_AUTO) {
       return _record.getListFields().keySet();
@@ -295,9 +291,9 @@ public class IdealState extends HelixProperty {
    * Get all of the partitions
    * @return a set of partitions
    */
-  public Set<PartitionId> getPartitionSet() {
+  public Set<PartitionId> getPartitionIdSet() {
     Set<PartitionId> partitionSet = Sets.newHashSet();
-    for (String partitionName : getPartitionStringSet()) {
+    for (String partitionName : getPartitionSet()) {
       partitionSet.add(PartitionId.from(partitionName));
     }
     return partitionSet;
@@ -308,7 +304,6 @@ public class IdealState extends HelixProperty {
    * @param partitionName the name of the partition
    * @return the instances where the replicas live and the state of each
    */
-  @Deprecated
   public Map<String, String> getInstanceStateMap(String partitionName) {
     return _record.getMapField(partitionName);
   }
@@ -350,7 +345,6 @@ public class IdealState extends HelixProperty {
    * @param partitionName the partition to look up
    * @return set of instance names
    */
-  @Deprecated
   public Set<String> getInstanceSet(String partitionName) {
     if (getRebalanceMode() == RebalanceMode.SEMI_AUTO
         || getRebalanceMode() == RebalanceMode.FULL_AUTO) {
@@ -407,7 +401,6 @@ public class IdealState extends HelixProperty {
    * @param partitionName the name of the partition
    * @return a list of instances that can serve replicas of the partition
    */
-  @Deprecated
   public List<String> getPreferenceList(String partitionName) {
     List<String> instanceStateList = _record.getListField(partitionName);
 
@@ -439,7 +432,6 @@ public class IdealState extends HelixProperty {
    * Get the state model associated with this resource
    * @return an identifier of the state model
    */
-  @Deprecated
   public String getStateModelDefRef() {
     return _record.getSimpleField(IdealStateProperty.STATE_MODEL_DEF_REF.toString());
   }
@@ -456,7 +448,6 @@ public class IdealState extends HelixProperty {
    * Set the state model associated with this resource
    * @param stateModel state model identifier
    */
-  @Deprecated
   public void setStateModelDefRef(String stateModel) {
     _record.setSimpleField(IdealStateProperty.STATE_MODEL_DEF_REF.toString(), stateModel);
   }
@@ -546,7 +537,6 @@ public class IdealState extends HelixProperty {
    * Set the state model factory associated with this resource
    * @param name state model factory name
    */
-  @Deprecated
   public void setStateModelFactoryName(String name) {
     _record.setSimpleField(IdealStateProperty.STATE_MODEL_FACTORY_NAME.toString(), name);
   }
@@ -565,7 +555,6 @@ public class IdealState extends HelixProperty {
    * Get the state model factory associated with this resource
    * @return state model factory name
    */
-  @Deprecated
   public String getStateModelFactoryName() {
     return _record.getStringField(IdealStateProperty.STATE_MODEL_FACTORY_NAME.toString(),
         HelixConstants.DEFAULT_STATE_MODEL_FACTORY);
@@ -609,7 +598,7 @@ public class IdealState extends HelixProperty {
 
       if (!replicaStr.equals(HelixConstants.StateModelToken.ANY_LIVEINSTANCE.toString())) {
         int replica = Integer.parseInt(replicaStr);
-        Set<String> partitionSet = getPartitionStringSet();
+        Set<String> partitionSet = getPartitionSet();
         for (String partition : partitionSet) {
           List<String> preferenceList = getPreferenceList(partition);
           if (preferenceList == null || preferenceList.size() != replica) {
@@ -655,7 +644,7 @@ public class IdealState extends HelixProperty {
     _record.getListFields().clear();
 
     // assign a partition at a time
-    for (PartitionId partition : assignment.getMappedPartitions()) {
+    for (PartitionId partition : assignment.getMappedPartitionIds()) {
       List<ParticipantId> preferenceList = new ArrayList<ParticipantId>();
       Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
 
@@ -665,7 +654,7 @@ public class IdealState extends HelixProperty {
       Multimaps.invertFrom(Multimaps.forMap(replicaMap), inverseMap);
 
       // update the ideal state in order of state priorities
-      for (State state : stateModelDef.getStatesPriorityList()) {
+      for (State state : stateModelDef.getTypedStatesPriorityList()) {
         if (!state.equals(State.from(HelixDefinedState.DROPPED))
             && !state.equals(State.from(HelixDefinedState.ERROR))) {
           List<ParticipantId> stateParticipants = inverseMap.get(state);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 d2e1187..8f776a0 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
@@ -228,7 +228,7 @@ public class InstanceConfig extends HelixProperty {
    * @param partitionId the partition to set
    * @param enabled true to enable, false to disable
    */
-  public void setInstanceEnabledForPartition(PartitionId partitionId, boolean enabled) {
+  public void setParticipantEnabledForPartition(PartitionId partitionId, boolean enabled) {
     setInstanceEnabledForPartition(partitionId.stringify(), enabled);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 fda144a..e9348ec 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
@@ -79,8 +79,16 @@ public class LiveInstance extends HelixProperty {
    * Get the session that this participant corresponds to
    * @return session identifier
    */
-  public SessionId getSessionId() {
-    return SessionId.from(_record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString()));
+  public SessionId getTypedSessionId() {
+    return SessionId.from(getSessionId());
+  }
+
+  /**
+   * Get the session that this participant corresponds to
+   * @return session identifier
+   */
+  public String getSessionId() {
+    return _record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString());
   }
 
   /**
@@ -103,8 +111,16 @@ public class LiveInstance extends HelixProperty {
    * Get the version of Helix that this participant is running
    * @return the version
    */
-  public HelixVersion getHelixVersion() {
-    return HelixVersion.from(_record.getSimpleField(LiveInstanceProperty.HELIX_VERSION.toString()));
+  public HelixVersion getTypedHelixVersion() {
+    return HelixVersion.from(getHelixVersion());
+  }
+
+  /**
+   * Get the version of Helix that this participant is running
+   * @return the version
+   */
+  public String getHelixVersion() {
+    return _record.getSimpleField(LiveInstanceProperty.HELIX_VERSION.toString());
   }
 
   /**
@@ -165,11 +181,11 @@ public class LiveInstance extends HelixProperty {
 
   @Override
   public boolean isValid() {
-    if (getSessionId() == null) {
+    if (getTypedSessionId() == null) {
       _logger.error("liveInstance does not have session id. id:" + _record.getId());
       return false;
     }
-    if (getHelixVersion() == null) {
+    if (getTypedHelixVersion() == null) {
       _logger.error("liveInstance does not have CLM verion. id:" + _record.getId());
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 86319e3..2bd313a 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
@@ -129,25 +129,52 @@ public class Message extends HelixProperty {
 
   /**
    * Instantiate a message
+   * @param type the message category
+   * @param msgId unique message identifier
+   */
+  public Message(MessageType type, String msgId) {
+    this(type, MessageId.from(msgId));
+  }
+
+  /**
+   * Instantiate a message
    * @param type {@link MessageType} as a string or a custom message type
    * @param msgId unique message identifier
    */
   public Message(String type, MessageId msgId) {
     super(new ZNRecord(msgId.stringify()));
     _record.setSimpleField(Attributes.MSG_TYPE.toString(), type);
-    setMsgId(msgId);
+    setMessageId(msgId);
     setMsgState(MessageState.NEW);
     _record.setLongField(Attributes.CREATE_TIMESTAMP.toString(), new Date().getTime());
   }
 
   /**
+   * Instantiate a message
+   * @param type {@link MessageType} as a string or a custom message type
+   * @param msgId unique message identifier
+   */
+  public Message(String type, String msgId) {
+    this(type, MessageId.from(msgId));
+  }
+
+  /**
    * Instantiate a message with a new id
    * @param record a ZNRecord corresponding to a message
    * @param id unique message identifier
    */
   public Message(ZNRecord record, MessageId id) {
     super(new ZNRecord(record, id.stringify()));
-    setMsgId(id);
+    setMessageId(id);
+  }
+
+  /**
+   * Instantiate a message with a new id
+   * @param record a ZNRecord corresponding to a message
+   * @param id unique message identifier
+   */
+  public Message(ZNRecord record, String id) {
+    this(record, MessageId.from(id));
   }
 
   /**
@@ -224,8 +251,16 @@ public class Message extends HelixProperty {
    * Get the session identifier of the destination node
    * @return session identifier
    */
-  public SessionId getTgtSessionId() {
-    return SessionId.from(_record.getSimpleField(Attributes.TGT_SESSION_ID.toString()));
+  public SessionId getTypedTgtSessionId() {
+    return SessionId.from(getTgtSessionId());
+  }
+
+  /**
+   * Get the session identifier of the destination node
+   * @return session identifier
+   */
+  public String getTgtSessionId() {
+    return _record.getSimpleField(Attributes.TGT_SESSION_ID.toString());
   }
 
   /**
@@ -233,15 +268,33 @@ public class Message extends HelixProperty {
    * @param tgtSessionId session identifier
    */
   public void setTgtSessionId(SessionId tgtSessionId) {
-    _record.setSimpleField(Attributes.TGT_SESSION_ID.toString(), tgtSessionId.stringify());
+    if (tgtSessionId != null) {
+      setTgtSessionId(tgtSessionId.stringify());
+    }
+  }
+
+  /**
+   * Set the session identifier of the destination node
+   * @param tgtSessionId session identifier
+   */
+  public void setTgtSessionId(String tgtSessionId) {
+    _record.setSimpleField(Attributes.TGT_SESSION_ID.toString(), tgtSessionId);
   }
 
   /**
    * Get the session identifier of the source node
    * @return session identifier
    */
-  public SessionId getSrcSessionId() {
-    return SessionId.from(_record.getSimpleField(Attributes.SRC_SESSION_ID.toString()));
+  public SessionId getTypedSrcSessionId() {
+    return SessionId.from(getSrcSessionId());
+  }
+
+  /**
+   * Get the session identifier of the source node
+   * @return session identifier
+   */
+  public String getSrcSessionId() {
+    return _record.getSimpleField(Attributes.SRC_SESSION_ID.toString());
   }
 
   /**
@@ -249,15 +302,33 @@ public class Message extends HelixProperty {
    * @param srcSessionId session identifier
    */
   public void setSrcSessionId(SessionId srcSessionId) {
-    _record.setSimpleField(Attributes.SRC_SESSION_ID.toString(), srcSessionId.stringify());
+    if (srcSessionId != null) {
+      setSrcSessionId(srcSessionId.stringify());
+    }
+  }
+
+  /**
+   * Set the session identifier of the source node
+   * @param srcSessionId session identifier
+   */
+  public void setSrcSessionId(String srcSessionId) {
+    _record.setSimpleField(Attributes.SRC_SESSION_ID.toString(), srcSessionId);
+  }
+
+  /**
+   * Get the session identifier of the node that executes the message
+   * @return session identifier
+   */
+  public SessionId getTypedExecutionSessionId() {
+    return SessionId.from(getExecutionSessionId());
   }
 
   /**
    * Get the session identifier of the node that executes the message
    * @return session identifier
    */
-  public SessionId getExecutionSessionId() {
-    return SessionId.from(_record.getSimpleField(Attributes.EXE_SESSION_ID.toString()));
+  public String getExecutionSessionId() {
+    return _record.getSimpleField(Attributes.EXE_SESSION_ID.toString());
   }
 
   /**
@@ -265,7 +336,17 @@ public class Message extends HelixProperty {
    * @param exeSessionId session identifier
    */
   public void setExecuteSessionId(SessionId exeSessionId) {
-    _record.setSimpleField(Attributes.EXE_SESSION_ID.toString(), exeSessionId.stringify());
+    if (exeSessionId != null) {
+      setExecuteSessionId(exeSessionId.stringify());
+    }
+  }
+
+  /**
+   * Set the session identifier of the node that executes the message
+   * @param exeSessionId session identifier
+   */
+  public void setExecuteSessionId(String exeSessionId) {
+    _record.setSimpleField(Attributes.EXE_SESSION_ID.toString(), exeSessionId);
   }
 
   /**
@@ -333,23 +414,51 @@ public class Message extends HelixProperty {
    * @param partitionId
    */
   public void setPartitionId(PartitionId partitionId) {
-    _record.setSimpleField(Attributes.PARTITION_NAME.toString(), partitionId.stringify());
+    if (partitionId != null) {
+      setPartitionName(partitionId.stringify());
+    }
+  }
+
+  /**
+   * Set the id of the partition this message concerns
+   * @param partitionId
+   */
+  public void setPartitionName(String partitionName) {
+    _record.setSimpleField(Attributes.PARTITION_NAME.toString(), partitionName);
   }
 
   /**
    * Get the unique identifier of this message
    * @return message identifier
    */
-  public MessageId getMsgId() {
-    return MessageId.from(_record.getSimpleField(Attributes.MSG_ID.toString()));
+  public MessageId getMessageId() {
+    return MessageId.from(getMsgId());
+  }
+
+  /**
+   * Get the unique identifier of this message
+   * @return message identifier
+   */
+  public String getMsgId() {
+    return _record.getSimpleField(Attributes.MSG_ID.toString());
   }
 
   /**
    * Set the unique identifier of this message
    * @param msgId message identifier
    */
-  public void setMsgId(MessageId msgId) {
-    _record.setSimpleField(Attributes.MSG_ID.toString(), msgId.toString());
+  public void setMessageId(MessageId msgId) {
+    if (msgId != null) {
+      setMsgId(msgId.stringify());
+    }
+  }
+
+  /**
+   * Set the unique identifier of this message
+   * @param msgId message identifier
+   */
+  public void setMsgId(String msgId) {
+    _record.setSimpleField(Attributes.MSG_ID.toString(), msgId);
   }
 
   /**
@@ -357,6 +466,16 @@ public class Message extends HelixProperty {
    * @param state the state
    */
   public void setFromState(State state) {
+    if (state != null) {
+      setFromState(state.toString());
+    }
+  }
+
+  /**
+   * Set the "from state" for transition-related messages
+   * @param state the state
+   */
+  public void setFromState(String state) {
     _record.setSimpleField(Attributes.FROM_STATE.toString(), state.toString());
   }
 
@@ -364,8 +483,16 @@ public class Message extends HelixProperty {
    * Get the "from-state" for transition-related messages
    * @return state, or null for other message types
    */
-  public State getFromState() {
-    return State.from(_record.getSimpleField(Attributes.FROM_STATE.toString()));
+  public State getTypedFromState() {
+    return State.from(getFromState());
+  }
+
+  /**
+   * Get the "from-state" for transition-related messages
+   * @return state, or null for other message types
+   */
+  public String getFromState() {
+    return _record.getSimpleField(Attributes.FROM_STATE.toString());
   }
 
   /**
@@ -373,6 +500,16 @@ public class Message extends HelixProperty {
    * @param state the state
    */
   public void setToState(State state) {
+    if (state != null) {
+      setToState(state.toString());
+    }
+  }
+
+  /**
+   * Set the "to state" for transition-related messages
+   * @param state the state
+   */
+  public void setToState(String state) {
     _record.setSimpleField(Attributes.TO_STATE.toString(), state.toString());
   }
 
@@ -380,8 +517,16 @@ public class Message extends HelixProperty {
    * Get the "to state" for transition-related messages
    * @return state, or null for other message types
    */
-  public State getToState() {
-    return State.from(_record.getSimpleField(Attributes.TO_STATE.toString()));
+  public State getTypedToState() {
+    return State.from(getToState());
+  }
+
+  /**
+   * Get the "to state" for transition-related messages
+   * @return state, or null for other message types
+   */
+  public String getToState() {
+    return _record.getSimpleField(Attributes.TO_STATE.toString());
   }
 
   /**
@@ -413,7 +558,17 @@ public class Message extends HelixProperty {
    * @param resourceId resource name to set
    */
   public void setResourceId(ResourceId resourceId) {
-    _record.setSimpleField(Attributes.RESOURCE_NAME.toString(), resourceId.stringify());
+    if (resourceId != null) {
+      setResourceName(resourceId.stringify());
+    }
+  }
+
+  /**
+   * Set the resource associated with this message
+   * @param resourceName resource name to set
+   */
+  public void setResourceName(String resourceName) {
+    _record.setSimpleField(Attributes.RESOURCE_NAME.toString(), resourceName);
   }
 
   /**
@@ -421,7 +576,15 @@ public class Message extends HelixProperty {
    * @return resource name
    */
   public ResourceId getResourceId() {
-    return ResourceId.from(_record.getSimpleField(Attributes.RESOURCE_NAME.toString()));
+    return ResourceId.from(getResourceName());
+  }
+
+  /**
+   * Get the resource associated with this message
+   * @return resource name
+   */
+  public String getResourceName() {
+    return _record.getSimpleField(Attributes.RESOURCE_NAME.toString());
   }
 
   /**
@@ -429,7 +592,15 @@ public class Message extends HelixProperty {
    * @return partition id
    */
   public PartitionId getPartitionId() {
-    return PartitionId.from(_record.getSimpleField(Attributes.PARTITION_NAME.toString()));
+    return PartitionId.from(getPartitionName());
+  }
+
+  /**
+   * Get the resource partition associated with this message
+   * @return partition id
+   */
+  public String getPartitionName() {
+    return _record.getSimpleField(Attributes.PARTITION_NAME.toString());
   }
 
   /**
@@ -453,7 +624,17 @@ public class Message extends HelixProperty {
    * @param stateModelDefName a reference to the state model definition, e.g. "MasterSlave"
    */
   public void setStateModelDef(StateModelDefId stateModelDefId) {
-    _record.setSimpleField(Attributes.STATE_MODEL_DEF.toString(), stateModelDefId.stringify());
+    if (stateModelDefId != null) {
+      setStateModelDef(stateModelDefId.stringify());
+    }
+  }
+
+  /**
+   * Set the state model definition
+   * @param stateModelDefName a reference to the state model definition, e.g. "MasterSlave"
+   */
+  public void setStateModelDef(String stateModelDefId) {
+    _record.setSimpleField(Attributes.STATE_MODEL_DEF.toString(), stateModelDefId);
   }
 
   /**
@@ -629,7 +810,7 @@ public class Message extends HelixProperty {
   public static Message createReplyMessage(Message srcMessage, String instanceName,
       Map<String, String> taskResultMap) {
     if (srcMessage.getCorrelationId() == null) {
-      throw new HelixException("Message " + srcMessage.getMsgId()
+      throw new HelixException("Message " + srcMessage.getMessageId()
           + " does not contain correlation id");
     }
     Message replyMessage =
@@ -679,6 +860,14 @@ public class Message extends HelixProperty {
   }
 
   /**
+   * Get a list of partitions associated with this message
+   * @return list of partition ids
+   */
+  public List<String> getPartitionNames() {
+    return _record.getListField(Attributes.PARTITION_NAME.toString());
+  }
+
+  /**
    * Check if this message is targetted for a controller
    * @return true if this is a controller message, false otherwise
    */
@@ -762,9 +951,9 @@ public class Message extends HelixProperty {
       boolean isNotValid =
           isNullOrEmpty(getTgtName()) || isNullOrEmpty(getPartitionId().stringify())
               || isNullOrEmpty(getResourceId().stringify()) || isNullOrEmpty(getStateModelDef())
-              || isNullOrEmpty(getToState().toString())
+              || isNullOrEmpty(getTypedToState().toString())
               || isNullOrEmpty(getStateModelFactoryName())
-              || isNullOrEmpty(getFromState().toString());
+              || isNullOrEmpty(getTypedFromState().toString());
 
       return !isNotValid;
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 c91a655..96d0ca7 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
@@ -33,6 +33,8 @@ import org.apache.helix.api.id.ResourceId;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 /**
  * Represents the assignments of replicas for an entire resource, keyed on partitions of the
@@ -75,7 +77,7 @@ public class ResourceAssignment extends HelixProperty {
    * Get the currently mapped partitions
    * @return list of Partition objects (immutable)
    */
-  public List<? extends PartitionId> getMappedPartitions() {
+  public List<? extends PartitionId> getMappedPartitionIds() {
     ImmutableList.Builder<PartitionId> builder = new ImmutableList.Builder<PartitionId>();
     for (String partitionName : _record.getMapFields().keySet()) {
       builder.add(PartitionId.from(partitionName));
@@ -84,6 +86,14 @@ public class ResourceAssignment extends HelixProperty {
   }
 
   /**
+   * Get the currently mapped partitions
+   * @return list of Partition objects (immutable)
+   */
+  public List<String> getMappedPartitions() {
+    return Lists.newArrayList(_record.getMapFields().keySet());
+  }
+
+  /**
    * Get the entire map of a resource
    * @return map of partition to participant to state
    */
@@ -111,13 +121,25 @@ public class ResourceAssignment extends HelixProperty {
   }
 
   /**
+   * Get the participant, state pairs for a partition
+   * @param partition the Partition to look up
+   * @return map of (participant id, state)
+   */
+  public Map<String, String> getReplicaMap(String partitionId) {
+    Map<String, String> rawReplicaMap = _record.getMapField(partitionId);
+    if (rawReplicaMap == null) {
+      return Collections.emptyMap();
+    }
+    return rawReplicaMap;
+  }
+
+  /**
    * Add participant, state pairs for a partition
-   * TODO: should be package-private, but builder can't see it
    * @param partitionId the partition to set
    * @param replicaMap map of (participant name, state)
    */
   public void addReplicaMap(PartitionId partitionId, Map<ParticipantId, State> replicaMap) {
-    Map<String, String> convertedMap = new HashMap<String, String>();
+    Map<String, String> convertedMap = Maps.newHashMap();
     for (ParticipantId participantId : replicaMap.keySet()) {
       convertedMap.put(participantId.stringify(), replicaMap.get(participantId).toString());
     }
@@ -125,6 +147,15 @@ public class ResourceAssignment extends HelixProperty {
   }
 
   /**
+   * Add participant, state pairs for a partition
+   * @param partitionId the partition to set
+   * @param replicaMap map of (participant name, state)
+   */
+  public void addReplicaMap(String partitionId, Map<String, String> replicaMap) {
+    _record.setMapField(partitionId, replicaMap);
+  }
+
+  /**
    * Helper for converting a map of strings to a concrete replica map
    * @param rawMap map of participant name to state name
    * @return map of participant id to state
@@ -133,7 +164,7 @@ public class ResourceAssignment extends HelixProperty {
     if (rawMap == null) {
       return Collections.emptyMap();
     }
-    Map<ParticipantId, State> replicaMap = new HashMap<ParticipantId, State>();
+    Map<ParticipantId, State> replicaMap = Maps.newHashMap();
     for (String participantName : rawMap.keySet()) {
       replicaMap.put(ParticipantId.from(participantName), State.from(rawMap.get(participantName)));
     }
@@ -150,8 +181,7 @@ public class ResourceAssignment extends HelixProperty {
     if (rawMaps == null) {
       return Collections.emptyMap();
     }
-    Map<PartitionId, Map<ParticipantId, State>> participantStateMaps =
-        new HashMap<PartitionId, Map<ParticipantId, State>>();
+    Map<PartitionId, Map<ParticipantId, State>> participantStateMaps = Maps.newHashMap();
     for (String partitionId : rawMaps.keySet()) {
       participantStateMaps.put(PartitionId.from(partitionId),
           replicaMapFromStringMap(rawMaps.get(partitionId)));
@@ -185,7 +215,7 @@ public class ResourceAssignment extends HelixProperty {
     if (replicaMaps == null) {
       return Collections.emptyMap();
     }
-    Map<String, Map<String, String>> rawMaps = new HashMap<String, Map<String, String>>();
+    Map<String, Map<String, String>> rawMaps = Maps.newHashMap();
     for (PartitionId partitionId : replicaMaps.keySet()) {
       rawMaps.put(partitionId.stringify(), stringMapFromReplicaMap(replicaMaps.get(partitionId)));
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 3a8542b..a9a6e49 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
@@ -175,7 +175,7 @@ public class StateModelDefinition extends HelixProperty {
    * Get an ordered priority list of states
    * @return state names, the first of which is highest priority
    */
-  public List<String> getStatesPriorityStringList() {
+  public List<String> getStatesPriorityList() {
     return _statesPriorityList;
   }
 
@@ -183,9 +183,9 @@ public class StateModelDefinition extends HelixProperty {
    * Get an ordered priority list of states
    * @return immutable list of states, the first of which is highest priority (immutable)
    */
-  public List<State> getStatesPriorityList() {
+  public List<State> getTypedStatesPriorityList() {
     ImmutableList.Builder<State> builder = new ImmutableList.Builder<State>();
-    for (String state : getStatesPriorityStringList()) {
+    for (String state : getStatesPriorityList()) {
       builder.add(State.from(state));
     }
     return builder.build();
@@ -223,7 +223,7 @@ public class StateModelDefinition extends HelixProperty {
    * Get the starting state in the model
    * @return name of the initial state
    */
-  public String getInitialStateString() {
+  public String getInitialState() {
     // return _record.getSimpleField(StateModelDefinitionProperty.INITIAL_STATE
     // .toString());
     return _initialState;
@@ -233,7 +233,7 @@ public class StateModelDefinition extends HelixProperty {
    * Get the starting state in the model
    * @return name of the initial state
    */
-  public State getInitialState() {
+  public State getTypedInitialState() {
     // return _record.getSimpleField(StateModelDefinitionProperty.INITIAL_STATE
     // .toString());
     return State.from(_initialState);
@@ -259,7 +259,7 @@ public class StateModelDefinition extends HelixProperty {
 
   @Override
   public boolean isValid() {
-    if (getInitialStateString() == null) {
+    if (getInitialState() == null) {
       _logger.error("State model does not contain init state, statemodel:" + _record.getId());
       return false;
     }
@@ -299,7 +299,16 @@ public class StateModelDefinition extends HelixProperty {
      * @param state
      */
     public Builder initialState(State initialState) {
-      this.initialState = initialState.toString();
+      return initialState(initialState.toString());
+    }
+
+    /**
+     * initial state of a replica when it starts, most commonly used initial
+     * state is OFFLINE
+     * @param state
+     */
+    public Builder initialState(String initialState) {
+      this.initialState = initialState;
       return this;
     }
 
@@ -312,7 +321,19 @@ public class StateModelDefinition extends HelixProperty {
      * @param states
      */
     public Builder addState(State state, int priority) {
-      statesMap.put(state.toString(), priority);
+      return addState(state.toString(), priority);
+    }
+
+    /**
+     * Define all valid states using this method. Set the priority in which the
+     * constraints must be satisfied. Lets say STATE1 has a constraint of 1 and
+     * STATE2 has a constraint of 3 but only one node is up then Helix will uses
+     * the priority to see STATE constraint has to be given higher preference <br/>
+     * Use -1 to indicates states with no constraints, like OFFLINE
+     * @param states
+     */
+    public Builder addState(String state, int priority) {
+      statesMap.put(state, priority);
       return this;
     }
 
@@ -326,6 +347,15 @@ public class StateModelDefinition extends HelixProperty {
     }
 
     /**
+     * Sets the priority to Integer.MAX_VALUE
+     * @param state
+     */
+    public Builder addState(String state) {
+      addState(state, Integer.MAX_VALUE);
+      return this;
+    }
+
+    /**
      * Define all legal transitions between states using this method. Priority
      * is used to order the transitions. Helix tries to maximize the number of
      * transitions that can be fired in parallel without violating the
@@ -343,6 +373,23 @@ public class StateModelDefinition extends HelixProperty {
     }
 
     /**
+     * Define all legal transitions between states using this method. Priority
+     * is used to order the transitions. Helix tries to maximize the number of
+     * transitions that can be fired in parallel without violating the
+     * constraint. The transitions are first sorted based on priority and
+     * transitions are selected in a greedy way until the constriants are not
+     * violated.
+     * @param fromState source
+     * @param toState destination
+     * @param priority priority, higher value is higher priority
+     * @return Builder
+     */
+    public Builder addTransition(String fromState, String toState, int priority) {
+      transitionMap.put(new Transition(fromState, toState), priority);
+      return this;
+    }
+
+    /**
      * Add a state transition with maximal priority value
      * @see #addTransition(String, String, int)
      * @param fromState
@@ -355,13 +402,35 @@ public class StateModelDefinition extends HelixProperty {
     }
 
     /**
+     * Add a state transition with maximal priority value
+     * @see #addTransition(String, String, int)
+     * @param fromState
+     * @param toState
+     * @return Builder
+     */
+    public Builder addTransition(String fromState, String toState) {
+      addTransition(fromState, toState, Integer.MAX_VALUE);
+      return this;
+    }
+
+    /**
      * Set a maximum for replicas in this state
      * @param state state name
      * @param upperBound maximum
      * @return Builder
      */
     public Builder upperBound(State state, int upperBound) {
-      stateConstraintMap.put(state.toString(), String.valueOf(upperBound));
+      return upperBound(state.toString(), upperBound);
+    }
+
+    /**
+     * Set a maximum for replicas in this state
+     * @param state state name
+     * @param upperBound maximum
+     * @return Builder
+     */
+    public Builder upperBound(String state, int upperBound) {
+      stateConstraintMap.put(state, String.valueOf(upperBound));
       return this;
     }
 
@@ -380,7 +449,25 @@ public class StateModelDefinition extends HelixProperty {
      * @return Builder
      */
     public Builder dynamicUpperBound(State state, String bound) {
-      stateConstraintMap.put(state.toString(), bound);
+      return dynamicUpperBound(state.toString(), bound);
+    }
+
+    /**
+     * You can use this to have the bounds dynamically change based on other
+     * parameters. <br/>
+     * Currently support 2 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.
+     * @param state
+     * @param bound
+     * @return Builder
+     */
+    public Builder dynamicUpperBound(String state, String bound) {
+      stateConstraintMap.put(state, bound);
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/model/Transition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Transition.java b/helix-core/src/main/java/org/apache/helix/model/Transition.java
index 16fc937..70f8635 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Transition.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Transition.java
@@ -38,6 +38,15 @@ public class Transition {
     _toState = toState;
   }
 
+  /**
+   * Instantiate with a source and destination state
+   * @param fromState source name
+   * @param toState destination name
+   */
+  public Transition(String fromState, String toState) {
+    this(State.from(fromState), State.from(toState));
+  }
+
   @Override
   public String toString() {
     return _fromState + "-" + _toState;
@@ -60,7 +69,7 @@ public class Transition {
    * Get the source state
    * @return source state name
    */
-  public State getFromState() {
+  public State getTypedFromState() {
     return _fromState;
   }
 
@@ -68,11 +77,27 @@ public class Transition {
    * Get the destination state
    * @return destination state name
    */
-  public State getToState() {
+  public State getTypedToState() {
     return _toState;
   }
 
   /**
+   * Get the source state
+   * @return source state name
+   */
+  public String getFromState() {
+    return _fromState.toString();
+  }
+
+  /**
+   * Get the destination state
+   * @return destination state name
+   */
+  public String getToState() {
+    return _toState.toString();
+  }
+
+  /**
    * Create a new transition
    * @param fromState string source state
    * @param toState string destination state

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java
index 4c6edf7..779f220 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/StateTransitionTableBuilder.java
@@ -126,8 +126,8 @@ public class StateTransitionTableBuilder {
     }
 
     for (Transition transition : transitions) {
-      State fromState = transition.getFromState();
-      State toState = transition.getToState();
+      State fromState = transition.getTypedFromState();
+      State toState = transition.getTypedToState();
       setPathVal(path, fromState.toString(), toState.toString(), 1);
       setNext(next, fromState.toString(), toState.toString(), toState.toString());
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 83b93d8..afd2886 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
@@ -89,7 +89,7 @@ public class ResourceMonitor implements ResourceMonitorMBean {
 
     // TODO fix this; IdealState shall have either map fields (CUSTOM mode)
     // or list fields (AUDO mode)
-    for (PartitionId partitionId : idealState.getPartitionSet()) {
+    for (PartitionId partitionId : idealState.getPartitionIdSet()) {
       Map<ParticipantId, State> idealRecord = idealState.getParticipantStateMap(partitionId);
       Map<ParticipantId, State> externalViewRecord = externalView.getStateMap(partitionId);
 
@@ -113,7 +113,7 @@ public class ResourceMonitor implements ResourceMonitorMBean {
     }
     _numOfErrorPartitions = numOfErrorPartitions;
     _externalViewIdealStateDiff = numOfDiff;
-    _numOfPartitionsInExternalView = externalView.getPartitionSet().size();
+    _numOfPartitionsInExternalView = externalView.getPartitionIdSet().size();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java
index 6ae8a1b..0e8c6fd 100644
--- a/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java
+++ b/helix-core/src/main/java/org/apache/helix/participant/DistClusterControllerElection.java
@@ -142,7 +142,7 @@ public class DistClusterControllerElection implements ControllerChangeListener {
 
     leader = accessor.getProperty(keyBuilder.controllerLeader());
     if (leader != null) {
-      String leaderSessionId = leader.getSessionId().stringify();
+      String leaderSessionId = leader.getTypedSessionId().stringify();
       LOG.info("Leader exists for cluster: " + manager.getClusterName() + ", currentLeader: "
           + leader.getInstanceName() + ", leaderSessionId: " + leaderSessionId);
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 2258b95..4e4fdf6 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
@@ -166,13 +166,13 @@ public class HelixStateMachineEngine implements StateMachineEngine {
 
     if (!type.equals(MessageType.STATE_TRANSITION.toString())) {
       throw new HelixException("Expect state-transition message type, but was "
-          + message.getMsgType() + ", msgId: " + message.getMsgId());
+          + message.getMsgType() + ", msgId: " + message.getMessageId());
     }
 
     PartitionId partitionKey = message.getPartitionId();
     StateModelDefId stateModelId = message.getStateModelDefId();
     ResourceId resourceId = message.getResourceId();
-    SessionId sessionId = message.getTgtSessionId();
+    SessionId sessionId = message.getTypedTgtSessionId();
     int bucketSize = message.getBucketSize();
 
     if (stateModelId == null) {
@@ -210,7 +210,7 @@ public class HelixStateMachineEngine implements StateMachineEngine {
 
     if (message.getBatchMessageMode() == false) {
       // create currentStateDelta for this partition
-      String initState = _stateModelDefs.get(message.getStateModelDef()).getInitialStateString();
+      String initState = _stateModelDefs.get(message.getStateModelDef()).getInitialState();
       StateModel stateModel = stateModelFactory.getStateModel(partitionKey.stringify());
       if (stateModel == null) {
         stateModel = stateModelFactory.createAndAddStateModel(partitionKey.stringify());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
index ed411d1..9bba660 100644
--- a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
@@ -133,7 +133,7 @@ public class RoutingTableProvider implements ExternalViewChangeListener, ConfigC
     if (externalViewList != null) {
       for (ExternalView extView : externalViewList) {
         String resourceName = extView.getId();
-        for (String partitionName : extView.getPartitionStringSet()) {
+        for (String partitionName : extView.getPartitionSet()) {
           Map<String, String> stateMap = extView.getStateMap(partitionName);
           for (String instanceName : stateMap.keySet()) {
             String currentState = stateMap.get(instanceName);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 ea03d58..f591a24 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
@@ -260,7 +260,7 @@ public class ClusterStateVerifier {
           ResourceAssignment resourceAssignment = bestPossOutput.getResourceAssignment(resourceId);
 
           ResourceAssignmentBuilder raBuilder = new ResourceAssignmentBuilder(resourceId);
-          List<? extends PartitionId> mappedPartitions = resourceAssignment.getMappedPartitions();
+          List<? extends PartitionId> mappedPartitions = resourceAssignment.getMappedPartitionIds();
           for (PartitionId partitionId : mappedPartitions) {
             raBuilder.addAssignments(partitionId, resourceAssignment.getReplicaMap(partitionId));
           }
@@ -315,7 +315,7 @@ public class ClusterStateVerifier {
         int extViewSize = extView.getRecord().getMapFields().size();
         int bestPossStateSize =
             bestPossOutput.getResourceAssignment(ResourceId.from(resourceName))
-                .getMappedPartitions().size();
+                .getMappedPartitionIds().size();
         if (extViewSize != bestPossStateSize) {
           LOG.info("exterView size (" + extViewSize + ") is different from bestPossState size ("
               + bestPossStateSize + ") for resource: " + resourceName);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 f3ed88e..8120981 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
@@ -66,7 +66,7 @@ public class MessagePoster {
     MessageId msgId = MessageId.from("TestMessageId-2");
 
     Message message = new Message(MessageType.STATE_TRANSITION, msgId);
-    message.setMsgId(msgId);
+    message.setMessageId(msgId);
     message.setSrcName(msgSrc);
     message.setTgtName(instanceName);
     message.setMsgState(MessageState.NEW);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java b/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java
index 58186d5..8b32ddc 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ZkLogAnalyzer.java
@@ -342,14 +342,14 @@ public class ZkLogAnalyzer {
               // sendMessageLines.add(inputLine);
               stats.msgSentCount++;
 
-              if (msg.getFromState().toString().equals("OFFLINE")
-                  && msg.getToState().toString().equals("SLAVE")) {
+              if (msg.getTypedFromState().toString().equals("OFFLINE")
+                  && msg.getTypedToState().toString().equals("SLAVE")) {
                 stats.msgSentCount_O2S++;
-              } else if (msg.getFromState().toString().equals("SLAVE")
-                  && msg.getToState().toString().equals("MASTER")) {
+              } else if (msg.getTypedFromState().toString().equals("SLAVE")
+                  && msg.getTypedToState().toString().equals("MASTER")) {
                 stats.msgSentCount_S2M++;
-              } else if (msg.getFromState().toString().equals("MASTER")
-                  && msg.getToState().toString().equals("SLAVE")) {
+              } else if (msg.getTypedFromState().toString().equals("MASTER")
+                  && msg.getTypedToState().toString().equals("SLAVE")) {
                 stats.msgSentCount_M2S++;
               }
               // System.out.println("Message create:"+new

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 e46ad13..d304a87 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
@@ -41,7 +41,7 @@ public class RebalanceUtil {
     Map<String, Integer> partitionIndex = new HashMap<String, Integer>();
     Map<String, String> reversePartitionIndex = new HashMap<String, String>();
     boolean indexInPartitionName = true;
-    for (PartitionId partitionId : state.getPartitionSet()) {
+    for (PartitionId partitionId : state.getPartitionIdSet()) {
       String partitionName = partitionId.stringify();
       int lastPos = partitionName.lastIndexOf("_");
       if (lastPos < 0) {
@@ -64,7 +64,7 @@ public class RebalanceUtil {
 
     if (indexInPartitionName == false) {
       List<String> partitions = new ArrayList<String>();
-      partitions.addAll(Lists.transform(Lists.newArrayList(state.getPartitionSet()),
+      partitions.addAll(Lists.transform(Lists.newArrayList(state.getPartitionIdSet()),
           Functions.toStringFunction()));
       Collections.sort(partitions);
       for (int i = 0; i < partitions.size(); i++) {
@@ -76,7 +76,7 @@ public class RebalanceUtil {
     Map<String, List<Integer>> nodeMasterAssignmentMap = new TreeMap<String, List<Integer>>();
     Map<String, Map<String, List<Integer>>> combinedNodeSlaveAssignmentMap =
         new TreeMap<String, Map<String, List<Integer>>>();
-    for (PartitionId partition : state.getPartitionSet()) {
+    for (PartitionId partition : state.getPartitionIdSet()) {
       List<String> instances = state.getRecord().getListField(partition.stringify());
       String master = instances.get(0);
       if (!nodeMasterAssignmentMap.containsKey(master)) {
@@ -111,7 +111,7 @@ public class RebalanceUtil {
 
     // StateModelDefinition def = new StateModelDefinition(stateModDef);
 
-    List<String> statePriorityList = stateModDef.getStatesPriorityStringList();
+    List<String> statePriorityList = stateModDef.getStatesPriorityList();
 
     for (String state : statePriorityList) {
       String count = stateModDef.getNumInstancesPerState(state);


[7/9] [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerRef.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerRef.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerRef.java
deleted file mode 100644
index a90b77a..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/RebalancerRef.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-/*
- * 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.util.HelixUtil;
-import org.apache.log4j.Logger;
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-/**
- * Reference to a class that extends {@link Rebalancer}. It loads the class automatically.
- */
-public class RebalancerRef {
-  private static final Logger LOG = Logger.getLogger(RebalancerRef.class);
-
-  @JsonProperty("rebalancerClassName")
-  private final String _rebalancerClassName;
-
-  @JsonCreator
-  private RebalancerRef(@JsonProperty("rebalancerClassName") String rebalancerClassName) {
-    _rebalancerClassName = rebalancerClassName;
-  }
-
-  /**
-   * Get an instantiated Rebalancer
-   * @return Rebalancer or null if instantiation failed
-   */
-  @JsonIgnore
-  public Rebalancer getRebalancer() {
-    try {
-      return (Rebalancer) (HelixUtil.loadClass(getClass(), _rebalancerClassName).newInstance());
-    } catch (Exception e) {
-      LOG.warn("Exception while invoking custom rebalancer class:" + _rebalancerClassName, e);
-    }
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    return _rebalancerClassName;
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that instanceof RebalancerRef) {
-      return this.toString().equals(((RebalancerRef) that).toString());
-    } else if (that instanceof String) {
-      return this.toString().equals(that);
-    }
-    return false;
-  }
-
-  /**
-   * Get a rebalancer class reference
-   * @param rebalancerClassName name of the class
-   * @return RebalancerRef or null if name is null
-   */
-  public static RebalancerRef from(String rebalancerClassName) {
-    if (rebalancerClassName == null) {
-      return null;
-    }
-    return new RebalancerRef(rebalancerClassName);
-  }
-
-  /**
-   * Get a RebalancerRef from a class object
-   * @param rebalancerClass class that implements Rebalancer
-   * @return RebalancerRef
-   */
-  public static RebalancerRef from(Class<? extends Rebalancer> rebalancerClass) {
-    if (rebalancerClass == null) {
-      return null;
-    }
-    return RebalancerRef.from(rebalancerClass.getName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java
deleted file mode 100644
index 525931d..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/ReplicatedRebalancerContext.java
+++ /dev/null
@@ -1,40 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-/*
- * 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.
- */
-
-/**
- * Methods specifying a rebalancer context that allows replicas. For instance, a rebalancer context
- * with partitions may accept state model definitions that support multiple replicas per partition,
- * and it's possible that the policy is that each live participant in the system should have a
- * replica.
- */
-public interface ReplicatedRebalancerContext extends RebalancerContext {
-  /**
-   * Check if this resource should be assigned to any live participant
-   * @return true if any live participant expected, false otherwise
-   */
-  public boolean anyLiveParticipant();
-
-  /**
-   * Get the number of replicas that each resource subunit should have
-   * @return replica count
-   */
-  public int getReplicaCount();
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 3f0dd13..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancer.java
+++ /dev/null
@@ -1,81 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-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;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-/*
- * 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.
- */
-
-/**
- * Rebalancer for the SEMI_AUTO mode. It expects a RebalancerConfig that understands the preferred
- * locations of each partition replica
- */
-public class SemiAutoRebalancer implements Rebalancer {
-  private static final Logger LOG = Logger.getLogger(SemiAutoRebalancer.class);
-
-  @Override
-  public void init(HelixManager helixManager) {
-    // do nothing
-  }
-
-  @Override
-  public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig,
-      Cluster cluster, ResourceCurrentState currentState) {
-    SemiAutoRebalancerContext config =
-        rebalancerConfig.getRebalancerContext(SemiAutoRebalancerContext.class);
-    StateModelDefinition stateModelDef =
-        cluster.getStateModelMap().get(config.getStateModelDefId());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing resource:" + config.getResourceId());
-    }
-    ResourceAssignment partitionMapping = new ResourceAssignment(config.getResourceId());
-    for (PartitionId partition : config.getPartitionSet()) {
-      Map<ParticipantId, State> currentStateMap =
-          currentState.getCurrentStateMap(config.getResourceId(), partition);
-      Set<ParticipantId> disabledInstancesForPartition =
-          NewConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
-              partition);
-      List<ParticipantId> preferenceList =
-          NewConstraintBasedAssignment.getPreferenceList(cluster, partition,
-              config.getPreferenceList(partition));
-      Map<State, String> upperBounds =
-          NewConstraintBasedAssignment.stateConstraints(stateModelDef, config.getResourceId(),
-              cluster.getConfig());
-      Map<ParticipantId, State> bestStateForPartition =
-          NewConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds, cluster
-              .getLiveParticipantMap().keySet(), stateModelDef, preferenceList, currentStateMap,
-              disabledInstancesForPartition);
-      partitionMapping.addReplicaMap(partition, bestStateForPartition);
-    }
-    return partitionMapping;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 72b3bc7..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.java
+++ /dev/null
@@ -1,176 +0,0 @@
-package org.apache.helix.controller.rebalancer.context;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-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.controller.rebalancer.util.NewConstraintBasedAssignment;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
-import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.StateModelDefinition;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonProperty;
-
-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.
- */
-
-/**
- * RebalancerContext for SEMI_AUTO rebalancer mode. It indicates the preferred locations of each
- * partition replica. By default, it corresponds to {@link SemiAutoRebalancer}
- */
-public final class SemiAutoRebalancerContext extends PartitionedRebalancerContext {
-  @JsonProperty("preferenceLists")
-  private Map<PartitionId, List<ParticipantId>> _preferenceLists;
-
-  /**
-   * Instantiate a SemiAutoRebalancerContext
-   */
-  public SemiAutoRebalancerContext() {
-    super(RebalanceMode.SEMI_AUTO);
-    setRebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class));
-    _preferenceLists = Maps.newHashMap();
-  }
-
-  /**
-   * Get the preference lists of all partitions of the resource
-   * @return map of partition id to list of participant ids
-   */
-  public Map<PartitionId, List<ParticipantId>> getPreferenceLists() {
-    return _preferenceLists;
-  }
-
-  /**
-   * Set the preference lists of all partitions of the resource
-   * @param preferenceLists
-   */
-  public void setPreferenceLists(Map<PartitionId, List<ParticipantId>> preferenceLists) {
-    _preferenceLists = preferenceLists;
-  }
-
-  /**
-   * Get the preference list of a partition
-   * @param partitionId the partition to look up
-   * @return list of participant ids
-   */
-  @JsonIgnore
-  public List<ParticipantId> getPreferenceList(PartitionId partitionId) {
-    return _preferenceLists.get(partitionId);
-  }
-
-  /**
-   * Generate preference lists based on a default cluster setup
-   * @param stateModelDef the state model definition to follow
-   * @param participantSet the set of participant ids to configure for
-   */
-  @Override
-  @JsonIgnore
-  public void generateDefaultConfiguration(StateModelDefinition stateModelDef,
-      Set<ParticipantId> participantSet) {
-    // compute default upper bounds
-    Map<State, String> upperBounds = Maps.newHashMap();
-    for (State state : stateModelDef.getTypedStatesPriorityList()) {
-      upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
-    }
-
-    // determine the current mapping
-    Map<PartitionId, Map<ParticipantId, State>> currentMapping = Maps.newHashMap();
-    for (PartitionId partitionId : getPartitionSet()) {
-      List<ParticipantId> preferenceList = getPreferenceList(partitionId);
-      if (preferenceList != null && !preferenceList.isEmpty()) {
-        Set<ParticipantId> disabledParticipants = Collections.emptySet();
-        Map<ParticipantId, State> emptyCurrentState = Collections.emptyMap();
-        Map<ParticipantId, State> initialMap =
-            NewConstraintBasedAssignment.computeAutoBestStateForPartition(upperBounds,
-                participantSet, stateModelDef, preferenceList, emptyCurrentState,
-                disabledParticipants);
-        currentMapping.put(partitionId, initialMap);
-      }
-    }
-
-    // determine the preference
-    LinkedHashMap<State, Integer> stateCounts =
-        NewConstraintBasedAssignment.stateCount(upperBounds, stateModelDef, participantSet.size(),
-            getReplicaCount());
-    ReplicaPlacementScheme placementScheme = new DefaultPlacementScheme();
-    List<ParticipantId> participantList = new ArrayList<ParticipantId>(participantSet);
-    List<PartitionId> partitionList = new ArrayList<PartitionId>(getPartitionSet());
-    AutoRebalanceStrategy strategy =
-        new AutoRebalanceStrategy(ResourceId.from(""), partitionList, stateCounts,
-            getMaxPartitionsPerParticipant(), placementScheme);
-    Map<String, List<String>> rawPreferenceLists =
-        strategy.typedComputePartitionAssignment(participantList, currentMapping, participantList)
-            .getListFields();
-    Map<PartitionId, List<ParticipantId>> preferenceLists =
-        Maps.newHashMap(IdealState.preferenceListsFromStringLists(rawPreferenceLists));
-    setPreferenceLists(preferenceLists);
-  }
-
-  /**
-   * Build a SemiAutoRebalancerContext. By default, it corresponds to {@link SemiAutoRebalancer}
-   */
-  public static final class Builder extends PartitionedRebalancerContext.AbstractBuilder<Builder> {
-    private final Map<PartitionId, List<ParticipantId>> _preferenceLists;
-
-    /**
-     * Instantiate for a resource
-     * @param resourceId resource id
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-      super.rebalancerRef(RebalancerRef.from(SemiAutoRebalancer.class));
-      _preferenceLists = Maps.newHashMap();
-    }
-
-    /**
-     * Add a preference list for a partition
-     * @param partitionId partition to set
-     * @param preferenceList ordered list of participants who can serve the partition
-     * @return Builder
-     */
-    public Builder preferenceList(PartitionId partitionId, List<ParticipantId> preferenceList) {
-      _preferenceLists.put(partitionId, preferenceList);
-      return self();
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-
-    @Override
-    public SemiAutoRebalancerContext build() {
-      SemiAutoRebalancerContext context = new SemiAutoRebalancerContext();
-      super.update(context);
-      context.setPreferenceLists(_preferenceLists);
-      return context;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
index 323be34..ce52a19 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
@@ -39,7 +39,9 @@ import org.apache.log4j.Logger;
 
 /**
  * Collection of functions that will compute the best possible states given the live instances and
- * an ideal state.
+ * an ideal state.<br/>
+ * <br/>
+ * Deprecated. Use {@link org.apache.helix.api.rebalancer.util.ConstraintBasedAssignment} instead.
  */
 @Deprecated
 public class ConstraintBasedAssignment {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index f703073..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/NewConstraintBasedAssignment.java
+++ /dev/null
@@ -1,244 +0,0 @@
-package org.apache.helix.controller.rebalancer.util;
-
-/*
- * 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.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-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.Participant;
-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;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-/**
- * Collection of functions that will compute the best possible state based on the participants and
- * the rebalancer configuration of a resource.
- */
-public class NewConstraintBasedAssignment {
-  private static Logger logger = Logger.getLogger(NewConstraintBasedAssignment.class);
-
-  /**
-   * Get a set of disabled participants for a partition
-   * @param participantMap map of all participants
-   * @param partitionId the partition to check
-   * @return a set of all participants that are disabled for the partition
-   */
-  public static Set<ParticipantId> getDisabledParticipants(
-      final Map<ParticipantId, Participant> participantMap, final PartitionId partitionId) {
-    Set<ParticipantId> participantSet = new HashSet<ParticipantId>(participantMap.keySet());
-    Set<ParticipantId> disabledParticipantsForPartition =
-        Sets.filter(participantSet, new Predicate<ParticipantId>() {
-          @Override
-          public boolean apply(ParticipantId participantId) {
-            Participant participant = participantMap.get(participantId);
-            return !participant.isEnabled()
-                || participant.getDisabledPartitionIds().contains(partitionId);
-          }
-        });
-    return disabledParticipantsForPartition;
-  }
-
-  /**
-   * Get an ordered list of participants that can serve a partition
-   * @param cluster cluster snapshot
-   * @param partitionId the partition to look up
-   * @param config rebalancing constraints
-   * @return list with most preferred participants first
-   */
-  public static List<ParticipantId> getPreferenceList(Cluster cluster, PartitionId partitionId,
-      List<ParticipantId> prefList) {
-    if (prefList != null && prefList.size() == 1
-        && StateModelToken.ANY_LIVEINSTANCE.toString().equals(prefList.get(0).stringify())) {
-      prefList = new ArrayList<ParticipantId>(cluster.getLiveParticipantMap().keySet());
-      Collections.sort(prefList);
-    }
-    return prefList;
-  }
-
-  /**
-   * Get a map of state to upper bound constraint given a cluster
-   * @param stateModelDef the state model definition to check
-   * @param resourceId the resource that is constraint
-   * @param cluster the cluster the resource belongs to
-   * @return map of state to upper bound
-   */
-  public static Map<State, String> stateConstraints(StateModelDefinition stateModelDef,
-      ResourceId resourceId, ClusterConfig cluster) {
-    Map<State, String> stateMap = Maps.newHashMap();
-    for (State state : stateModelDef.getTypedStatesPriorityList()) {
-      String num =
-          cluster.getStateUpperBoundConstraint(Scope.resource(resourceId),
-              stateModelDef.getStateModelDefId(), state);
-      stateMap.put(state, num);
-    }
-    return stateMap;
-  }
-
-  /**
-   * compute best state for resource in SEMI_AUTO and FULL_AUTO modes
-   * @param upperBounds map of state to upper bound
-   * @param liveParticipantSet set of live participant ids
-   * @param stateModelDef
-   * @param participantPreferenceList
-   * @param currentStateMap
-   *          : participant->state for each partition
-   * @param disabledParticipantsForPartition
-   * @return
-   */
-  public static Map<ParticipantId, State> computeAutoBestStateForPartition(
-      Map<State, String> upperBounds, Set<ParticipantId> liveParticipantSet,
-      StateModelDefinition stateModelDef, List<ParticipantId> participantPreferenceList,
-      Map<ParticipantId, State> currentStateMap, Set<ParticipantId> disabledParticipantsForPartition) {
-    Map<ParticipantId, State> participantStateMap = new HashMap<ParticipantId, State>();
-
-    // if the resource is deleted, instancePreferenceList will be empty and
-    // we should drop all resources.
-    if (currentStateMap != null) {
-      for (ParticipantId participantId : currentStateMap.keySet()) {
-        if ((participantPreferenceList == null || !participantPreferenceList
-            .contains(participantId)) && !disabledParticipantsForPartition.contains(participantId)) {
-          // if dropped and not disabled, transit to DROPPED
-          participantStateMap.put(participantId, State.from(HelixDefinedState.DROPPED));
-        } else if ((currentStateMap.get(participantId) == null || !currentStateMap.get(
-            participantId).equals(State.from(HelixDefinedState.ERROR)))
-            && disabledParticipantsForPartition.contains(participantId)) {
-          // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
-        }
-      }
-    }
-
-    // resource is deleted
-    if (participantPreferenceList == null) {
-      return participantStateMap;
-    }
-
-    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
-    boolean assigned[] = new boolean[participantPreferenceList.size()];
-
-    for (State state : statesPriorityList) {
-      String num = upperBounds.get(state);
-      int stateCount = -1;
-      if ("N".equals(num)) {
-        Set<ParticipantId> liveAndEnabled = new HashSet<ParticipantId>(liveParticipantSet);
-        liveAndEnabled.removeAll(disabledParticipantsForPartition);
-        stateCount = liveAndEnabled.size();
-      } else if ("R".equals(num)) {
-        stateCount = participantPreferenceList.size();
-      } else {
-        try {
-          stateCount = Integer.parseInt(num);
-        } catch (Exception e) {
-          logger.error("Invalid count for state:" + state + " ,count=" + num);
-        }
-      }
-      if (stateCount > -1) {
-        int count = 0;
-        for (int i = 0; i < participantPreferenceList.size(); i++) {
-          ParticipantId participantId = participantPreferenceList.get(i);
-
-          boolean notInErrorState =
-              currentStateMap == null
-                  || currentStateMap.get(participantId) == null
-                  || !currentStateMap.get(participantId)
-                      .equals(State.from(HelixDefinedState.ERROR));
-
-          if (liveParticipantSet.contains(participantId) && !assigned[i] && notInErrorState
-              && !disabledParticipantsForPartition.contains(participantId)) {
-            participantStateMap.put(participantId, state);
-            count = count + 1;
-            assigned[i] = true;
-            if (count == stateCount) {
-              break;
-            }
-          }
-        }
-      }
-    }
-    return participantStateMap;
-  }
-
-  /**
-   * Get the number of replicas that should be in each state for a partition
-   * @param upperBounds map of state to upper bound
-   * @param stateModelDef StateModelDefinition object
-   * @param liveNodesNb number of live nodes
-   * @param total number of replicas
-   * @return state count map: state->count
-   */
-  public static LinkedHashMap<State, Integer> stateCount(Map<State, String> upperBounds,
-      StateModelDefinition stateModelDef, int liveNodesNb, int totalReplicas) {
-    LinkedHashMap<State, Integer> stateCountMap = new LinkedHashMap<State, Integer>();
-    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
-
-    int replicas = totalReplicas;
-    for (State state : statesPriorityList) {
-      String num = upperBounds.get(state);
-      if ("N".equals(num)) {
-        stateCountMap.put(state, liveNodesNb);
-      } else if ("R".equals(num)) {
-        // wait until we get the counts for all other states
-        continue;
-      } else {
-        int stateCount = -1;
-        try {
-          stateCount = Integer.parseInt(num);
-        } catch (Exception e) {
-          // LOG.error("Invalid count for state: " + state + ", count: " + num +
-          // ", use -1 instead");
-        }
-
-        if (stateCount > 0) {
-          stateCountMap.put(state, stateCount);
-          replicas -= stateCount;
-        }
-      }
-    }
-
-    // get state count for R
-    for (State state : statesPriorityList) {
-      String num = upperBounds.get(state);
-      if ("R".equals(num)) {
-        stateCountMap.put(state, replicas);
-        // should have at most one state using R
-        break;
-      }
-    }
-    return stateCountMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 9699dcb..51301f0 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
@@ -20,114 +20,123 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
 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.api.id.StateModelDefId;
+import org.apache.helix.api.rebalancer.Rebalancer;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
+import org.apache.helix.api.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.controller.rebalancer.AutoRebalancer;
-import org.apache.helix.controller.rebalancer.CustomRebalancer;
-import org.apache.helix.controller.rebalancer.Rebalancer;
-import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.util.HelixUtil;
+import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
 /**
  * For partition compute best possible (instance,state) pair based on
  * IdealState,StateModel,LiveInstance
  */
-@Deprecated
 public class BestPossibleStateCalcStage extends AbstractBaseStage {
-  private static final Logger logger = Logger.getLogger(BestPossibleStateCalcStage.class.getName());
+  private static final Logger LOG = Logger.getLogger(BestPossibleStateCalcStage.class.getName());
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     long startTime = System.currentTimeMillis();
-    logger.info("START BestPossibleStateCalcStage.process()");
+    if (LOG.isInfoEnabled()) {
+      LOG.info("START BestPossibleStateCalcStage.process()");
+    }
 
-    CurrentStateOutput currentStateOutput =
+    ResourceCurrentState currentStateOutput =
         event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
+    Cluster cluster = event.getAttribute("ClusterDataCache");
 
-    if (currentStateOutput == null || resourceMap == null || cache == null) {
+    if (currentStateOutput == null || resourceMap == null || cluster == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires CURRENT_STATE|RESOURCES|DataCache");
     }
 
     BestPossibleStateOutput bestPossibleStateOutput =
-        compute(event, resourceMap, currentStateOutput);
+        compute(cluster, event, resourceMap, currentStateOutput);
     event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossibleStateOutput);
 
     long endTime = System.currentTimeMillis();
-    logger.info("END BestPossibleStateCalcStage.process(). took: " + (endTime - startTime) + " ms");
+    if (LOG.isInfoEnabled()) {
+      LOG.info("END BestPossibleStateCalcStage.process(). took: " + (endTime - startTime) + " ms");
+    }
   }
 
-  private BestPossibleStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      CurrentStateOutput currentStateOutput) {
-    // for each ideal state
-    // read the state model def
-    // for each resource
-    // get the preference list
-    // for each instanceName check if its alive then assign a state
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
+  /**
+   * Fallback for cases when the resource has been dropped, but current state exists
+   * @param cluster cluster snapshot
+   * @param resourceId the resource for which to generate an assignment
+   * @param currentStateOutput full snapshot of the current state
+   * @param stateModelDef state model the resource follows
+   * @return assignment for the dropped resource
+   */
+  private ResourceAssignment mapDroppedResource(Cluster cluster, ResourceId resourceId,
+      ResourceCurrentState currentStateOutput, StateModelDefinition stateModelDef) {
+    ResourceAssignment partitionMapping = new ResourceAssignment(resourceId);
+    Set<? extends PartitionId> mappedPartitions =
+        currentStateOutput.getCurrentStateMappedPartitions(resourceId);
+    if (mappedPartitions == null) {
+      return partitionMapping;
+    }
+    for (PartitionId partitionId : mappedPartitions) {
+      Set<ParticipantId> disabledParticipantsForPartition =
+          ConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
+              partitionId);
+      Map<State, String> upperBounds =
+          ConstraintBasedAssignment.stateConstraints(stateModelDef, resourceId,
+              cluster.getConfig());
+      partitionMapping.addReplicaMap(partitionId, ConstraintBasedAssignment
+          .computeAutoBestStateForPartition(upperBounds, cluster.getLiveParticipantMap().keySet(),
+              stateModelDef, null, currentStateOutput.getCurrentStateMap(resourceId, partitionId),
+              disabledParticipantsForPartition));
+    }
+    return partitionMapping;
+  }
 
+  private BestPossibleStateOutput compute(Cluster cluster, ClusterEvent event,
+      Map<ResourceId, ResourceConfig> resourceMap, ResourceCurrentState currentStateOutput) {
     BestPossibleStateOutput output = new BestPossibleStateOutput();
+    Map<StateModelDefId, StateModelDefinition> stateModelDefs = cluster.getStateModelMap();
 
-    for (String resourceName : resourceMap.keySet()) {
-      logger.debug("Processing resource:" + resourceName);
-
-      Resource resource = resourceMap.get(resourceName);
-      // Ideal state may be gone. In that case we need to get the state model name
-      // from the current state
-      IdealState idealState = cache.getIdealState(resourceName);
-
-      if (idealState == null) {
-        // if ideal state is deleted, use an empty one
-        logger.info("resource:" + resourceName + " does not exist anymore");
-        idealState = new IdealState(resourceName);
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Processing resource:" + resourceId);
       }
-
-      Rebalancer rebalancer = null;
-      if (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED
-          && idealState.getRebalancerRef() != null) {
-        String rebalancerClassName = idealState.getRebalancerRef().toString();
-        logger
-            .info("resource " + resourceName + " use idealStateRebalancer " + rebalancerClassName);
-        try {
-          rebalancer =
-              (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance());
-        } catch (Exception e) {
-          logger.warn("Exception while invoking custom rebalancer class:" + rebalancerClassName, e);
+      ResourceConfig resourceConfig = resourceMap.get(resourceId);
+      RebalancerConfig rebalancerConfig = resourceConfig.getRebalancerConfig();
+      ResourceAssignment resourceAssignment = null;
+      if (rebalancerConfig != null) {
+        Rebalancer rebalancer = rebalancerConfig.getRebalancer();
+        if (rebalancer != null) {
+          HelixManager manager = event.getAttribute("helixmanager");
+          rebalancer.init(manager);
+          resourceAssignment =
+              rebalancer.computeResourceMapping(rebalancerConfig, cluster, currentStateOutput);
         }
       }
-      if (rebalancer == null) {
-        if (idealState.getRebalanceMode() == RebalanceMode.FULL_AUTO) {
-          rebalancer = new AutoRebalancer();
-        } else if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
-          rebalancer = new SemiAutoRebalancer();
-        } else {
-          rebalancer = new CustomRebalancer();
-        }
+      if (resourceAssignment == null) {
+        RebalancerContext context = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
+        StateModelDefinition stateModelDef = stateModelDefs.get(context.getStateModelDefId());
+        resourceAssignment =
+            mapDroppedResource(cluster, resourceId, currentStateOutput, stateModelDef);
       }
 
-      HelixManager manager = event.getAttribute("helixmanager");
-      rebalancer.init(manager);
-      ResourceAssignment partitionStateAssignment =
-          rebalancer.computeResourceMapping(resource, idealState, currentStateOutput, cache);
-      for (Partition partition : resource.getPartitions()) {
-        Map<ParticipantId, State> newStateMap =
-            partitionStateAssignment.getReplicaMap(PartitionId.from(partition.getPartitionName()));
-        output.setParticipantStateMap(resourceName, partition, newStateMap);
-      }
+      output.setResourceAssignment(resourceId, resourceAssignment);
     }
+
     return output;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 362bbb6..afcb6f7 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
@@ -1,82 +1,49 @@
 package org.apache.helix.controller.stages;
 
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
-import org.apache.helix.api.State;
-import org.apache.helix.api.id.ParticipantId;
-import org.apache.helix.model.Partition;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.model.ResourceAssignment;
 
-@Deprecated
-public class BestPossibleStateOutput {
-  // resource->partition->instance->state
-  Map<String, Map<Partition, Map<String, String>>> _dataMap;
+import com.google.common.collect.Maps;
 
-  public BestPossibleStateOutput() {
-    _dataMap = new HashMap<String, Map<Partition, Map<String, String>>>();
-  }
+public class BestPossibleStateOutput {
 
-  public void setState(String resourceName, Partition resource,
-      Map<String, String> bestInstanceStateMappingForResource) {
-    if (!_dataMap.containsKey(resourceName)) {
-      _dataMap.put(resourceName, new HashMap<Partition, Map<String, String>>());
-    }
-    Map<Partition, Map<String, String>> map = _dataMap.get(resourceName);
-    map.put(resource, bestInstanceStateMappingForResource);
-  }
+  Map<ResourceId, ResourceAssignment> _resourceAssignmentMap;
 
-  public void setParticipantStateMap(String resourceName, Partition partition,
-      Map<ParticipantId, State> bestInstanceStateMappingForResource) {
-    Map<String, String> rawStateMap = new HashMap<String, String>();
-    for (ParticipantId participantId : bestInstanceStateMappingForResource.keySet()) {
-      rawStateMap.put(participantId.stringify(),
-          bestInstanceStateMappingForResource.get(participantId).toString());
-    }
-    setState(resourceName, partition, rawStateMap);
+  public BestPossibleStateOutput() {
+    _resourceAssignmentMap = Maps.newHashMap();
   }
 
-  public Map<String, String> getInstanceStateMap(String resourceName, Partition resource) {
-    Map<Partition, Map<String, String>> map = _dataMap.get(resourceName);
-    if (map != null) {
-      return map.get(resource);
-    }
-    return Collections.emptyMap();
+  /**
+   * Set the computed resource assignment for a resource
+   * @param resourceId the resource to set
+   * @param resourceAssignment the computed assignment
+   */
+  public void setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) {
+    _resourceAssignmentMap.put(resourceId, resourceAssignment);
   }
 
-  public Map<Partition, Map<String, String>> getResourceMap(String resourceName) {
-    Map<Partition, Map<String, String>> map = _dataMap.get(resourceName);
-    if (map != null) {
-      return map;
-    }
-    return Collections.emptyMap();
+  /**
+   * Get the resource assignment computed for a resource
+   * @param resourceId resource to look up
+   * @return ResourceAssignment computed by the best possible state calculation
+   */
+  public ResourceAssignment getResourceAssignment(ResourceId resourceId) {
+    return _resourceAssignmentMap.get(resourceId);
   }
 
-  public Map<String, Map<Partition, Map<String, String>>> getStateMap() {
-    return _dataMap;
+  /**
+   * Get all of the resources currently assigned
+   * @return set of assigned resource ids
+   */
+  public Set<ResourceId> getAssignedResources() {
+    return _resourceAssignmentMap.keySet();
   }
 
   @Override
   public String toString() {
-    return _dataMap.toString();
+    return _resourceAssignmentMap.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
index 64e881c..532ecb5 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
@@ -23,10 +23,12 @@ import java.util.Map;
 
 import org.apache.helix.HelixManager;
 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.id.ParticipantId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.model.LiveInstance;
 import org.apache.log4j.Logger;
 
 /**
@@ -38,16 +40,17 @@ public class CompatibilityCheckStage extends AbstractBaseStage {
   @Override
   public void process(ClusterEvent event) throws Exception {
     HelixManager manager = event.getAttribute("helixmanager");
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    if (manager == null || cache == null) {
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    if (manager == null || cluster == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires HelixManager | DataCache");
     }
 
     HelixManagerProperties properties = manager.getProperties();
-    Map<String, LiveInstance> liveInstanceMap = cache.getLiveInstances();
-    for (LiveInstance liveInstance : liveInstanceMap.values()) {
-      HelixVersion version = liveInstance.getTypedHelixVersion();
+    // Map<String, LiveInstance> liveInstanceMap = cache.getLiveInstances();
+    Map<ParticipantId, Participant> liveParticipants = cluster.getLiveParticipantMap();
+    for (Participant liveParticipant : liveParticipants.values()) {
+      HelixVersion version = liveParticipant.getRunningInstance().getVersion();
       String participantVersion = (version != null) ? version.toString() : null;
       if (!properties.isParticipantCompatible(participantVersion)) {
         String errorMsg =
@@ -55,7 +58,7 @@ public class CompatibilityCheckStage extends AbstractBaseStage {
                 + manager.getInstanceName() + ", controllerVersion: " + properties.getVersion()
                 + ", minimumSupportedParticipantVersion: "
                 + properties.getProperty("minimum_supported_version.participant")
-                + ", participant: " + liveInstance.getInstanceName() + ", participantVersion: "
+                + ", participant: " + liveParticipant.getId() + ", participantVersion: "
                 + participantVersion;
         LOG.error(errorMsg);
         throw new StageException(errorMsg);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 7036512..5730289 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,59 +22,68 @@ package org.apache.helix.controller.stages;
 import java.util.List;
 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.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.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 
 /**
  * For each LiveInstances select currentState and message whose sessionId matches
  * sessionId from LiveInstance Get Partition,State for all the resources computed in
  * previous State [ResourceComputationStage]
  */
-@Deprecated
 public class CurrentStateComputationStage extends AbstractBaseStage {
   @Override
   public void process(ClusterEvent event) throws Exception {
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
 
-    if (cache == null || resourceMap == null) {
+    if (cluster == null || resourceMap == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires DataCache|RESOURCE");
     }
 
-    Map<String, LiveInstance> liveInstances = cache.getLiveInstances();
-    CurrentStateOutput currentStateOutput = new CurrentStateOutput();
+    ResourceCurrentState currentStateOutput = new ResourceCurrentState();
 
-    for (LiveInstance instance : liveInstances.values()) {
-      String instanceName = instance.getInstanceName();
-      Map<String, Message> instanceMessages = cache.getMessages(instanceName);
-      for (Message message : instanceMessages.values()) {
+    for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) {
+      ParticipantId participantId = liveParticipant.getId();
+
+      // add pending messages
+      Map<MessageId, Message> instanceMsgs = liveParticipant.getMessageMap();
+      for (Message message : instanceMsgs.values()) {
         if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())) {
           continue;
         }
-        if (!instance.getTypedSessionId().equals(message.getTypedTgtSessionId())) {
+
+        if (!liveParticipant.getRunningInstance().getSessionId().equals(message.getTypedTgtSessionId())) {
           continue;
         }
+
         ResourceId resourceId = message.getResourceId();
-        Resource resource = resourceMap.get(resourceId.stringify());
+        ResourceConfig resource = resourceMap.get(resourceId);
         if (resource == null) {
           continue;
         }
 
         if (!message.getBatchMessageMode()) {
           PartitionId partitionId = message.getPartitionId();
-          Partition partition = resource.getPartition(partitionId.stringify());
+          Partition partition = resource.getSubUnit(partitionId);
           if (partition != null) {
-            currentStateOutput.setPendingState(resourceId.stringify(), partition, instanceName,
-                message.getTypedToState().toString());
+            currentStateOutput.setPendingState(resourceId, partitionId, participantId,
+                message.getTypedToState());
           } else {
             // log
           }
@@ -82,10 +91,10 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
           List<PartitionId> partitionNames = message.getPartitionIds();
           if (!partitionNames.isEmpty()) {
             for (PartitionId partitionId : partitionNames) {
-              Partition partition = resource.getPartition(partitionId.stringify());
+              Partition partition = resource.getSubUnit(partitionId);
               if (partition != null) {
-                currentStateOutput.setPendingState(resourceId.stringify(), partition, instanceName,
-                    message.getTypedToState().toString());
+                currentStateOutput.setPendingState(resourceId, partitionId, participantId,
+                    message.getTypedToState());
               } else {
                 // log
               }
@@ -93,43 +102,41 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
           }
         }
       }
-    }
-    for (LiveInstance instance : liveInstances.values()) {
-      String instanceName = instance.getInstanceName();
-
-      String clientSessionId = instance.getTypedSessionId().stringify();
-      Map<String, CurrentState> currentStateMap =
-          cache.getCurrentState(instanceName, clientSessionId);
-      for (CurrentState currentState : currentStateMap.values()) {
 
-        if (!instance.getTypedSessionId().equals(currentState.getTypedSessionId())) {
+      // add current state
+      SessionId sessionId = liveParticipant.getRunningInstance().getSessionId();
+      Map<ResourceId, CurrentState> curStateMap = liveParticipant.getCurrentStateMap();
+      for (CurrentState curState : curStateMap.values()) {
+        if (!sessionId.equals(curState.getTypedSessionId())) {
           continue;
         }
-        String resourceName = currentState.getResourceName();
-        String stateModelDefName = currentState.getStateModelDefRef();
-        Resource resource = resourceMap.get(resourceName);
+
+        ResourceId resourceId = curState.getResourceId();
+        StateModelDefId stateModelDefId = curState.getStateModelDefId();
+        ResourceConfig resource = resourceMap.get(resourceId);
         if (resource == null) {
           continue;
         }
-        if (stateModelDefName != null) {
-          currentStateOutput.setResourceStateModelDef(resourceName, stateModelDefName);
+
+        if (stateModelDefId != null) {
+          currentStateOutput.setResourceStateModelDef(resourceId, stateModelDefId);
         }
 
-        currentStateOutput.setBucketSize(resourceName, currentState.getBucketSize());
+        currentStateOutput.setBucketSize(resourceId, curState.getBucketSize());
 
-        Map<String, String> partitionStateMap = currentState.getPartitionStateMap();
-        for (String partitionName : partitionStateMap.keySet()) {
-          Partition partition = resource.getPartition(partitionName);
+        Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
+        for (PartitionId partitionId : partitionStateMap.keySet()) {
+          Partition partition = resource.getSubUnit(partitionId);
           if (partition != null) {
-            currentStateOutput.setCurrentState(resourceName, partition, instanceName,
-                currentState.getState(partitionName));
-
+            currentStateOutput.setCurrentState(resourceId, partitionId, participantId,
+                curState.getState(partitionId));
           } else {
             // log
           }
         }
       }
     }
+
     event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
index 3c3a9d9..5ecbddf 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
@@ -34,33 +34,39 @@ import org.apache.helix.PropertyKey.Builder;
 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.State;
+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.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.helix.model.StatusUpdate;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
 import org.apache.log4j.Logger;
 
-@Deprecated
 public class ExternalViewComputeStage extends AbstractBaseStage {
-  private static Logger log = Logger.getLogger(ExternalViewComputeStage.class);
+  private static Logger LOG = Logger.getLogger(ExternalViewComputeStage.class);
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     long startTime = System.currentTimeMillis();
-    log.info("START ExternalViewComputeStage.process()");
+    LOG.info("START ExternalViewComputeStage.process()");
 
     HelixManager manager = event.getAttribute("helixmanager");
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
+    Cluster cluster = event.getAttribute("ClusterDataCache");
 
-    if (manager == null || resourceMap == null || cache == null) {
+    if (manager == null || resourceMap == null || cluster == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires ClusterManager|RESOURCES|DataCache");
     }
@@ -68,58 +74,64 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
     HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
     PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
 
-    CurrentStateOutput currentStateOutput =
+    ResourceCurrentState currentStateOutput =
         event.getAttribute(AttributeName.CURRENT_STATE.toString());
 
     List<ExternalView> newExtViews = new ArrayList<ExternalView>();
     List<PropertyKey> keys = new ArrayList<PropertyKey>();
 
+    // TODO use external-view accessor
     Map<String, ExternalView> curExtViews =
         dataAccessor.getChildValuesMap(keyBuilder.externalViews());
 
-    for (String resourceName : resourceMap.keySet()) {
-      ExternalView view = new ExternalView(resourceName);
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      ExternalView view = new ExternalView(resourceId.stringify());
       // view.setBucketSize(currentStateOutput.getBucketSize(resourceName));
       // if resource ideal state has bucket size, set it
       // otherwise resource has been dropped, use bucket size from current state instead
-      Resource resource = resourceMap.get(resourceName);
+      ResourceConfig resource = resourceMap.get(resourceId);
+      RebalancerConfig rebalancerConfig = resource.getRebalancerConfig();
+      SchedulerTaskConfig schedulerTaskConfig = resource.getSchedulerTaskConfig();
+
       if (resource.getBucketSize() > 0) {
         view.setBucketSize(resource.getBucketSize());
       } else {
-        view.setBucketSize(currentStateOutput.getBucketSize(resourceName));
+        view.setBucketSize(currentStateOutput.getBucketSize(resourceId));
       }
-
-      for (Partition partition : resource.getPartitions()) {
-        Map<String, String> currentStateMap =
-            currentStateOutput.getCurrentStateMap(resourceName, partition);
+      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+        Map<ParticipantId, State> currentStateMap =
+            currentStateOutput.getCurrentStateMap(resourceId, partitionId);
         if (currentStateMap != null && currentStateMap.size() > 0) {
           // Set<String> disabledInstances
           // = cache.getDisabledInstancesForResource(resource.toString());
-          for (String instance : currentStateMap.keySet()) {
+          for (ParticipantId participantId : currentStateMap.keySet()) {
             // if (!disabledInstances.contains(instance))
             // {
-            view.setState(partition.getPartitionName(), instance, currentStateMap.get(instance));
+            view.setState(partitionId.stringify(), participantId.stringify(),
+                currentStateMap.get(participantId).toString());
             // }
           }
         }
       }
+
+      // TODO fix this
       // Update cluster status monitor mbean
-      ClusterStatusMonitor clusterStatusMonitor =
-          (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
-      IdealState idealState = cache._idealStateMap.get(view.getResourceName());
-      if (idealState != null) {
-        if (clusterStatusMonitor != null
-            && !idealState.getStateModelDefRef().equalsIgnoreCase(
-                DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
-          clusterStatusMonitor.onExternalViewChange(view,
-              cache._idealStateMap.get(view.getResourceName()));
-        }
-      }
+      // ClusterStatusMonitor clusterStatusMonitor =
+      // (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
+      // IdealState idealState = cache._idealStateMap.get(view.getResourceName());
+      // if (idealState != null) {
+      // if (clusterStatusMonitor != null
+      // && !idealState.getStateModelDefRef().equalsIgnoreCase(
+      // DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
+      // clusterStatusMonitor.onExternalViewChange(view,
+      // cache._idealStateMap.get(view.getResourceName()));
+      // }
+      // }
 
       // compare the new external view with current one, set only on different
-      ExternalView curExtView = curExtViews.get(resourceName);
+      ExternalView curExtView = curExtViews.get(resourceId.stringify());
       if (curExtView == null || !curExtView.getRecord().equals(view.getRecord())) {
-        keys.add(keyBuilder.externalView(resourceName));
+        keys.add(keyBuilder.externalView(resourceId.stringify()));
         newExtViews.add(view);
 
         // For SCHEDULER_TASK_RESOURCE resource group (helix task queue), we need to find out which
@@ -127,10 +139,13 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
         // partitions are finished (COMPLETED or ERROR), update the status update of the original
         // scheduler
         // message, and then remove the partitions from the ideal state
-        if (idealState != null
-            && idealState.getStateModelDefRef().equalsIgnoreCase(
-                DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
-          updateScheduledTaskStatus(view, manager, idealState);
+        RebalancerContext rebalancerContext =
+            (rebalancerConfig != null) ? rebalancerConfig
+                .getRebalancerContext(RebalancerContext.class) : null;
+        if (rebalancerContext != null
+            && rebalancerContext.getStateModelDefId().equalsIgnoreCase(
+                StateModelDefId.SchedulerTaskQueue)) {
+          updateScheduledTaskStatus(resourceId, view, manager, schedulerTaskConfig);
         }
       }
     }
@@ -144,18 +159,21 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
 
     // remove dead external-views
     for (String resourceName : curExtViews.keySet()) {
-      if (!resourceMap.keySet().contains(resourceName)) {
+      if (!resourceMap.containsKey(ResourceId.from(resourceName))) {
         dataAccessor.removeProperty(keyBuilder.externalView(resourceName));
       }
     }
 
     long endTime = System.currentTimeMillis();
-    log.info("END ExternalViewComputeStage.process(). took: " + (endTime - startTime) + " ms");
+    LOG.info("END ExternalViewComputeStage.process(). took: " + (endTime - startTime) + " ms");
   }
 
-  private void updateScheduledTaskStatus(ExternalView ev, HelixManager manager,
-      IdealState taskQueueIdealState) {
+  // TODO fix it
+  private void updateScheduledTaskStatus(ResourceId resourceId, ExternalView ev,
+      HelixManager manager, SchedulerTaskConfig schedulerTaskConfig) {
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    Builder keyBuilder = accessor.keyBuilder();
+
     ZNRecord finishedTasks = new ZNRecord(ev.getResourceName());
 
     // Place holder for finished partitions
@@ -166,23 +184,21 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
     Map<String, Map<String, String>> controllerMsgUpdates =
         new HashMap<String, Map<String, String>>();
 
-    Builder keyBuilder = accessor.keyBuilder();
-
     for (String taskPartitionName : ev.getPartitionSet()) {
       for (String taskState : ev.getStateMap(taskPartitionName).values()) {
         if (taskState.equalsIgnoreCase(HelixDefinedState.ERROR.toString())
             || taskState.equalsIgnoreCase("COMPLETED")) {
-          log.info(taskPartitionName + " finished as " + taskState);
-          finishedTasks.getListFields().put(taskPartitionName, emptyList);
-          finishedTasks.getMapFields().put(taskPartitionName, emptyMap);
+          LOG.info(taskPartitionName + " finished as " + taskState);
+          finishedTasks.setListField(taskPartitionName, emptyList);
+          finishedTasks.setMapField(taskPartitionName, emptyMap);
 
           // Update original scheduler message status update
-          if (taskQueueIdealState.getRecord().getMapField(taskPartitionName) != null) {
-            String controllerMsgId =
-                taskQueueIdealState.getRecord().getMapField(taskPartitionName)
-                    .get(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID);
+          Message innerMessage =
+              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
+          if (innerMessage != null) {
+            String controllerMsgId = innerMessage.getControllerMessagId();
             if (controllerMsgId != null) {
-              log.info(taskPartitionName + " finished with controllerMsg " + controllerMsgId);
+              LOG.info(taskPartitionName + " finished with controllerMsg " + controllerMsgId);
               if (!controllerMsgUpdates.containsKey(controllerMsgId)) {
                 controllerMsgUpdates.put(controllerMsgId, new HashMap<String, String>());
               }
@@ -193,16 +209,16 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
       }
     }
     // fill the controllerMsgIdCountMap
-    for (String taskId : taskQueueIdealState.getPartitionSet()) {
-      String controllerMsgId =
-          taskQueueIdealState.getRecord().getMapField(taskId)
-              .get(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID);
+    for (PartitionId taskId : schedulerTaskConfig.getPartitionSet()) {
+      Message innerMessage = schedulerTaskConfig.getInnerMessage(taskId);
+      String controllerMsgId = innerMessage.getControllerMessagId();
+
       if (controllerMsgId != null) {
-        if (!controllerMsgIdCountMap.containsKey(controllerMsgId)) {
-          controllerMsgIdCountMap.put(controllerMsgId, 0);
+        Integer curCnt = controllerMsgIdCountMap.get(controllerMsgId);
+        if (curCnt == null) {
+          curCnt = 0;
         }
-        controllerMsgIdCountMap.put(controllerMsgId,
-            (controllerMsgIdCountMap.get(controllerMsgId) + 1));
+        controllerMsgIdCountMap.put(controllerMsgId, curCnt + 1);
       }
     }
 
@@ -212,18 +228,16 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
             keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), controllerMsgId);
         StatusUpdate controllerStatusUpdate = accessor.getProperty(controllerStatusUpdateKey);
         for (String taskPartitionName : controllerMsgUpdates.get(controllerMsgId).keySet()) {
+          Message innerMessage =
+              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
+
           Map<String, String> result = new HashMap<String, String>();
           result.put("Result", controllerMsgUpdates.get(controllerMsgId).get(taskPartitionName));
           controllerStatusUpdate.getRecord().setMapField(
-              "MessageResult "
-                  + taskQueueIdealState.getRecord().getMapField(taskPartitionName)
-                      .get(Message.Attributes.TGT_NAME.toString())
-                  + " "
-                  + taskPartitionName
-                  + " "
-                  + taskQueueIdealState.getRecord().getMapField(taskPartitionName)
-                      .get(Message.Attributes.MSG_ID.toString()), result);
+              "MessageResult " + innerMessage.getTgtName() + " " + taskPartitionName + " "
+                  + innerMessage.getMessageId(), result);
         }
+
         // All done for the scheduled tasks that came from controllerMsgId, add summary for it
         if (controllerMsgUpdates.get(controllerMsgId).size() == controllerMsgIdCountMap.get(
             controllerMsgId).intValue()) {
@@ -255,12 +269,12 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
       ZNRecordDelta znDelta = new ZNRecordDelta(finishedTasks, MergeOperation.SUBTRACT);
       List<ZNRecordDelta> deltaList = new LinkedList<ZNRecordDelta>();
       deltaList.add(znDelta);
-      IdealState delta = new IdealState(taskQueueIdealState.getResourceName());
+      IdealState delta = new IdealState(resourceId);
       delta.setDeltaList(deltaList);
 
       // Remove the finished (COMPLETED or ERROR) tasks from the SCHEDULER_TASK_RESOURCE idealstate
       keyBuilder = accessor.keyBuilder();
-      accessor.updateProperty(keyBuilder.idealState(taskQueueIdealState.getResourceName()), delta);
+      accessor.updateProperty(keyBuilder.idealState(resourceId.stringify()), delta);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 3056cd5..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
+++ /dev/null
@@ -1,215 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.helix.HelixManager;
-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.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;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageState;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-/**
- * Compares the currentState, pendingState with IdealState and generate messages
- */
-@Deprecated
-public class MessageGenerationPhase extends AbstractBaseStage {
-  private static Logger logger = Logger.getLogger(MessageGenerationPhase.class);
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    HelixManager manager = event.getAttribute("helixmanager");
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
-    CurrentStateOutput currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
-    if (manager == null || cache == null || resourceMap == null || currentStateOutput == null
-        || bestPossibleStateOutput == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires HelixManager|DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE");
-    }
-
-    Map<String, LiveInstance> liveInstances = cache.getLiveInstances();
-    Map<String, String> sessionIdMap = new HashMap<String, String>();
-
-    for (LiveInstance liveInstance : liveInstances.values()) {
-      sessionIdMap
-          .put(liveInstance.getInstanceName(), liveInstance.getTypedSessionId().stringify());
-    }
-    MessageGenerationOutput output = new MessageGenerationOutput();
-
-    for (String resourceName : resourceMap.keySet()) {
-      Resource resource = resourceMap.get(resourceName);
-      int bucketSize = resource.getBucketSize();
-
-      StateModelDefinition stateModelDef = cache.getStateModelDef(resource.getStateModelDefRef());
-
-      for (Partition partition : resource.getPartitions()) {
-        Map<String, String> instanceStateMap =
-            bestPossibleStateOutput.getInstanceStateMap(resourceName, partition);
-
-        // we should generate message based on the desired-state priority
-        // so keep generated messages in a temp map keyed by state
-        // desired-state->list of generated-messages
-        Map<String, List<Message>> messageMap = new HashMap<String, List<Message>>();
-
-        for (String instanceName : instanceStateMap.keySet()) {
-          String desiredState = instanceStateMap.get(instanceName);
-
-          String currentState =
-              currentStateOutput.getCurrentState(resourceName, partition, instanceName);
-          if (currentState == null) {
-            currentState = stateModelDef.getInitialState();
-          }
-
-          if (desiredState.equalsIgnoreCase(currentState)) {
-            continue;
-          }
-
-          String pendingState =
-              currentStateOutput.getPendingState(resourceName, partition, instanceName);
-
-          String nextState = stateModelDef.getNextStateForTransition(currentState, desiredState);
-          if (nextState == null) {
-            logger.error("Unable to find a next state for partition: "
-                + partition.getPartitionName() + " from stateModelDefinition"
-                + stateModelDef.getClass() + " from:" + currentState + " to:" + desiredState);
-            continue;
-          }
-
-          if (pendingState != null) {
-            if (nextState.equalsIgnoreCase(pendingState)) {
-              logger.debug("Message already exists for " + instanceName + " to transit "
-                  + partition.getPartitionName() + " from " + currentState + " to " + nextState);
-            } else if (currentState.equalsIgnoreCase(pendingState)) {
-              logger.info("Message hasn't been removed for " + instanceName + " to transit"
-                  + partition.getPartitionName() + " to " + pendingState + ", desiredState: "
-                  + desiredState);
-            } else {
-              logger.info("IdealState changed before state transition completes for "
-                  + partition.getPartitionName() + " on " + instanceName + ", pendingState: "
-                  + pendingState + ", currentState: " + currentState + ", nextState: " + nextState);
-            }
-          } else {
-            Message message =
-                createMessage(manager, resourceName, partition.getPartitionName(), instanceName,
-                    currentState, nextState, sessionIdMap.get(instanceName), stateModelDef.getId(),
-                    resource.getStateModelFactoryname(), bucketSize);
-            IdealState idealState = cache.getIdealState(resourceName);
-            if (idealState != null
-                && idealState.getStateModelDefRef().equalsIgnoreCase(
-                    DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
-              if (idealState.getRecord().getMapField(partition.getPartitionName()) != null) {
-                message.getRecord().setMapField(Message.Attributes.INNER_MESSAGE.toString(),
-                    idealState.getRecord().getMapField(partition.getPartitionName()));
-              }
-            }
-            // Set timeout of needed
-            String stateTransition =
-                currentState + "-" + nextState + "_" + Message.Attributes.TIMEOUT;
-            if (idealState != null) {
-              String timeOutStr = idealState.getRecord().getSimpleField(stateTransition);
-              if (timeOutStr == null
-                  && idealState.getStateModelDefRef().equalsIgnoreCase(
-                      DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
-                // scheduled task queue
-                if (idealState.getRecord().getMapField(partition.getPartitionName()) != null) {
-                  timeOutStr =
-                      idealState.getRecord().getMapField(partition.getPartitionName())
-                          .get(Message.Attributes.TIMEOUT.toString());
-                }
-              }
-              if (timeOutStr != null) {
-                try {
-                  int timeout = Integer.parseInt(timeOutStr);
-                  if (timeout > 0) {
-                    message.setExecutionTimeout(timeout);
-                  }
-                } catch (Exception e) {
-                  logger.error("", e);
-                }
-              }
-            }
-            message.getRecord().setSimpleField("ClusterEventName", event.getName());
-            // output.addMessage(resourceName, partition, message);
-            if (!messageMap.containsKey(desiredState)) {
-              messageMap.put(desiredState, new ArrayList<Message>());
-            }
-            messageMap.get(desiredState).add(message);
-          }
-        }
-
-        // add generated messages to output according to state priority
-        List<String> statesPriorityList = stateModelDef.getStatesPriorityList();
-        for (String state : statesPriorityList) {
-          if (messageMap.containsKey(state)) {
-            for (Message message : messageMap.get(state)) {
-              output.addMessage(resourceName, partition, message);
-            }
-          }
-        }
-
-      } // end of for-each-partition
-    }
-    event.addAttribute(AttributeName.MESSAGES_ALL.toString(), output);
-  }
-
-  private Message createMessage(HelixManager manager, String resourceName, String partitionName,
-      String instanceName, String currentState, String nextState, String sessionId,
-      String stateModelDefName, String stateModelFactoryName, int bucketSize) {
-    MessageId uuid = MessageId.from(UUID.randomUUID().toString());
-    Message message = new Message(MessageType.STATE_TRANSITION, uuid);
-    message.setSrcName(manager.getInstanceName());
-    message.setTgtName(instanceName);
-    message.setMsgState(MessageState.NEW);
-    message.setPartitionId(PartitionId.from(partitionName));
-    message.setResourceId(ResourceId.from(resourceName));
-    message.setFromState(State.from(currentState));
-    message.setToState(State.from(nextState));
-    message.setTgtSessionId(SessionId.from(sessionId));
-    message.setSrcSessionId(SessionId.from(manager.getSessionId()));
-    message.setStateModelDef(StateModelDefId.from(stateModelDefName));
-    message.setStateModelFactoryName(stateModelFactoryName);
-    message.setBucketSize(bucketSize);
-
-    return message;
-  }
-}


[3/9] git commit: [HELIX-209] Backward compatible function naming in the model package

Posted by ka...@apache.org.
[HELIX-209] Backward compatible function naming in the model package


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

Branch: refs/heads/helix-logical-model
Commit: 6b57486b52a54a0b7593c2ad14c8402838ab4eef
Parents: 2d50077
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Tue Oct 15 16:24:26 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Tue Oct 15 16:24:26 2013 -0700

----------------------------------------------------------------------
 .../resources/SchedulerTasksResource.java       |  10 +-
 .../helix/tools/TestResetPartitionState.java    |   6 +-
 .../org/apache/helix/agent/AgentStateModel.java |   8 +-
 .../java/org/apache/helix/api/Controller.java   |   2 +-
 .../java/org/apache/helix/api/Resource.java     |   2 +-
 .../helix/api/accessor/ClusterAccessor.java     |   4 +-
 .../helix/api/accessor/ParticipantAccessor.java |  16 +-
 .../helix/api/accessor/ResourceAccessor.java    |   2 +-
 .../apache/helix/api/config/ClusterConfig.java  |   2 +-
 .../controller/GenericHelixController.java      |   2 +-
 .../controller/rebalancer/AutoRebalancer.java   |   2 +-
 .../controller/rebalancer/CustomRebalancer.java |   2 +-
 .../rebalancer/context/CustomRebalancer.java    |   2 +-
 .../context/CustomRebalancerContext.java        |   2 +-
 .../context/PartitionedRebalancerContext.java   |   8 +-
 .../context/SemiAutoRebalancerContext.java      |   2 +-
 .../util/ConstraintBasedAssignment.java         |   6 +-
 .../util/NewConstraintBasedAssignment.java      |   8 +-
 .../controller/stages/ClusterDataCache.java     |   4 +-
 .../stages/CompatibilityCheckStage.java         |   2 +-
 .../stages/CurrentStateComputationStage.java    |  12 +-
 .../stages/ExternalViewComputeStage.java        |   4 +-
 .../stages/MessageGenerationPhase.java          |   7 +-
 .../stages/MessageSelectionStage.java           |  12 +-
 .../stages/NewCurrentStateComputationStage.java |  10 +-
 .../stages/NewExternalViewComputeStage.java     |   4 +-
 .../stages/NewMessageGenerationStage.java       |   4 +-
 .../stages/NewMessageSelectionStage.java        |  12 +-
 .../stages/NewResourceComputationStage.java     |   6 +-
 .../stages/NewTaskAssignmentStage.java          |   8 +-
 .../stages/RebalanceIdealStateStage.java        |   2 +-
 .../stages/ResourceComputationStage.java        |  10 +-
 .../controller/stages/TaskAssignmentStage.java  |  10 +-
 .../helix/manager/zk/ControllerManager.java     |   2 +-
 .../manager/zk/CurStateCarryOverUpdater.java    |   2 +-
 .../DefaultControllerMessageHandlerFactory.java |   8 +-
 ...ltParticipantErrorMessageHandlerFactory.java |   4 +-
 .../DefaultSchedulerMessageHandlerFactory.java  |  38 +--
 .../zk/DistributedControllerManager.java        |   2 +-
 .../manager/zk/DistributedLeaderElection.java   |   2 +-
 .../manager/zk/ParticipantManagerHelper.java    |  11 +-
 .../apache/helix/manager/zk/ZKHelixAdmin.java   |  10 +-
 .../apache/helix/manager/zk/ZKHelixManager.java |   2 +-
 .../apache/helix/messaging/AsyncCallback.java   |   2 +-
 .../messaging/DefaultMessagingService.java      |   4 +-
 .../handling/AsyncCallbackService.java          |  12 +-
 .../handling/HelixStateTransitionHandler.java   |  24 +-
 .../helix/messaging/handling/HelixTask.java     |  26 +-
 .../messaging/handling/HelixTaskExecutor.java   |  12 +-
 .../messaging/handling/MessageTimeoutTask.java  |   2 +-
 .../org/apache/helix/model/AlertStatus.java     |  19 ++
 .../java/org/apache/helix/model/Alerts.java     |  19 ++
 .../apache/helix/model/ClusterConstraints.java  |  34 ++-
 .../org/apache/helix/model/CurrentState.java    |  41 +++-
 .../org/apache/helix/model/ExternalView.java    |   6 +-
 .../java/org/apache/helix/model/IdealState.java |  23 +-
 .../org/apache/helix/model/InstanceConfig.java  |   2 +-
 .../org/apache/helix/model/LiveInstance.java    |  28 ++-
 .../java/org/apache/helix/model/Message.java    | 243 ++++++++++++++++---
 .../apache/helix/model/ResourceAssignment.java  |  44 +++-
 .../helix/model/StateModelDefinition.java       | 107 +++++++-
 .../java/org/apache/helix/model/Transition.java |  29 ++-
 .../builder/StateTransitionTableBuilder.java    |   4 +-
 .../monitoring/mbeans/ResourceMonitor.java      |   4 +-
 .../DistClusterControllerElection.java          |   2 +-
 .../participant/HelixStateMachineEngine.java    |   6 +-
 .../helix/spectator/RoutingTableProvider.java   |   2 +-
 .../helix/tools/ClusterStateVerifier.java       |   4 +-
 .../org/apache/helix/tools/MessagePoster.java   |   2 +-
 .../org/apache/helix/tools/ZkLogAnalyzer.java   |  12 +-
 .../org/apache/helix/util/RebalanceUtil.java    |   8 +-
 .../org/apache/helix/util/StatusUpdateUtil.java |  26 +-
 .../org/apache/helix/TestHelixTaskExecutor.java |   2 +-
 .../org/apache/helix/TestHelixTaskHandler.java  |   4 +-
 .../java/org/apache/helix/ZkUnitTestBase.java   |   2 +-
 .../org/apache/helix/api/TestNewStages.java     |   6 +-
 .../stages/TestMsgSelectionStage.java           |   2 +-
 .../stages/TestRebalancePipeline.java           |  20 +-
 .../stages/TestResourceComputationStage.java    |   2 +-
 .../strategy/TestNewAutoRebalanceStrategy.java  |   2 +-
 .../helix/healthcheck/TestAddDropAlert.java     |   4 +-
 .../helix/healthcheck/TestExpandAlert.java      |   4 +-
 .../helix/healthcheck/TestSimpleAlert.java      |   4 +-
 .../healthcheck/TestSimpleWildcardAlert.java    |   4 +-
 .../helix/healthcheck/TestStalenessAlert.java   |   4 +-
 .../helix/healthcheck/TestWildcardAlert.java    |   4 +-
 .../helix/integration/TestAutoRebalance.java    |   2 +-
 .../TestAutoRebalancePartitionLimit.java        |  12 +-
 .../integration/TestCleanupExternalView.java    |   4 +-
 .../TestCustomizedIdealStateRebalancer.java     |  10 +-
 .../org/apache/helix/integration/TestDrop.java  |   2 +-
 .../TestEnablePartitionDuringDisable.java       |   4 +-
 .../helix/integration/TestHelixInstanceTag.java |   4 +-
 .../TestMessagePartitionStateMismatch.java      |   6 +-
 .../helix/integration/TestMessagingService.java |  18 +-
 .../integration/TestResetPartitionState.java    |   6 +-
 .../helix/integration/TestSchedulerMessage.java |  18 +-
 .../integration/TestSchedulerMsgContraints.java |   2 +-
 .../integration/TestSchedulerMsgUsingQueue.java |   6 +-
 .../integration/TestStateTransitionTimeout.java |   4 +-
 .../helix/integration/TestStatusUpdate.java     |   2 +-
 .../helix/manager/zk/TestZkClusterManager.java  |  10 +-
 .../helix/messaging/TestAsyncCallbackSvc.java   |  10 +-
 .../handling/TestHelixTaskExecutor.java         |   8 +-
 .../helix/mock/controller/MockController.java   |   2 +-
 .../helix/mock/participant/ErrTransition.java   |   4 +-
 .../apache/helix/tools/TestHelixAdminCli.java   |   6 +-
 .../examples/MasterSlaveStateModelFactory.java  |  16 +-
 .../org/apache/helix/examples/Quickstart.java   |   2 +-
 .../helix/lockmanager/LockManagerDemo.java      |   2 +-
 .../helix/filestore/FileStoreStateModel.java    |  24 +-
 .../org/apache/helix/taskexecution/Task.java    |   2 +-
 112 files changed, 850 insertions(+), 421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 40c527a..942566c 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
@@ -159,7 +159,7 @@ public class SchedulerTasksResource extends Resource {
 
       schedulerMessage.getRecord().getMapFields().put(MESSAGETEMPLATE, messageTemplate);
 
-      schedulerMessage.setTgtSessionId(SessionId.from(leader.getSessionId().stringify()));
+      schedulerMessage.setTgtSessionId(SessionId.from(leader.getTypedSessionId().stringify()));
       schedulerMessage.setTgtName("CONTROLLER");
       schedulerMessage.setSrcInstanceType(InstanceType.CONTROLLER);
       String taskQueueName =
@@ -169,22 +169,22 @@ public class SchedulerTasksResource extends Resource {
             DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE, taskQueueName);
       }
       accessor.setProperty(
-          accessor.keyBuilder().controllerMessage(schedulerMessage.getMsgId().stringify()),
+          accessor.keyBuilder().controllerMessage(schedulerMessage.getMessageId().stringify()),
           schedulerMessage);
 
       Map<String, String> resultMap = new HashMap<String, String>();
       resultMap.put("StatusUpdatePath", PropertyPathConfig.getPath(
           PropertyType.STATUSUPDATES_CONTROLLER, clusterName, MessageType.SCHEDULER_MSG.toString(),
-          schedulerMessage.getMsgId().stringify()));
+          schedulerMessage.getMessageId().stringify()));
       resultMap.put("MessageType", Message.MessageType.SCHEDULER_MSG.toString());
-      resultMap.put("MsgId", schedulerMessage.getMsgId().stringify());
+      resultMap.put("MsgId", schedulerMessage.getMessageId().stringify());
 
       // Assemble the rest URL for task status update
       String ipAddress = InetAddress.getLocalHost().getCanonicalHostName();
       String url =
           "http://" + ipAddress + ":" + getContext().getAttributes().get(RestAdminApplication.PORT)
               + "/clusters/" + clusterName + "/Controller/statusUpdates/SCHEDULER_MSG/"
-              + schedulerMessage.getMsgId();
+              + schedulerMessage.getMessageId();
       resultMap.put("statusUpdateUrl", url);
 
       getResponse().setEntity(ClusterRepresentationUtil.ObjectToJson(resultMap),

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
----------------------------------------------------------------------
diff --git a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
index ec52151..9b07445 100644
--- a/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
+++ b/helix-admin-webapp/src/test/java/org/apache/helix/tools/TestResetPartitionState.java
@@ -68,8 +68,8 @@ public class TestResetPartitionState extends AdminTestBase {
     @Override
     public void doTransition(Message message, NotificationContext context) {
       super.doTransition(message, context);
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       if (fromState.toString().equals("ERROR") && toState.toString().equals("OFFLINE")) {
         // System.err.println("doReset() invoked");
         _errToOfflineInvoked.incrementAndGet();
@@ -192,7 +192,7 @@ public class TestResetPartitionState extends AdminTestBase {
     Builder keyBuilder = accessor.keyBuilder();
 
     LiveInstance liveInstance = accessor.getProperty(keyBuilder.liveInstance(instance));
-    accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getSessionId()
+    accessor.removeProperty(keyBuilder.stateTransitionStatus(instance, liveInstance.getTypedSessionId()
         .stringify(), resource, partition));
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java
----------------------------------------------------------------------
diff --git a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java
index b0106a0..d227ac3 100644
--- a/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java
+++ b/helix-agent/src/main/java/org/apache/helix/agent/AgentStateModel.java
@@ -72,8 +72,8 @@ public class AgentStateModel extends StateModel {
 
     HelixManager manager = context.getManager();
     String clusterName = manager.getClusterName();
-    State fromState = message.getFromState();
-    State toState = message.getToState();
+    State fromState = message.getTypedFromState();
+    State toState = message.getTypedToState();
 
     // construct keys for command-config
     String cmdKey = buildKey(fromState.toString(), toState.toString(), CommandAttribute.COMMAND);
@@ -116,8 +116,8 @@ public class AgentStateModel extends StateModel {
     }
 
     if (cmd == null) {
-      throw new Exception("Unable to find command for transition from:" + message.getFromState()
-          + " to:" + message.getToState());
+      throw new Exception("Unable to find command for transition from:" + message.getTypedFromState()
+          + " to:" + message.getTypedToState());
     }
     _logger.info("Executing command: " + cmd + ", using workingDir: " + workingDir + ", timeout: "
         + timeout + ", on " + manager.getInstanceName());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 33e85ed..1218287 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
@@ -39,7 +39,7 @@ public class Controller {
 
     if (liveInstance != null) {
       _runningInstance =
-          new RunningInstance(liveInstance.getSessionId(), liveInstance.getHelixVersion(),
+          new RunningInstance(liveInstance.getTypedSessionId(), liveInstance.getTypedHelixVersion(),
               liveInstance.getProcessId());
     } else {
       _runningInstance = null;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 589c8a6..79a1e09 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
@@ -96,7 +96,7 @@ public class Resource {
 
     Map<PartitionId, Message> innerMsgMap = new HashMap<PartitionId, Message>();
     if (idealState.getStateModelDefId().equalsIgnoreCase(StateModelDefId.SchedulerTaskQueue)) {
-      for (PartitionId partitionId : idealState.getPartitionSet()) {
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
         // TODO refactor: scheduler-task-queue state model uses map-field to store inner-messages
         // this is different from all other state-models
         Map<String, String> innerMsgStrMap =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
index abeb649..85b8432 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ClusterAccessor.java
@@ -372,7 +372,7 @@ public class ClusterAccessor {
         new HashMap<String, Map<String, CurrentState>>();
     for (String participantName : liveInstanceMap.keySet()) {
       LiveInstance liveInstance = liveInstanceMap.get(participantName);
-      SessionId sessionId = liveInstance.getSessionId();
+      SessionId sessionId = liveInstance.getTypedSessionId();
       Map<String, CurrentState> instanceCurStateMap =
           _accessor.getChildValuesMap(_keyBuilder.currentStates(participantName,
               sessionId.stringify()));
@@ -794,7 +794,7 @@ public class ClusterAccessor {
     }
     Set<PartitionId> disabledPartitions = participant.getDisabledPartitions();
     for (PartitionId partitionId : disabledPartitions) {
-      instanceConfig.setInstanceEnabledForPartition(partitionId, false);
+      instanceConfig.setParticipantEnabledForPartition(partitionId, false);
     }
     _accessor.setProperty(_keyBuilder.instanceConfig(participantId.stringify()), instanceConfig);
     return true;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
index ac8f79d..83dd53e 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ParticipantAccessor.java
@@ -277,7 +277,7 @@ public class ParticipantAccessor {
     for (ParticipantId participantId : resetParticipantIdSet) {
       for (ExternalView extView : extViews) {
         Set<PartitionId> resetPartitionIdSet = Sets.newHashSet();
-        for (PartitionId partitionId : extView.getPartitionSet()) {
+        for (PartitionId partitionId : extView.getPartitionIdSet()) {
           Map<ParticipantId, State> stateMap = extView.getStateMap(partitionId);
           if (stateMap.containsKey(participantId)
               && stateMap.get(participantId).equals(State.from(HelixDefinedState.ERROR))) {
@@ -346,7 +346,7 @@ public class ParticipantAccessor {
     // make sure that there are no pending transition messages
     for (Message message : participant.getMessageMap().values()) {
       if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())
-          || !runningInstance.getSessionId().equals(message.getTgtSessionId())
+          || !runningInstance.getSessionId().equals(message.getTypedTgtSessionId())
           || !resourceId.equals(message.getResourceId())
           || !resetPartitionIdSet.contains(message.getPartitionId())) {
         continue;
@@ -384,10 +384,10 @@ public class ParticipantAccessor {
       message.setTgtSessionId(runningInstance.getSessionId());
       message.setStateModelDef(stateModelDefId);
       message.setFromState(State.from(HelixDefinedState.ERROR.toString()));
-      message.setToState(stateModelDef.getInitialState());
+      message.setToState(stateModelDef.getTypedInitialState());
       message.setStateModelFactoryId(context.getStateModelFactoryId());
 
-      messageMap.put(message.getMsgId(), message);
+      messageMap.put(message.getMessageId(), message);
     }
 
     // send the messages
@@ -477,7 +477,7 @@ public class ParticipantAccessor {
       instanceConfig.addTag(tag);
     }
     for (PartitionId partitionId : participantConfig.getDisabledPartitions()) {
-      instanceConfig.setInstanceEnabledForPartition(partitionId, false);
+      instanceConfig.setParticipantEnabledForPartition(partitionId, false);
     }
     instanceConfig.setInstanceEnabled(participantConfig.isEnabled());
     instanceConfig.addNamespacedConfig(participantConfig.getUserConfig());
@@ -528,7 +528,7 @@ public class ParticipantAccessor {
     RunningInstance runningInstance = null;
     if (liveInstance != null) {
       runningInstance =
-          new RunningInstance(liveInstance.getSessionId(), liveInstance.getHelixVersion(),
+          new RunningInstance(liveInstance.getTypedSessionId(), liveInstance.getTypedHelixVersion(),
               liveInstance.getProcessId());
     }
 
@@ -574,7 +574,7 @@ public class ParticipantAccessor {
     Map<String, Message> instanceMsgMap = Collections.emptyMap();
     Map<String, CurrentState> instanceCurStateMap = Collections.emptyMap();
     if (liveInstance != null) {
-      SessionId sessionId = liveInstance.getSessionId();
+      SessionId sessionId = liveInstance.getTypedSessionId();
 
       instanceMsgMap = _accessor.getChildValuesMap(_keyBuilder.messages(participantName));
       instanceCurStateMap =
@@ -681,7 +681,7 @@ public class ParticipantAccessor {
    */
   protected void swapParticipantsInIdealState(IdealState idealState,
       ParticipantId oldParticipantId, ParticipantId newParticipantId) {
-    for (PartitionId partitionId : idealState.getPartitionSet()) {
+    for (PartitionId partitionId : idealState.getPartitionIdSet()) {
       List<ParticipantId> oldPreferenceList = idealState.getPreferenceList(partitionId);
       if (oldPreferenceList != null) {
         List<ParticipantId> newPreferenceList = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
index 58b226d..517c8c4 100644
--- a/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/accessor/ResourceAccessor.java
@@ -292,7 +292,7 @@ public class ResourceAccessor {
       }
 
       Map<ParticipantId, Set<PartitionId>> resetPartitionIds = Maps.newHashMap();
-      for (PartitionId partitionId : extView.getPartitionSet()) {
+      for (PartitionId partitionId : extView.getPartitionIdSet()) {
         Map<ParticipantId, State> stateMap = extView.getStateMap(partitionId);
         for (ParticipantId participantId : stateMap.keySet()) {
           State state = stateMap.get(participantId);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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
index ed9750a..22a1528 100644
--- 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
@@ -767,7 +767,7 @@ public class ClusterConfig {
     public Builder addStateModelDefinition(StateModelDefinition stateModelDef) {
       _stateModelMap.put(stateModelDef.getStateModelDefId(), stateModelDef);
       // add state constraints from the state model definition
-      for (State state : stateModelDef.getStatesPriorityList()) {
+      for (State state : stateModelDef.getTypedStatesPriorityList()) {
         if (!stateModelDef.getNumParticipantsPerState(state).equals("-1")) {
           addStateUpperBoundConstraint(Scope.cluster(_id), stateModelDef.getStateModelDefId(),
               state, stateModelDef.getNumParticipantsPerState(state));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
index 3c118ce..b27398d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
@@ -469,7 +469,7 @@ public class GenericHelixController implements ConfigChangeListener, IdealStateC
     Map<String, LiveInstance> curSessions = new HashMap<String, LiveInstance>();
     for (LiveInstance liveInstance : liveInstances) {
       curInstances.put(liveInstance.getInstanceName(), liveInstance);
-      curSessions.put(liveInstance.getSessionId().stringify(), liveInstance);
+      curSessions.put(liveInstance.getTypedSessionId().stringify(), liveInstance);
     }
 
     Map<String, LiveInstance> lastInstances = _lastSeenInstances.get();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 6d65009..880f2c0 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
@@ -74,7 +74,7 @@ public class AutoRebalancer implements Rebalancer {
   public ResourceAssignment computeResourceMapping(Resource resource, IdealState currentIdealState,
       CurrentStateOutput currentStateOutput, ClusterDataCache clusterData) {
     // Compute a preference list based on the current ideal state
-    List<String> partitions = new ArrayList<String>(currentIdealState.getPartitionStringSet());
+    List<String> partitions = new ArrayList<String>(currentIdealState.getPartitionSet());
     String stateModelName = currentIdealState.getStateModelDefRef();
     StateModelDefinition stateModelDef = clusterData.getStateModelDef(stateModelName);
     Map<String, LiveInstance> liveInstance = clusterData.getLiveInstances();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 512af80..f6ea60f 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
@@ -108,7 +108,7 @@ public class CustomRebalancer implements Rebalancer {
             HelixDefinedState.ERROR.toString()))
             && disabledInstancesForPartition.contains(instance)) {
           // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          instanceStateMap.put(instance, stateModelDef.getInitialStateString());
+          instanceStateMap.put(instance, stateModelDef.getInitialState());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 d245fae..00219af 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
@@ -97,7 +97,7 @@ public class CustomRebalancer implements Rebalancer {
             participantId).equals(State.from(HelixDefinedState.ERROR)))
             && disabledParticipantsForPartition.contains(participantId)) {
           // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          participantStateMap.put(participantId, stateModelDef.getInitialState());
+          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 904907e..6e1485b 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
@@ -93,7 +93,7 @@ public class CustomRebalancerContext extends PartitionedRebalancerContext {
       Set<ParticipantId> participantSet) {
     // compute default upper bounds
     Map<State, String> upperBounds = Maps.newHashMap();
-    for (State state : stateModelDef.getStatesPriorityList()) {
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
       upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 decc78d..768e40c 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
@@ -180,7 +180,7 @@ public class PartitionedRebalancerContext extends BasicRebalancerContext impleme
     case SEMI_AUTO:
       SemiAutoRebalancerContext.Builder semiAutoBuilder =
           new SemiAutoRebalancerContext.Builder(idealState.getResourceId());
-      for (PartitionId partitionId : idealState.getPartitionSet()) {
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
         semiAutoBuilder.preferenceList(partitionId, idealState.getPreferenceList(partitionId));
       }
       populateContext(semiAutoBuilder, idealState);
@@ -189,7 +189,7 @@ public class PartitionedRebalancerContext extends BasicRebalancerContext impleme
     case CUSTOMIZED:
       CustomRebalancerContext.Builder customBuilder =
           new CustomRebalancerContext.Builder(idealState.getResourceId());
-      for (PartitionId partitionId : idealState.getPartitionSet()) {
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
         customBuilder.preferenceMap(partitionId, idealState.getParticipantStateMap(partitionId));
       }
       populateContext(customBuilder, idealState);
@@ -219,11 +219,11 @@ public class PartitionedRebalancerContext extends BasicRebalancerContext impleme
     } else {
       replicaCount = Integer.parseInt(replicas);
     }
-    if (idealState.getNumPartitions() > 0 && idealState.getPartitionSet().size() == 0) {
+    if (idealState.getNumPartitions() > 0 && idealState.getPartitionIdSet().size() == 0) {
       // backwards compatibility: partition sets were based on pref lists/maps previously
       builder.addPartitions(idealState.getNumPartitions());
     } else {
-      for (PartitionId partitionId : idealState.getPartitionSet()) {
+      for (PartitionId partitionId : idealState.getPartitionIdSet()) {
         builder.addPartition(new Partition(partitionId));
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 71b5076..72b3bc7 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
@@ -96,7 +96,7 @@ public final class SemiAutoRebalancerContext extends PartitionedRebalancerContex
       Set<ParticipantId> participantSet) {
     // compute default upper bounds
     Map<State, String> upperBounds = Maps.newHashMap();
-    for (State state : stateModelDef.getStatesPriorityList()) {
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
       upperBounds.put(state, stateModelDef.getNumParticipantsPerState(state));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
index dce0a07..323be34 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/ConstraintBasedAssignment.java
@@ -87,7 +87,7 @@ public class ConstraintBasedAssignment {
             HelixDefinedState.ERROR.toString()))
             && disabledInstancesForPartition.contains(instance)) {
           // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          instanceStateMap.put(instance, stateModelDef.getInitialStateString());
+          instanceStateMap.put(instance, stateModelDef.getInitialState());
         }
       }
     }
@@ -97,7 +97,7 @@ public class ConstraintBasedAssignment {
       return instanceStateMap;
     }
 
-    List<String> statesPriorityList = stateModelDef.getStatesPriorityStringList();
+    List<String> statesPriorityList = stateModelDef.getStatesPriorityList();
     boolean assigned[] = new boolean[instancePreferenceList.size()];
 
     Map<String, LiveInstance> liveInstancesMap = cache.getLiveInstances();
@@ -152,7 +152,7 @@ public class ConstraintBasedAssignment {
   public static LinkedHashMap<String, Integer> stateCount(StateModelDefinition stateModelDef,
       int liveNodesNb, int totalReplicas) {
     LinkedHashMap<String, Integer> stateCountMap = new LinkedHashMap<String, Integer>();
-    List<String> statesPriorityList = stateModelDef.getStatesPriorityStringList();
+    List<String> statesPriorityList = stateModelDef.getStatesPriorityList();
 
     int replicas = totalReplicas;
     for (String state : statesPriorityList) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 d5531b1..f703073 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
@@ -100,7 +100,7 @@ public class NewConstraintBasedAssignment {
   public static Map<State, String> stateConstraints(StateModelDefinition stateModelDef,
       ResourceId resourceId, ClusterConfig cluster) {
     Map<State, String> stateMap = Maps.newHashMap();
-    for (State state : stateModelDef.getStatesPriorityList()) {
+    for (State state : stateModelDef.getTypedStatesPriorityList()) {
       String num =
           cluster.getStateUpperBoundConstraint(Scope.resource(resourceId),
               stateModelDef.getStateModelDefId(), state);
@@ -138,7 +138,7 @@ public class NewConstraintBasedAssignment {
             participantId).equals(State.from(HelixDefinedState.ERROR)))
             && disabledParticipantsForPartition.contains(participantId)) {
           // if disabled and not in ERROR state, transit to initial-state (e.g. OFFLINE)
-          participantStateMap.put(participantId, stateModelDef.getInitialState());
+          participantStateMap.put(participantId, stateModelDef.getTypedInitialState());
         }
       }
     }
@@ -148,7 +148,7 @@ public class NewConstraintBasedAssignment {
       return participantStateMap;
     }
 
-    List<State> statesPriorityList = stateModelDef.getStatesPriorityList();
+    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
     boolean assigned[] = new boolean[participantPreferenceList.size()];
 
     for (State state : statesPriorityList) {
@@ -204,7 +204,7 @@ public class NewConstraintBasedAssignment {
   public static LinkedHashMap<State, Integer> stateCount(Map<State, String> upperBounds,
       StateModelDefinition stateModelDef, int liveNodesNb, int totalReplicas) {
     LinkedHashMap<State, Integer> stateCountMap = new LinkedHashMap<State, Integer>();
-    List<State> statesPriorityList = stateModelDef.getStatesPriorityList();
+    List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
 
     int replicas = totalReplicas;
     for (State state : statesPriorityList) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
index 2c4d8e1..ac1cef4 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ClusterDataCache.java
@@ -73,7 +73,7 @@ public class ClusterDataCache {
     _liveInstanceMap = accessor.getChildValuesMap(keyBuilder.liveInstances());
 
     for (LiveInstance instance : _liveInstanceMap.values()) {
-      LOG.trace("live instance: " + instance.getParticipantId() + " " + instance.getSessionId());
+      LOG.trace("live instance: " + instance.getParticipantId() + " " + instance.getTypedSessionId());
     }
 
     _stateModelDefMap = accessor.getChildValuesMap(keyBuilder.stateModelDefs());
@@ -91,7 +91,7 @@ public class ClusterDataCache {
         new HashMap<String, Map<String, Map<String, CurrentState>>>();
     for (String instanceName : _liveInstanceMap.keySet()) {
       LiveInstance liveInstance = _liveInstanceMap.get(instanceName);
-      String sessionId = liveInstance.getSessionId().stringify();
+      String sessionId = liveInstance.getTypedSessionId().stringify();
       if (!allCurStateMap.containsKey(instanceName)) {
         allCurStateMap.put(instanceName, new HashMap<String, Map<String, CurrentState>>());
       }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
index 50adf97..64e881c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/CompatibilityCheckStage.java
@@ -47,7 +47,7 @@ public class CompatibilityCheckStage extends AbstractBaseStage {
     HelixManagerProperties properties = manager.getProperties();
     Map<String, LiveInstance> liveInstanceMap = cache.getLiveInstances();
     for (LiveInstance liveInstance : liveInstanceMap.values()) {
-      HelixVersion version = liveInstance.getHelixVersion();
+      HelixVersion version = liveInstance.getTypedHelixVersion();
       String participantVersion = (version != null) ? version.toString() : null;
       if (!properties.isParticipantCompatible(participantVersion)) {
         String errorMsg =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 3f5682e..7036512 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
@@ -60,7 +60,7 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
         if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())) {
           continue;
         }
-        if (!instance.getSessionId().equals(message.getTgtSessionId())) {
+        if (!instance.getTypedSessionId().equals(message.getTypedTgtSessionId())) {
           continue;
         }
         ResourceId resourceId = message.getResourceId();
@@ -74,7 +74,7 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
           Partition partition = resource.getPartition(partitionId.stringify());
           if (partition != null) {
             currentStateOutput.setPendingState(resourceId.stringify(), partition, instanceName,
-                message.getToState().toString());
+                message.getTypedToState().toString());
           } else {
             // log
           }
@@ -85,7 +85,7 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
               Partition partition = resource.getPartition(partitionId.stringify());
               if (partition != null) {
                 currentStateOutput.setPendingState(resourceId.stringify(), partition, instanceName,
-                    message.getToState().toString());
+                    message.getTypedToState().toString());
               } else {
                 // log
               }
@@ -97,12 +97,12 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
     for (LiveInstance instance : liveInstances.values()) {
       String instanceName = instance.getInstanceName();
 
-      String clientSessionId = instance.getSessionId().stringify();
+      String clientSessionId = instance.getTypedSessionId().stringify();
       Map<String, CurrentState> currentStateMap =
           cache.getCurrentState(instanceName, clientSessionId);
       for (CurrentState currentState : currentStateMap.values()) {
 
-        if (!instance.getSessionId().equals(currentState.getSessionId())) {
+        if (!instance.getTypedSessionId().equals(currentState.getTypedSessionId())) {
           continue;
         }
         String resourceName = currentState.getResourceName();
@@ -117,7 +117,7 @@ public class CurrentStateComputationStage extends AbstractBaseStage {
 
         currentStateOutput.setBucketSize(resourceName, currentState.getBucketSize());
 
-        Map<String, String> partitionStateMap = currentState.getPartitionStateStringMap();
+        Map<String, String> partitionStateMap = currentState.getPartitionStateMap();
         for (String partitionName : partitionStateMap.keySet()) {
           Partition partition = resource.getPartition(partitionName);
           if (partition != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
index 9e16568..3c3a9d9 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ExternalViewComputeStage.java
@@ -168,7 +168,7 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
 
     Builder keyBuilder = accessor.keyBuilder();
 
-    for (String taskPartitionName : ev.getPartitionStringSet()) {
+    for (String taskPartitionName : ev.getPartitionSet()) {
       for (String taskState : ev.getStateMap(taskPartitionName).values()) {
         if (taskState.equalsIgnoreCase(HelixDefinedState.ERROR.toString())
             || taskState.equalsIgnoreCase("COMPLETED")) {
@@ -193,7 +193,7 @@ public class ExternalViewComputeStage extends AbstractBaseStage {
       }
     }
     // fill the controllerMsgIdCountMap
-    for (String taskId : taskQueueIdealState.getPartitionStringSet()) {
+    for (String taskId : taskQueueIdealState.getPartitionSet()) {
       String controllerMsgId =
           taskQueueIdealState.getRecord().getMapField(taskId)
               .get(DefaultSchedulerMessageHandlerFactory.CONTROLLER_MSG_ID);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 9c24bd6..3056cd5 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
@@ -71,7 +71,8 @@ public class MessageGenerationPhase extends AbstractBaseStage {
     Map<String, String> sessionIdMap = new HashMap<String, String>();
 
     for (LiveInstance liveInstance : liveInstances.values()) {
-      sessionIdMap.put(liveInstance.getInstanceName(), liveInstance.getSessionId().stringify());
+      sessionIdMap
+          .put(liveInstance.getInstanceName(), liveInstance.getTypedSessionId().stringify());
     }
     MessageGenerationOutput output = new MessageGenerationOutput();
 
@@ -96,7 +97,7 @@ public class MessageGenerationPhase extends AbstractBaseStage {
           String currentState =
               currentStateOutput.getCurrentState(resourceName, partition, instanceName);
           if (currentState == null) {
-            currentState = stateModelDef.getInitialStateString();
+            currentState = stateModelDef.getInitialState();
           }
 
           if (desiredState.equalsIgnoreCase(currentState)) {
@@ -177,7 +178,7 @@ public class MessageGenerationPhase extends AbstractBaseStage {
         }
 
         // add generated messages to output according to state priority
-        List<String> statesPriorityList = stateModelDef.getStatesPriorityStringList();
+        List<String> statesPriorityList = stateModelDef.getStatesPriorityList();
         for (String state : statesPriorityList) {
           if (messageMap.containsKey(state)) {
             for (Message message : messageMap.get(state)) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
index 9e430b7..1a3f37b 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
@@ -106,7 +106,7 @@ public class MessageSelectionStage extends AbstractBaseStage {
             selectMessages(cache.getLiveInstances(),
                 currentStateOutput.getCurrentStateMap(resourceName, partition),
                 currentStateOutput.getPendingStateMap(resourceName, partition), messages,
-                stateConstraints, stateTransitionPriorities, stateModelDef.getInitialStateString());
+                stateConstraints, stateTransitionPriorities, stateModelDef.getInitialState());
         output.addMessages(resourceName, partition, selectedMessages);
       }
     }
@@ -171,8 +171,8 @@ public class MessageSelectionStage extends AbstractBaseStage {
     Map<Integer, List<Message>> messagesGroupByStateTransitPriority =
         new TreeMap<Integer, List<Message>>();
     for (Message message : messages) {
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String transition = fromState + "-" + toState;
       int priority = Integer.MAX_VALUE;
 
@@ -189,8 +189,8 @@ public class MessageSelectionStage extends AbstractBaseStage {
     // select messages
     for (List<Message> messageList : messagesGroupByStateTransitPriority.values()) {
       for (Message message : messageList) {
-        State fromState = message.getFromState();
-        State toState = message.getToState();
+        State fromState = message.getTypedFromState();
+        State toState = message.getTypedToState();
 
         if (!bounds.containsKey(fromState)) {
           LOG.error("Message's fromState is not in currentState. message: " + message);
@@ -241,7 +241,7 @@ public class MessageSelectionStage extends AbstractBaseStage {
       IdealState idealState, ClusterDataCache cache) {
     Map<String, Bounds> stateConstraints = new HashMap<String, Bounds>();
 
-    List<String> statePriorityList = stateModelDefinition.getStatesPriorityStringList();
+    List<String> statePriorityList = stateModelDefinition.getStatesPriorityList();
     for (String state : statePriorityList) {
       String numInstancesPerState = stateModelDefinition.getNumInstancesPerState(state);
       int max = -1;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 f1c9323..f7f2a5f 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
@@ -68,7 +68,7 @@ public class NewCurrentStateComputationStage extends AbstractBaseStage {
           continue;
         }
 
-        if (!liveParticipant.getRunningInstance().getSessionId().equals(message.getTgtSessionId())) {
+        if (!liveParticipant.getRunningInstance().getSessionId().equals(message.getTypedTgtSessionId())) {
           continue;
         }
 
@@ -83,7 +83,7 @@ public class NewCurrentStateComputationStage extends AbstractBaseStage {
           Partition partition = resource.getSubUnit(partitionId);
           if (partition != null) {
             currentStateOutput.setPendingState(resourceId, partitionId, participantId,
-                message.getToState());
+                message.getTypedToState());
           } else {
             // log
           }
@@ -94,7 +94,7 @@ public class NewCurrentStateComputationStage extends AbstractBaseStage {
               Partition partition = resource.getSubUnit(partitionId);
               if (partition != null) {
                 currentStateOutput.setPendingState(resourceId, partitionId, participantId,
-                    message.getToState());
+                    message.getTypedToState());
               } else {
                 // log
               }
@@ -107,7 +107,7 @@ public class NewCurrentStateComputationStage extends AbstractBaseStage {
       SessionId sessionId = liveParticipant.getRunningInstance().getSessionId();
       Map<ResourceId, CurrentState> curStateMap = liveParticipant.getCurrentStateMap();
       for (CurrentState curState : curStateMap.values()) {
-        if (!sessionId.equals(curState.getSessionId())) {
+        if (!sessionId.equals(curState.getTypedSessionId())) {
           continue;
         }
 
@@ -124,7 +124,7 @@ public class NewCurrentStateComputationStage extends AbstractBaseStage {
 
         currentStateOutput.setBucketSize(resourceId, curState.getBucketSize());
 
-        Map<PartitionId, State> partitionStateMap = curState.getPartitionStateMap();
+        Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
         for (PartitionId partitionId : partitionStateMap.keySet()) {
           Partition partition = resource.getSubUnit(partitionId);
           if (partition != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 8ff52bd..d67931d 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
@@ -184,7 +184,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
     Map<String, Map<String, String>> controllerMsgUpdates =
         new HashMap<String, Map<String, String>>();
 
-    for (String taskPartitionName : ev.getPartitionStringSet()) {
+    for (String taskPartitionName : ev.getPartitionSet()) {
       for (String taskState : ev.getStateMap(taskPartitionName).values()) {
         if (taskState.equalsIgnoreCase(HelixDefinedState.ERROR.toString())
             || taskState.equalsIgnoreCase("COMPLETED")) {
@@ -235,7 +235,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
           result.put("Result", controllerMsgUpdates.get(controllerMsgId).get(taskPartitionName));
           controllerStatusUpdate.getRecord().setMapField(
               "MessageResult " + innerMessage.getTgtName() + " " + taskPartitionName + " "
-                  + innerMessage.getMsgId(), result);
+                  + innerMessage.getMessageId(), result);
         }
 
         // All done for the scheduled tasks that came from controllerMsgId, add summary for it

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 fb6dfe8..3d51bd0 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
@@ -96,7 +96,7 @@ public class NewMessageGenerationStage extends AbstractBaseStage {
           State currentState =
               currentStateOutput.getCurrentState(resourceId, subUnitId, participantId);
           if (currentState == null) {
-            currentState = stateModelDef.getInitialState();
+            currentState = stateModelDef.getTypedInitialState();
           }
 
           if (desiredState.equals(currentState)) {
@@ -174,7 +174,7 @@ public class NewMessageGenerationStage extends AbstractBaseStage {
         }
 
         // add generated messages to output according to state priority
-        List<State> statesPriorityList = stateModelDef.getStatesPriorityList();
+        List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
         for (State state : statesPriorityList) {
           if (messageMap.containsKey(state)) {
             for (Message message : messageMap.get(state)) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 4adfbcb..4a46a4c 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
@@ -126,7 +126,7 @@ public class NewMessageSelectionStage extends AbstractBaseStage {
             selectMessages(cluster.getLiveParticipantMap(),
                 currentStateOutput.getCurrentStateMap(resourceId, partitionId),
                 currentStateOutput.getPendingStateMap(resourceId, partitionId), messages,
-                stateConstraints, stateTransitionPriorities, stateModelDef.getInitialState());
+                stateConstraints, stateTransitionPriorities, stateModelDef.getTypedInitialState());
         output.setMessages(resourceId, partitionId, selectedMessages);
       }
     }
@@ -191,8 +191,8 @@ public class NewMessageSelectionStage extends AbstractBaseStage {
     Map<Integer, List<Message>> messagesGroupByStateTransitPriority =
         new TreeMap<Integer, List<Message>>();
     for (Message message : messages) {
-      State fromState = message.getFromState();
-      State toState = message.getToState();
+      State fromState = message.getTypedFromState();
+      State toState = message.getTypedToState();
       String transition = fromState.toString() + "-" + toState.toString();
       int priority = Integer.MAX_VALUE;
 
@@ -209,8 +209,8 @@ public class NewMessageSelectionStage extends AbstractBaseStage {
     // select messages
     for (List<Message> messageList : messagesGroupByStateTransitPriority.values()) {
       for (Message message : messageList) {
-        State fromState = message.getFromState();
-        State toState = message.getToState();
+        State fromState = message.getTypedFromState();
+        State toState = message.getTypedToState();
 
         if (!bounds.containsKey(fromState)) {
           LOG.error("Message's fromState is not in currentState. message: " + message);
@@ -268,7 +268,7 @@ public class NewMessageSelectionStage extends AbstractBaseStage {
             .getRebalancerContext(ReplicatedRebalancerContext.class) : null;
     Map<State, Bounds> stateConstraints = new HashMap<State, Bounds>();
 
-    List<State> statePriorityList = stateModelDefinition.getStatesPriorityList();
+    List<State> statePriorityList = stateModelDefinition.getTypedStatesPriorityList();
     for (State state : statePriorityList) {
       String numInstancesPerState =
           cluster.getStateUpperBoundConstraint(Scope.cluster(cluster.getId()),

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 e839a98..b531bd7 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
@@ -98,8 +98,8 @@ public class NewResourceComputationStage extends AbstractBaseStage {
 
         if (currentState.getStateModelDefRef() == null) {
           LOG.error("state model def is null." + "resource:" + currentState.getResourceId()
-              + ", partitions: " + currentState.getPartitionStateStringMap().keySet()
-              + ", states: " + currentState.getPartitionStateStringMap().values());
+              + ", partitions: " + currentState.getPartitionStateMap().keySet()
+              + ", states: " + currentState.getPartitionStateMap().values());
           throw new StageException("State model def is null for resource:"
               + currentState.getResourceId());
         }
@@ -119,7 +119,7 @@ public class NewResourceComputationStage extends AbstractBaseStage {
         }
 
         PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId);
-        for (PartitionId partitionId : currentState.getPartitionStateMap().keySet()) {
+        for (PartitionId partitionId : currentState.getTypedPartitionStateMap().keySet()) {
           rebCtxBuilder.addPartition(new Partition(partitionId));
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 d8fa81c..51c9284 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
@@ -109,9 +109,9 @@ public class NewTaskAssignmentStage extends AbstractBaseStage {
       }
 
       String key =
-          keyBuilder.currentState(message.getTgtName(), message.getTgtSessionId().stringify(),
+          keyBuilder.currentState(message.getTgtName(), message.getTypedTgtSessionId().stringify(),
               message.getResourceId().stringify()).getPath()
-              + "/" + message.getFromState() + "/" + message.getToState();
+              + "/" + message.getTypedFromState() + "/" + message.getTypedToState();
 
       if (!batchMessages.containsKey(key)) {
         Message batchMessage = new Message(message.getRecord());
@@ -134,9 +134,9 @@ public class NewTaskAssignmentStage extends AbstractBaseStage {
 
     List<PropertyKey> keys = new ArrayList<PropertyKey>();
     for (Message message : messages) {
-      logger.info("Sending Message " + message.getMsgId() + " to " + message.getTgtName()
+      logger.info("Sending Message " + message.getMessageId() + " to " + message.getTgtName()
           + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:"
-          + message.getFromState() + " to:" + message.getToState());
+          + message.getTypedFromState() + " to:" + message.getTypedToState());
 
       // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to "
       // + message.getTgtName() + " transit " + message.getPartitionId() + "|"

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java
index ffc14d6..949cfca 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/RebalanceIdealStateStage.java
@@ -62,7 +62,7 @@ public class RebalanceIdealStateStage extends AbstractBaseStage {
               (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance());
           balancer.init(manager);
           Resource resource = new Resource(resourceName);
-          for (String partitionName : currentIdealState.getPartitionStringSet()) {
+          for (String partitionName : currentIdealState.getPartitionSet()) {
             resource.addPartition(partitionName);
           }
           ResourceAssignment resourceAssignment =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
index cb2a7ed..da38ee2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
@@ -54,7 +54,7 @@ public class ResourceComputationStage extends AbstractBaseStage {
 
     if (idealStates != null && idealStates.size() > 0) {
       for (IdealState idealState : idealStates.values()) {
-        Set<String> partitionSet = idealState.getPartitionStringSet();
+        Set<String> partitionSet = idealState.getPartitionSet();
         String resourceName = idealState.getResourceName();
 
         for (String partition : partitionSet) {
@@ -75,7 +75,7 @@ public class ResourceComputationStage extends AbstractBaseStage {
     if (availableInstances != null && availableInstances.size() > 0) {
       for (LiveInstance instance : availableInstances.values()) {
         String instanceName = instance.getInstanceName();
-        String clientSessionId = instance.getSessionId().stringify();
+        String clientSessionId = instance.getTypedSessionId().stringify();
 
         Map<String, CurrentState> currentStateMap =
             cache.getCurrentState(instanceName, clientSessionId);
@@ -85,7 +85,7 @@ public class ResourceComputationStage extends AbstractBaseStage {
         for (CurrentState currentState : currentStateMap.values()) {
 
           String resourceName = currentState.getResourceName();
-          Map<String, String> resourceStateMap = currentState.getPartitionStateStringMap();
+          Map<String, String> resourceStateMap = currentState.getPartitionStateMap();
 
           // don't overwrite ideal state settings
           if (!resourceMap.containsKey(resourceName)) {
@@ -99,8 +99,8 @@ public class ResourceComputationStage extends AbstractBaseStage {
 
           if (currentState.getStateModelDefRef() == null) {
             LOG.error("state model def is null." + "resource:" + currentState.getResourceName()
-                + ", partitions: " + currentState.getPartitionStateStringMap().keySet()
-                + ", states: " + currentState.getPartitionStateStringMap().values());
+                + ", partitions: " + currentState.getPartitionStateMap().keySet()
+                + ", states: " + currentState.getPartitionStateMap().values());
             throw new StageException("State model def is null for resource:"
                 + currentState.getResourceName());
           }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 a50f76b..c942db9 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
@@ -98,7 +98,7 @@ public class TaskAssignmentStage extends AbstractBaseStage {
       LiveInstance liveInstance = liveInstanceMap.get(instanceName);
       String participantVersion = null;
       if (liveInstance != null) {
-        participantVersion = liveInstance.getHelixVersion().toString();
+        participantVersion = liveInstance.getTypedHelixVersion().toString();
       }
 
       if (resource == null || !resource.getBatchMessageMode() || participantVersion == null
@@ -108,9 +108,9 @@ public class TaskAssignmentStage extends AbstractBaseStage {
       }
 
       String key =
-          keyBuilder.currentState(message.getTgtName(), message.getTgtSessionId().stringify(),
+          keyBuilder.currentState(message.getTgtName(), message.getTypedTgtSessionId().stringify(),
               message.getResourceId().stringify()).getPath()
-              + "/" + message.getFromState() + "/" + message.getToState();
+              + "/" + message.getTypedFromState() + "/" + message.getTypedToState();
 
       if (!batchMessages.containsKey(key)) {
         Message batchMessage = new Message(message.getRecord());
@@ -133,9 +133,9 @@ public class TaskAssignmentStage extends AbstractBaseStage {
 
     List<PropertyKey> keys = new ArrayList<PropertyKey>();
     for (Message message : messages) {
-      logger.info("Sending Message " + message.getMsgId() + " to " + message.getTgtName()
+      logger.info("Sending Message " + message.getMessageId() + " to " + message.getTgtName()
           + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:"
-          + message.getFromState() + " to:" + message.getToState());
+          + message.getTypedFromState() + " to:" + message.getTypedToState());
 
       // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to " +
       // message.getTgtName()

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
index 55ace7a..dd8e9be 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
@@ -149,7 +149,7 @@ public class ControllerManager extends AbstractManager {
       LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader());
       if (leader != null) {
         String leaderName = leader.getInstanceName();
-        String sessionId = leader.getSessionId().stringify();
+        String sessionId = leader.getTypedSessionId().stringify();
         if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null
             && sessionId.equals(_sessionId)) {
           return true;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 bd98632..11222e1 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
@@ -60,7 +60,7 @@ class CurStateCarryOverUpdater implements DataUpdater<ZNRecord> {
       curState = new CurrentState(currentData);
     }
 
-    for (PartitionId partitionId : _lastCurState.getPartitionStateMap().keySet()) {
+    for (PartitionId partitionId : _lastCurState.getTypedPartitionStateMap().keySet()) {
       // carry-over only when current-state not exist
       if (curState.getState(partitionId) == null) {
         curState.setState(partitionId, State.from(_initState));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java
index 5f6d083..33d271b 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultControllerMessageHandlerFactory.java
@@ -36,7 +36,7 @@ public class DefaultControllerMessageHandlerFactory implements MessageHandlerFac
     String type = message.getMsgType();
 
     if (!type.equals(getMessageType())) {
-      throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:"
+      throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:"
           + message.getMsgType());
     }
 
@@ -63,18 +63,18 @@ public class DefaultControllerMessageHandlerFactory implements MessageHandlerFac
       String type = _message.getMsgType();
       HelixTaskResult result = new HelixTaskResult();
       if (!type.equals(MessageType.CONTROLLER_MSG.toString())) {
-        throw new HelixException("Unexpected msg type for message " + _message.getMsgId()
+        throw new HelixException("Unexpected msg type for message " + _message.getMessageId()
             + " type:" + _message.getMsgType());
       }
       result.getTaskResultMap().put("ControllerResult",
-          "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed");
+          "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed");
       result.setSuccess(true);
       return result;
     }
 
     @Override
     public void onError(Exception e, ErrorCode code, ErrorType type) {
-      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e);
+      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java
index f297252..5e3a7ea 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultParticipantErrorMessageHandlerFactory.java
@@ -99,7 +99,7 @@ public class DefaultParticipantErrorMessageHandlerFactory implements MessageHand
 
     @Override
     public void onError(Exception e, ErrorCode code, ErrorType type) {
-      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e);
+      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e);
     }
 
   }
@@ -109,7 +109,7 @@ public class DefaultParticipantErrorMessageHandlerFactory implements MessageHand
     String type = message.getMsgType();
 
     if (!type.equals(getMessageType())) {
-      throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:"
+      throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:"
           + message.getMsgType());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/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 38a067b..e494507 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
@@ -78,7 +78,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
 
     @Override
     public void onTimeOut() {
-      _logger.info("Scheduler msg timeout " + _originalMessage.getMsgId() + " timout with "
+      _logger.info("Scheduler msg timeout " + _originalMessage.getMessageId() + " timout with "
           + _timeout + " Ms");
 
       _statusUpdateUtil.logError(_originalMessage, SchedulerAsyncCallback.class, "Task timeout",
@@ -88,13 +88,13 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
 
     @Override
     public void onReplyMessage(Message message) {
-      _logger.info("Update for scheduler msg " + _originalMessage.getMsgId() + " Message "
+      _logger.info("Update for scheduler msg " + _originalMessage.getMessageId() + " Message "
           + message.getMsgSrc() + " id " + message.getCorrelationId() + " completed");
       String key = "MessageResult " + message.getMsgSrc() + " " + UUID.randomUUID();
       _resultSummaryMap.put(key, message.getResultMap());
 
       if (this.isDone()) {
-        _logger.info("Scheduler msg " + _originalMessage.getMsgId() + " completed");
+        _logger.info("Scheduler msg " + _originalMessage.getMessageId() + " completed");
         _statusUpdateUtil.logInfo(_originalMessage, SchedulerAsyncCallback.class,
             "Scheduler task completed", _manager.getHelixDataAccessor());
         addSummary(_resultSummaryMap, _originalMessage, _manager, false);
@@ -113,11 +113,11 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       ZNRecord statusUpdate =
           accessor.getProperty(
               keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), originalMessage
-                  .getMsgId().stringify())).getRecord();
+                  .getMessageId().stringify())).getRecord();
 
       statusUpdate.getMapFields().putAll(_resultSummaryMap);
       accessor.setProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
-          originalMessage.getMsgId().stringify()), new StatusUpdate(statusUpdate));
+          originalMessage.getMessageId().stringify()), new StatusUpdate(statusUpdate));
 
     }
   }
@@ -134,7 +134,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
     String type = message.getMsgType();
 
     if (!type.equals(getMessageType())) {
-      throw new HelixException("Unexpected msg type for message " + message.getMsgId() + " type:"
+      throw new HelixException("Unexpected msg type for message " + message.getMessageId() + " type:"
           + message.getMsgType());
     }
 
@@ -208,7 +208,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
             newAddedScheduledTasks.getRecord().setMapField(partitionId,
                 task.getRecord().getSimpleFields());
             _logger.info("Scheduling for controllerMsg " + controllerMsgId + " , sending task "
-                + partitionId + " " + task.getMsgId() + " to " + instanceName);
+                + partitionId + " " + task.getMessageId() + " to " + instanceName);
 
             if (_logger.isDebugEnabled()) {
               _logger.debug(task.getRecord().getSimpleFields());
@@ -225,16 +225,16 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       ZNRecord statusUpdate =
           accessor.getProperty(
               keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), _message
-                  .getMsgId().stringify())).getRecord();
+                  .getMessageId().stringify())).getRecord();
 
       statusUpdate.getMapFields().put("SentMessageCount", sendSummary);
       accessor.updateProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
-          _message.getMsgId().stringify()), new StatusUpdate(statusUpdate));
+          _message.getMessageId().stringify()), new StatusUpdate(statusUpdate));
     }
 
     private int findTopPartitionId(IdealState currentTaskQueue) {
       int topId = 0;
-      for (PartitionId partitionId : currentTaskQueue.getPartitionSet()) {
+      for (PartitionId partitionId : currentTaskQueue.getPartitionIdSet()) {
         try {
           String partitionName = partitionId.stringify();
           String partitionNumStr = partitionName.substring(partitionName.lastIndexOf('_') + 1);
@@ -254,7 +254,7 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       String type = _message.getMsgType();
       HelixTaskResult result = new HelixTaskResult();
       if (!type.equals(MessageType.SCHEDULER_MSG.toString())) {
-        throw new HelixException("Unexpected msg type for message " + _message.getMsgId()
+        throw new HelixException("Unexpected msg type for message " + _message.getMessageId()
             + " type:" + _message.getMsgType());
       }
       // Parse timeout value
@@ -301,11 +301,11 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       if (InstanceType.PARTICIPANT == recipientCriteria.getRecipientInstanceType()
           && hasSchedulerTaskQueue) {
         handleMessageUsingScheduledTaskQueue(recipientCriteria, messageTemplate,
-            _message.getMsgId());
+            _message.getMessageId());
         result.setSuccess(true);
-        result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMsgId().stringify());
+        result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMessageId().stringify());
         result.getTaskResultMap().put("ControllerResult",
-            "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed");
+            "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed");
         return result;
       }
 
@@ -331,23 +331,23 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
       ZNRecord statusUpdate =
           accessor.getProperty(
               keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), _message
-                  .getMsgId().stringify())).getRecord();
+                  .getMessageId().stringify())).getRecord();
 
       statusUpdate.getMapFields().put("SentMessageCount", sendSummary);
 
       accessor.setProperty(keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(),
-          _message.getMsgId().stringify()), new StatusUpdate(statusUpdate));
+          _message.getMessageId().stringify()), new StatusUpdate(statusUpdate));
 
       result.getTaskResultMap().put("ControllerResult",
-          "msg " + _message.getMsgId() + " from " + _message.getMsgSrc() + " processed");
-      result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMsgId().stringify());
+          "msg " + _message.getMessageId() + " from " + _message.getMsgSrc() + " processed");
+      result.getTaskResultMap().put(SCHEDULER_MSG_ID, _message.getMessageId().stringify());
       result.setSuccess(true);
       return result;
     }
 
     @Override
     public void onError(Exception e, ErrorCode code, ErrorType type) {
-      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMsgId(), e);
+      _logger.error("Message handling pipeline get an exception. MsgId:" + _message.getMessageId(), e);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
index 0a005f2..f169317 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedControllerManager.java
@@ -175,7 +175,7 @@ public class DistributedControllerManager extends AbstractManager {
       LiveInstance leader = _dataAccessor.getProperty(_keyBuilder.controllerLeader());
       if (leader != null) {
         String leaderName = leader.getInstanceName();
-        String sessionId = leader.getSessionId().stringify();
+        String sessionId = leader.getTypedSessionId().stringify();
         if (leaderName != null && leaderName.equals(_instanceName) && sessionId != null
             && sessionId.equals(_sessionId)) {
           return true;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
index 3cf9244..caf4dae 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DistributedLeaderElection.java
@@ -138,7 +138,7 @@ public class DistributedLeaderElection implements ControllerChangeListener {
 
     leader = accessor.getProperty(keyBuilder.controllerLeader());
     if (leader != null) {
-      String leaderSessionId = leader.getSessionId().stringify();
+      String leaderSessionId = leader.getTypedSessionId().stringify();
       LOG.info("Leader exists for cluster: " + manager.getClusterName() + ", currentLeader: "
           + leader.getInstanceName() + ", leaderSessionId: " + leaderSessionId);
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/6b57486b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
index 743fcc3..aa84c4d 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManagerHelper.java
@@ -149,14 +149,14 @@ public class ParticipantManagerHelper {
              * update sessionId field in live-instance if necessary
              */
             LiveInstance curLiveInstance = new LiveInstance(record);
-            if (!curLiveInstance.getSessionId().stringify().equals(_sessionId)) {
+            if (!curLiveInstance.getTypedSessionId().stringify().equals(_sessionId)) {
               /**
                * in last handle-new-session,
                * live-instance is created by new zkconnection with stale session-id inside
                * just update session-id field
                */
               LOG.info("overwriting session-id by ephemeralOwner: " + ephemeralOwner
-                  + ", old-sessionId: " + curLiveInstance.getSessionId() + ", new-sessionId: "
+                  + ", old-sessionId: " + curLiveInstance.getTypedSessionId() + ", new-sessionId: "
                   + _sessionId);
 
               curLiveInstance.setSessionId(_sessionId);
@@ -228,10 +228,9 @@ public class ParticipantManagerHelper {
         String curStatePath =
             _keyBuilder.currentState(_instanceName, _sessionId, lastCurState.getResourceName())
                 .getPath();
-        _dataAccessor.getBaseDataAccessor().update(
-            curStatePath,
-            new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialStateString(),
-                lastCurState), AccessOption.PERSISTENT);
+        _dataAccessor.getBaseDataAccessor().update(curStatePath,
+            new CurStateCarryOverUpdater(_sessionId, stateModel.getInitialState(), lastCurState),
+            AccessOption.PERSISTENT);
       }
     }
 


[5/9] [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 4a46a4c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageSelectionStage.java
+++ /dev/null
@@ -1,317 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Participant;
-import org.apache.helix.api.Resource;
-import org.apache.helix.api.Scope;
-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.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;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.controller.rebalancer.context.ReplicatedRebalancerContext;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-public class NewMessageSelectionStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(NewMessageSelectionStage.class);
-
-  public static class Bounds {
-    private int upper;
-    private int lower;
-
-    public Bounds(int lower, int upper) {
-      this.lower = lower;
-      this.upper = upper;
-    }
-
-    public void increaseUpperBound() {
-      upper++;
-    }
-
-    public void increaseLowerBound() {
-      lower++;
-    }
-
-    public void decreaseUpperBound() {
-      upper--;
-    }
-
-    public void decreaseLowerBound() {
-      lower--;
-    }
-
-    public int getLowerBound() {
-      return lower;
-    }
-
-    public int getUpperBound() {
-      return upper;
-    }
-
-    @Override
-    public String toString() {
-      return String.format("%d-%d", lower, upper);
-    }
-  }
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    Map<StateModelDefId, StateModelDefinition> stateModelDefMap = cluster.getStateModelMap();
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-    ResourceCurrentState currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    NewMessageOutput messageGenOutput = event.getAttribute(AttributeName.MESSAGES_ALL.toString());
-    if (cluster == null || resourceMap == null || currentStateOutput == null
-        || messageGenOutput == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires DataCache|RESOURCES|CURRENT_STATE|MESSAGES_ALL");
-    }
-
-    NewMessageOutput output = new NewMessageOutput();
-
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resource = resourceMap.get(resourceId);
-      StateModelDefinition stateModelDef =
-          stateModelDefMap.get(resource.getRebalancerConfig()
-              .getRebalancerContext(RebalancerContext.class).getStateModelDefId());
-
-      // TODO have a logical model for transition
-      Map<String, Integer> stateTransitionPriorities = getStateTransitionPriorityMap(stateModelDef);
-      Resource configResource = cluster.getResource(resourceId);
-
-      // if configResource == null, the resource has been dropped
-      Map<State, Bounds> stateConstraints =
-          computeStateConstraints(stateModelDef,
-              configResource == null ? null : configResource.getRebalancerConfig(), cluster);
-
-      // TODO fix it
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
-        List<Message> messages = messageGenOutput.getMessages(resourceId, partitionId);
-        List<Message> selectedMessages =
-            selectMessages(cluster.getLiveParticipantMap(),
-                currentStateOutput.getCurrentStateMap(resourceId, partitionId),
-                currentStateOutput.getPendingStateMap(resourceId, partitionId), messages,
-                stateConstraints, stateTransitionPriorities, stateModelDef.getTypedInitialState());
-        output.setMessages(resourceId, partitionId, selectedMessages);
-      }
-    }
-    event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), output);
-  }
-
-  // TODO: This method deserves its own class. The class should not understand helix but
-  // just be
-  // able to solve the problem using the algo. I think the method is following that but if
-  // we don't move it to another class its quite easy to break that contract
-  /**
-   * greedy message selection algorithm: 1) calculate CS+PS state lower/upper-bounds 2)
-   * group messages by state transition and sorted by priority 3) from highest priority to
-   * lowest, for each message group with the same transition add message one by one and
-   * make sure state constraint is not violated update state lower/upper-bounds when a new
-   * message is selected
-   * @param currentStates
-   * @param pendingStates
-   * @param messages
-   * @param stateConstraints
-   *          : STATE -> bound (lower:upper)
-   * @param stateTransitionPriorities
-   *          : FROME_STATE-TO_STATE -> priority
-   * @return: selected messages
-   */
-  List<Message> selectMessages(Map<ParticipantId, Participant> liveParticipants,
-      Map<ParticipantId, State> currentStates, Map<ParticipantId, State> pendingStates,
-      List<Message> messages, Map<State, Bounds> stateConstraints,
-      final Map<String, Integer> stateTransitionPriorities, State initialState) {
-    if (messages == null || messages.isEmpty()) {
-      return Collections.emptyList();
-    }
-
-    List<Message> selectedMessages = new ArrayList<Message>();
-    Map<State, Bounds> bounds = new HashMap<State, Bounds>();
-
-    // count currentState, if no currentState, count as in initialState
-    for (ParticipantId liveParticipantId : liveParticipants.keySet()) {
-      State state = initialState;
-      if (currentStates.containsKey(liveParticipantId)) {
-        state = currentStates.get(liveParticipantId);
-      }
-
-      if (!bounds.containsKey(state)) {
-        bounds.put(state, new Bounds(0, 0));
-      }
-      bounds.get(state).increaseLowerBound();
-      bounds.get(state).increaseUpperBound();
-    }
-
-    // count pendingStates
-    for (ParticipantId participantId : pendingStates.keySet()) {
-      State state = pendingStates.get(participantId);
-      if (!bounds.containsKey(state)) {
-        bounds.put(state, new Bounds(0, 0));
-      }
-      // TODO: add lower bound, need to refactor pendingState to include fromState also
-      bounds.get(state).increaseUpperBound();
-    }
-
-    // group messages based on state transition priority
-    Map<Integer, List<Message>> messagesGroupByStateTransitPriority =
-        new TreeMap<Integer, List<Message>>();
-    for (Message message : messages) {
-      State fromState = message.getTypedFromState();
-      State toState = message.getTypedToState();
-      String transition = fromState.toString() + "-" + toState.toString();
-      int priority = Integer.MAX_VALUE;
-
-      if (stateTransitionPriorities.containsKey(transition)) {
-        priority = stateTransitionPriorities.get(transition);
-      }
-
-      if (!messagesGroupByStateTransitPriority.containsKey(priority)) {
-        messagesGroupByStateTransitPriority.put(priority, new ArrayList<Message>());
-      }
-      messagesGroupByStateTransitPriority.get(priority).add(message);
-    }
-
-    // select messages
-    for (List<Message> messageList : messagesGroupByStateTransitPriority.values()) {
-      for (Message message : messageList) {
-        State fromState = message.getTypedFromState();
-        State toState = message.getTypedToState();
-
-        if (!bounds.containsKey(fromState)) {
-          LOG.error("Message's fromState is not in currentState. message: " + message);
-          continue;
-        }
-
-        if (!bounds.containsKey(toState)) {
-          bounds.put(toState, new Bounds(0, 0));
-        }
-
-        // check lower bound of fromState
-        if (stateConstraints.containsKey(fromState)) {
-          int newLowerBound = bounds.get(fromState).getLowerBound() - 1;
-          if (newLowerBound < 0) {
-            LOG.error("Number of currentState in " + fromState
-                + " is less than number of messages transiting from " + fromState);
-            continue;
-          }
-
-          if (newLowerBound < stateConstraints.get(fromState).getLowerBound()) {
-            continue;
-          }
-        }
-
-        // check upper bound of toState
-        if (stateConstraints.containsKey(toState)) {
-          int newUpperBound = bounds.get(toState).getUpperBound() + 1;
-          if (newUpperBound > stateConstraints.get(toState).getUpperBound()) {
-            continue;
-          }
-        }
-
-        selectedMessages.add(message);
-        bounds.get(fromState).increaseLowerBound();
-        bounds.get(toState).increaseUpperBound();
-      }
-    }
-
-    return selectedMessages;
-  }
-
-  /**
-   * TODO: This code is duplicate in multiple places. Can we do it in to one place in the
-   * beginning and compute the stateConstraint instance once and re use at other places.
-   * Each IdealState must have a constraint object associated with it
-   * @param stateModelDefinition
-   * @param rebalancerConfig if rebalancerConfig == null, we can't evaluate R thus no constraints
-   * @param cluster
-   * @return
-   */
-  private Map<State, Bounds> computeStateConstraints(StateModelDefinition stateModelDefinition,
-      RebalancerConfig rebalancerConfig, Cluster cluster) {
-    ReplicatedRebalancerContext context =
-        (rebalancerConfig != null) ? rebalancerConfig
-            .getRebalancerContext(ReplicatedRebalancerContext.class) : null;
-    Map<State, Bounds> stateConstraints = new HashMap<State, Bounds>();
-
-    List<State> statePriorityList = stateModelDefinition.getTypedStatesPriorityList();
-    for (State state : statePriorityList) {
-      String numInstancesPerState =
-          cluster.getStateUpperBoundConstraint(Scope.cluster(cluster.getId()),
-              stateModelDefinition.getStateModelDefId(), state);
-      int max = -1;
-      if ("N".equals(numInstancesPerState)) {
-        max = cluster.getLiveParticipantMap().size();
-      } else if ("R".equals(numInstancesPerState)) {
-        // idealState is null when resource has been dropped,
-        // R can't be evaluated and ignore state constraints
-        if (context != null) {
-          if (context.anyLiveParticipant()) {
-            max = cluster.getLiveParticipantMap().size();
-          } else {
-            max = context.getReplicaCount();
-          }
-        }
-      } else {
-        try {
-          max = Integer.parseInt(numInstancesPerState);
-        } catch (Exception e) {
-          // use -1
-        }
-      }
-
-      if (max > -1) {
-        // if state has no constraint, will not put in map
-        stateConstraints.put(state, new Bounds(0, max));
-      }
-    }
-
-    return stateConstraints;
-  }
-
-  // TODO: if state transition priority is not provided then use lexicographical sorting
-  // so that behavior is consistent
-  private Map<String, Integer> getStateTransitionPriorityMap(StateModelDefinition stateModelDef) {
-    Map<String, Integer> stateTransitionPriorities = new HashMap<String, Integer>();
-    List<String> stateTransitionPriorityList = stateModelDef.getStateTransitionPriorityStringList();
-    for (int i = 0; i < stateTransitionPriorityList.size(); i++) {
-      stateTransitionPriorities.put(stateTransitionPriorityList.get(i), i);
-    }
-
-    return stateTransitionPriorities;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index dfea7fc..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageThrottleStage.java
+++ /dev/null
@@ -1,198 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Participant;
-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;
-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;
-import org.apache.log4j.Logger;
-
-public class NewMessageThrottleStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(NewMessageThrottleStage.class.getName());
-
-  int valueOf(String valueStr) {
-    int value = Integer.MAX_VALUE;
-
-    try {
-      ConstraintValue valueToken = ConstraintValue.valueOf(valueStr);
-      switch (valueToken) {
-      case ANY:
-        value = Integer.MAX_VALUE;
-        break;
-      default:
-        LOG.error("Invalid constraintValue token:" + valueStr + ". Use default value:"
-            + Integer.MAX_VALUE);
-        break;
-      }
-    } catch (Exception e) {
-      try {
-        value = Integer.parseInt(valueStr);
-      } catch (NumberFormatException ne) {
-        LOG.error("Invalid constraintValue string:" + valueStr + ". Use default value:"
-            + Integer.MAX_VALUE);
-      }
-    }
-    return value;
-  }
-
-  /**
-   * constraints are selected in the order of the following rules: 1) don't select
-   * constraints with CONSTRAINT_VALUE=ANY; 2) if one constraint is more specific than the
-   * other, select the most specific one 3) if a message matches multiple constraints of
-   * incomparable specificity, select the one with the minimum value 4) if a message
-   * matches multiple constraints of incomparable specificity, and they all have the same
-   * value, select the first in alphabetic order
-   */
-  Set<ConstraintItem> selectConstraints(Set<ConstraintItem> items,
-      Map<ConstraintAttribute, String> attributes) {
-    Map<String, ConstraintItem> selectedItems = new HashMap<String, ConstraintItem>();
-    for (ConstraintItem item : items) {
-      // don't select constraints with CONSTRAINT_VALUE=ANY
-      if (item.getConstraintValue().equals(ConstraintValue.ANY.toString())) {
-        continue;
-      }
-
-      String key = item.filter(attributes).toString();
-      if (!selectedItems.containsKey(key)) {
-        selectedItems.put(key, item);
-      } else {
-        ConstraintItem existingItem = selectedItems.get(key);
-        if (existingItem.match(item.getAttributes())) {
-          // item is more specific than existingItem
-          selectedItems.put(key, item);
-        } else if (!item.match(existingItem.getAttributes())) {
-          // existingItem and item are of incomparable specificity
-          int value = valueOf(item.getConstraintValue());
-          int existingValue = valueOf(existingItem.getConstraintValue());
-          if (value < existingValue) {
-            // item's constraint value is less than that of existingItem
-            selectedItems.put(key, item);
-          } else if (value == existingValue) {
-            if (item.toString().compareTo(existingItem.toString()) < 0) {
-              // item is ahead of existingItem in alphabetic order
-              selectedItems.put(key, item);
-            }
-          }
-        }
-      }
-    }
-    return new HashSet<ConstraintItem>(selectedItems.values());
-  }
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    NewMessageOutput msgSelectionOutput =
-        event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-
-    if (cluster == null || resourceMap == null || msgSelectionOutput == null) {
-      throw new StageException("Missing attributes in event: " + event
-          + ". Requires ClusterDataCache|RESOURCES|MESSAGES_SELECTED");
-    }
-
-    NewMessageOutput output = new NewMessageOutput();
-
-    // TODO fix it
-    ClusterConstraints constraint = cluster.getConstraint(ConstraintType.MESSAGE_CONSTRAINT);
-    Map<String, Integer> throttleCounterMap = new HashMap<String, Integer>();
-
-    if (constraint != null) {
-      // go through all pending messages, they should be counted but not throttled
-      for (ParticipantId participantId : cluster.getLiveParticipantMap().keySet()) {
-        Participant liveParticipant = cluster.getLiveParticipantMap().get(participantId);
-        throttle(throttleCounterMap, constraint, new ArrayList<Message>(liveParticipant
-            .getMessageMap().values()), false);
-      }
-    }
-
-    // go through all new messages, throttle if necessary
-    // assume messages should be sorted by state transition priority in messageSelection stage
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resource = resourceMap.get(resourceId);
-      // TODO fix it
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
-        List<Message> messages = msgSelectionOutput.getMessages(resourceId, partitionId);
-        if (constraint != null && messages != null && messages.size() > 0) {
-          messages = throttle(throttleCounterMap, constraint, messages, true);
-        }
-        output.setMessages(resourceId, partitionId, messages);
-      }
-    }
-
-    event.addAttribute(AttributeName.MESSAGES_THROTTLE.toString(), output);
-  }
-
-  private List<Message> throttle(Map<String, Integer> throttleMap, ClusterConstraints constraint,
-      List<Message> messages, final boolean needThrottle) {
-
-    List<Message> throttleOutputMsgs = new ArrayList<Message>();
-    for (Message message : messages) {
-      Map<ConstraintAttribute, String> msgAttr = ClusterConstraints.toConstraintAttributes(message);
-
-      Set<ConstraintItem> matches = constraint.match(msgAttr);
-      matches = selectConstraints(matches, msgAttr);
-
-      boolean msgThrottled = false;
-      for (ConstraintItem item : matches) {
-        String key = item.filter(msgAttr).toString();
-        if (!throttleMap.containsKey(key)) {
-          throttleMap.put(key, valueOf(item.getConstraintValue()));
-        }
-        int value = throttleMap.get(key);
-        throttleMap.put(key, --value);
-
-        if (needThrottle && value < 0) {
-          msgThrottled = true;
-
-          if (LOG.isDebugEnabled()) {
-            // TODO: printout constraint item that throttles the message
-            LOG.debug("message: " + message + " is throttled by constraint: " + item);
-          }
-        }
-      }
-
-      if (!msgThrottled) {
-        throttleOutputMsgs.add(message);
-      }
-    }
-
-    return throttleOutputMsgs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 26050f8..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
+++ /dev/null
@@ -1,73 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-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;
-import org.apache.log4j.Logger;
-
-public class NewReadClusterDataStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(NewReadClusterDataStage.class.getName());
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-    LOG.info("START ReadClusterDataStage.process()");
-
-    HelixManager manager = event.getAttribute("helixmanager");
-    if (manager == null) {
-      throw new StageException("HelixManager attribute value is null");
-    }
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    ClusterId clusterId = ClusterId.from(manager.getClusterName());
-    ClusterAccessor clusterAccessor = new ClusterAccessor(clusterId, accessor);
-
-    Cluster cluster = clusterAccessor.readCluster();
-
-    ClusterStatusMonitor clusterStatusMonitor =
-        (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
-    if (clusterStatusMonitor != null) {
-      // TODO fix it
-      // int disabledInstances = 0;
-      // int disabledPartitions = 0;
-      // for (InstanceConfig config : _cache._instanceConfigMap.values()) {
-      // if (config.getInstanceEnabled() == false) {
-      // disabledInstances++;
-      // }
-      // if (config.getDisabledPartitions() != null) {
-      // disabledPartitions += config.getDisabledPartitions().size();
-      // }
-      // }
-      // clusterStatusMonitor.setClusterStatusCounters(_cache._liveInstanceMap.size(),
-      // _cache._instanceConfigMap.size(), disabledInstances, disabledPartitions);
-    }
-
-    event.addAttribute("ClusterDataCache", cluster);
-
-    long endTime = System.currentTimeMillis();
-    LOG.info("END ReadClusterDataStage.process(). took: " + (endTime - startTime) + " ms");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index b531bd7..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
+++ /dev/null
@@ -1,138 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.HashMap;
-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.Resource;
-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;
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.model.CurrentState;
-import org.apache.log4j.Logger;
-
-/**
- * This stage computes all the resources in a cluster. The resources are
- * computed from IdealStates -> this gives all the resources currently active
- * CurrentState for liveInstance-> Helps in finding resources that are inactive
- * and needs to be dropped
- */
-public class NewResourceComputationStage extends AbstractBaseStage {
-  private static Logger LOG = Logger.getLogger(NewResourceComputationStage.class);
-
-  @Override
-  public void process(ClusterEvent event) throws StageException {
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    if (cluster == null) {
-      throw new StageException("Missing attributes in event: " + event + ". Requires Cluster");
-    }
-
-    Map<ResourceId, ResourceConfig> resCfgMap = new HashMap<ResourceId, ResourceConfig>();
-    Map<ResourceId, ResourceConfig> csResCfgMap = getCurStateResourceCfgMap(cluster);
-
-    // ideal-state may be removed, add all resource config in current-state but not in ideal-state
-    for (ResourceId resourceId : csResCfgMap.keySet()) {
-      if (!cluster.getResourceMap().keySet().contains(resourceId)) {
-        resCfgMap.put(resourceId, csResCfgMap.get(resourceId));
-      }
-    }
-
-    for (ResourceId resourceId : cluster.getResourceMap().keySet()) {
-      Resource resource = cluster.getResource(resourceId);
-      RebalancerConfig rebalancerCfg = resource.getRebalancerConfig();
-
-      ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId);
-      resCfgBuilder.bucketSize(resource.getBucketSize());
-      resCfgBuilder.batchMessageMode(resource.getBatchMessageMode());
-      resCfgBuilder.schedulerTaskConfig(resource.getSchedulerTaskConfig());
-      resCfgBuilder.rebalancerContext(rebalancerCfg.getRebalancerContext(RebalancerContext.class));
-      resCfgMap.put(resourceId, resCfgBuilder.build());
-    }
-
-    event.addAttribute(AttributeName.RESOURCES.toString(), resCfgMap);
-  }
-
-  /**
-   * Get resource config's from current-state
-   * @param cluster
-   * @return resource config map or empty map if not available
-   * @throws StageException
-   */
-  Map<ResourceId, ResourceConfig> getCurStateResourceCfgMap(Cluster cluster) throws StageException {
-    Map<ResourceId, ResourceConfig.Builder> resCfgBuilderMap =
-        new HashMap<ResourceId, ResourceConfig.Builder>();
-
-    Map<ResourceId, PartitionedRebalancerContext.Builder> rebCtxBuilderMap =
-        new HashMap<ResourceId, PartitionedRebalancerContext.Builder>();
-
-    for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) {
-      for (ResourceId resourceId : liveParticipant.getCurrentStateMap().keySet()) {
-        CurrentState currentState = liveParticipant.getCurrentStateMap().get(resourceId);
-
-        if (currentState.getStateModelDefRef() == null) {
-          LOG.error("state model def is null." + "resource:" + currentState.getResourceId()
-              + ", partitions: " + currentState.getPartitionStateMap().keySet()
-              + ", states: " + currentState.getPartitionStateMap().values());
-          throw new StageException("State model def is null for resource:"
-              + currentState.getResourceId());
-        }
-
-        if (!resCfgBuilderMap.containsKey(resourceId)) {
-          PartitionedRebalancerContext.Builder rebCtxBuilder =
-              new PartitionedRebalancerContext.Builder(resourceId);
-          rebCtxBuilder.stateModelDefId(currentState.getStateModelDefId());
-          rebCtxBuilder.stateModelFactoryId(StateModelFactoryId.from(currentState
-              .getStateModelFactoryName()));
-          rebCtxBuilderMap.put(resourceId, rebCtxBuilder);
-
-          ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId);
-          resCfgBuilder.bucketSize(currentState.getBucketSize());
-          resCfgBuilder.batchMessageMode(currentState.getBatchMessageMode());
-          resCfgBuilderMap.put(resourceId, resCfgBuilder);
-        }
-
-        PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId);
-        for (PartitionId partitionId : currentState.getTypedPartitionStateMap().keySet()) {
-          rebCtxBuilder.addPartition(new Partition(partitionId));
-        }
-      }
-    }
-
-    Map<ResourceId, ResourceConfig> resCfgMap = new HashMap<ResourceId, ResourceConfig>();
-    for (ResourceId resourceId : resCfgBuilderMap.keySet()) {
-      ResourceConfig.Builder resCfgBuilder = resCfgBuilderMap.get(resourceId);
-      PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId);
-      resCfgBuilder.rebalancerContext(rebCtxBuilder.build());
-      resCfgMap.put(resourceId, resCfgBuilder.build());
-    }
-
-    return resCfgMap;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 51c9284..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
+++ /dev/null
@@ -1,151 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.HelixDataAccessor;
-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.Cluster;
-import org.apache.helix.api.Participant;
-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;
-import org.apache.log4j.Logger;
-
-public class NewTaskAssignmentStage extends AbstractBaseStage {
-  private static Logger logger = Logger.getLogger(NewTaskAssignmentStage.class);
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-    logger.info("START TaskAssignmentStage.process()");
-
-    HelixManager manager = event.getAttribute("helixmanager");
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-    NewMessageOutput messageOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    Map<ParticipantId, Participant> liveParticipantMap = cluster.getLiveParticipantMap();
-
-    if (manager == null || resourceMap == null || messageOutput == null || cluster == null
-        || liveParticipantMap == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|DataCache|liveInstanceMap");
-    }
-
-    HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
-    List<Message> messagesToSend = new ArrayList<Message>();
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resource = resourceMap.get(resourceId);
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
-        List<Message> messages = messageOutput.getMessages(resourceId, partitionId);
-        messagesToSend.addAll(messages);
-      }
-    }
-
-    List<Message> outputMessages =
-        batchMessage(dataAccessor.keyBuilder(), messagesToSend, resourceMap, liveParticipantMap,
-            manager.getProperties());
-    sendMessages(dataAccessor, outputMessages);
-
-    long endTime = System.currentTimeMillis();
-    logger.info("END TaskAssignmentStage.process(). took: " + (endTime - startTime) + " ms");
-
-  }
-
-  List<Message> batchMessage(Builder keyBuilder, List<Message> messages,
-      Map<ResourceId, ResourceConfig> resourceMap,
-      Map<ParticipantId, Participant> liveParticipantMap, HelixManagerProperties properties) {
-    // group messages by its CurrentState path + "/" + fromState + "/" + toState
-    Map<String, Message> batchMessages = new HashMap<String, Message>();
-    List<Message> outputMessages = new ArrayList<Message>();
-
-    Iterator<Message> iter = messages.iterator();
-    while (iter.hasNext()) {
-      Message message = iter.next();
-      ResourceId resourceId = message.getResourceId();
-      ResourceConfig resource = resourceMap.get(resourceId);
-
-      ParticipantId participantId = ParticipantId.from(message.getTgtName());
-      Participant liveParticipant = liveParticipantMap.get(participantId);
-      String participantVersion = null;
-      if (liveParticipant != null) {
-        participantVersion = liveParticipant.getRunningInstance().getVersion().toString();
-      }
-
-      if (resource == null || !resource.getBatchMessageMode() || participantVersion == null
-          || !properties.isFeatureSupported("batch_message", participantVersion)) {
-        outputMessages.add(message);
-        continue;
-      }
-
-      String key =
-          keyBuilder.currentState(message.getTgtName(), message.getTypedTgtSessionId().stringify(),
-              message.getResourceId().stringify()).getPath()
-              + "/" + message.getTypedFromState() + "/" + message.getTypedToState();
-
-      if (!batchMessages.containsKey(key)) {
-        Message batchMessage = new Message(message.getRecord());
-        batchMessage.setBatchMessageMode(true);
-        outputMessages.add(batchMessage);
-        batchMessages.put(key, batchMessage);
-      }
-      batchMessages.get(key).addPartitionName(message.getPartitionId().stringify());
-    }
-
-    return outputMessages;
-  }
-
-  protected void sendMessages(HelixDataAccessor dataAccessor, List<Message> messages) {
-    if (messages == null || messages.isEmpty()) {
-      return;
-    }
-
-    Builder keyBuilder = dataAccessor.keyBuilder();
-
-    List<PropertyKey> keys = new ArrayList<PropertyKey>();
-    for (Message message : messages) {
-      logger.info("Sending Message " + message.getMessageId() + " to " + message.getTgtName()
-          + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:"
-          + message.getTypedFromState() + " to:" + message.getTypedToState());
-
-      // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to "
-      // + message.getTgtName() + " transit " + message.getPartitionId() + "|"
-      // + message.getPartitionIds() + " from: " + message.getFromState() + " to: "
-      // + message.getToState());
-
-      keys.add(keyBuilder.message(message.getTgtName(), message.getId()));
-    }
-
-    dataAccessor.createChildren(keys, new ArrayList<Message>(messages));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 be0b7f0..31dbb08 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
@@ -35,7 +35,7 @@ public class PersistAssignmentStage extends AbstractBaseStage {
     HelixManager helixManager = event.getAttribute("helixmanager");
     HelixDataAccessor accessor = helixManager.getHelixDataAccessor();
     ResourceAccessor resourceAccessor = new ResourceAccessor(accessor);
-    NewBestPossibleStateOutput assignments =
+    BestPossibleStateOutput assignments =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     for (ResourceId resourceId : assignments.getAssignedResources()) {
       ResourceAssignment assignment = assignments.getResourceAssignment(resourceId);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
index ce81f1f..44fddb6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadClusterDataStage.java
@@ -21,53 +21,53 @@ package org.apache.helix.controller.stages;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.accessor.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.model.InstanceConfig;
 import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
 import org.apache.log4j.Logger;
 
-@Deprecated
 public class ReadClusterDataStage extends AbstractBaseStage {
-  private static final Logger logger = Logger.getLogger(ReadClusterDataStage.class.getName());
-  ClusterDataCache _cache;
-
-  public ReadClusterDataStage() {
-    _cache = new ClusterDataCache();
-  }
+  private static final Logger LOG = Logger.getLogger(ReadClusterDataStage.class.getName());
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     long startTime = System.currentTimeMillis();
-    logger.info("START ReadClusterDataStage.process()");
+    LOG.info("START ReadClusterDataStage.process()");
 
     HelixManager manager = event.getAttribute("helixmanager");
     if (manager == null) {
       throw new StageException("HelixManager attribute value is null");
     }
-    HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
-    _cache.refresh(dataAccessor);
+    HelixDataAccessor accessor = manager.getHelixDataAccessor();
+    ClusterId clusterId = ClusterId.from(manager.getClusterName());
+    ClusterAccessor clusterAccessor = new ClusterAccessor(clusterId, accessor);
+
+    Cluster cluster = clusterAccessor.readCluster();
 
     ClusterStatusMonitor clusterStatusMonitor =
         (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
     if (clusterStatusMonitor != null) {
-      int disabledInstances = 0;
-      int disabledPartitions = 0;
-      for (InstanceConfig config : _cache._instanceConfigMap.values()) {
-        if (config.getInstanceEnabled() == false) {
-          disabledInstances++;
-        }
-        if (config.getDisabledPartitions() != null) {
-          disabledPartitions += config.getDisabledPartitions().size();
-        }
-      }
-      clusterStatusMonitor.setClusterStatusCounters(_cache._liveInstanceMap.size(),
-          _cache._instanceConfigMap.size(), disabledInstances, disabledPartitions);
+      // TODO fix it
+      // int disabledInstances = 0;
+      // int disabledPartitions = 0;
+      // for (InstanceConfig config : _cache._instanceConfigMap.values()) {
+      // if (config.getInstanceEnabled() == false) {
+      // disabledInstances++;
+      // }
+      // if (config.getDisabledPartitions() != null) {
+      // disabledPartitions += config.getDisabledPartitions().size();
+      // }
+      // }
+      // clusterStatusMonitor.setClusterStatusCounters(_cache._liveInstanceMap.size(),
+      // _cache._instanceConfigMap.size(), disabledInstances, disabledPartitions);
     }
 
-    event.addAttribute("ClusterDataCache", _cache);
+    event.addAttribute("ClusterDataCache", cluster);
 
     long endTime = System.currentTimeMillis();
-    logger.info("END ReadClusterDataStage.process(). took: " + (endTime - startTime) + " ms");
+    LOG.info("END ReadClusterDataStage.process(). took: " + (endTime - startTime) + " ms");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
index ae873c7..859c1d0 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ReadHealthDataStage.java
@@ -23,10 +23,8 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.log4j.Logger;
 
 public class ReadHealthDataStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(ReadHealthDataStage.class.getName());
   HealthDataCache _cache;
 
   public ReadHealthDataStage() {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
index da38ee2..dc56b89 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceComputationStage.java
@@ -19,16 +19,23 @@ package org.apache.helix.controller.stages;
  * under the License.
  */
 
-import java.util.LinkedHashMap;
+import java.util.HashMap;
 import java.util.Map;
-import java.util.Set;
 
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.Participant;
+import org.apache.helix.api.Partition;
+import org.apache.helix.api.Resource;
+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.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.LiveInstance;
-import org.apache.helix.model.Resource;
 import org.apache.log4j.Logger;
 
 /**
@@ -37,102 +44,95 @@ import org.apache.log4j.Logger;
  * CurrentState for liveInstance-> Helps in finding resources that are inactive
  * and needs to be dropped
  */
-@Deprecated
 public class ResourceComputationStage extends AbstractBaseStage {
   private static Logger LOG = Logger.getLogger(ResourceComputationStage.class);
 
   @Override
-  public void process(ClusterEvent event) throws Exception {
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    if (cache == null) {
-      throw new StageException("Missing attributes in event:" + event + ". Requires DataCache");
+  public void process(ClusterEvent event) throws StageException {
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    if (cluster == null) {
+      throw new StageException("Missing attributes in event: " + event + ". Requires Cluster");
     }
 
-    Map<String, IdealState> idealStates = cache.getIdealStates();
+    Map<ResourceId, ResourceConfig> resCfgMap = new HashMap<ResourceId, ResourceConfig>();
+    Map<ResourceId, ResourceConfig> csResCfgMap = getCurStateResourceCfgMap(cluster);
 
-    Map<String, Resource> resourceMap = new LinkedHashMap<String, Resource>();
+    // ideal-state may be removed, add all resource config in current-state but not in ideal-state
+    for (ResourceId resourceId : csResCfgMap.keySet()) {
+      if (!cluster.getResourceMap().keySet().contains(resourceId)) {
+        resCfgMap.put(resourceId, csResCfgMap.get(resourceId));
+      }
+    }
 
-    if (idealStates != null && idealStates.size() > 0) {
-      for (IdealState idealState : idealStates.values()) {
-        Set<String> partitionSet = idealState.getPartitionSet();
-        String resourceName = idealState.getResourceName();
+    for (ResourceId resourceId : cluster.getResourceMap().keySet()) {
+      Resource resource = cluster.getResource(resourceId);
+      RebalancerConfig rebalancerCfg = resource.getRebalancerConfig();
 
-        for (String partition : partitionSet) {
-          addPartition(partition, resourceName, resourceMap);
-          Resource resource = resourceMap.get(resourceName);
-          resource.setStateModelDefRef(idealState.getStateModelDefRef());
-          resource.setStateModelFactoryName(idealState.getStateModelFactoryName());
-          resource.setBucketSize(idealState.getBucketSize());
-          resource.setBatchMessageMode(idealState.getBatchMessageMode());
-        }
-      }
+      ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId);
+      resCfgBuilder.bucketSize(resource.getBucketSize());
+      resCfgBuilder.batchMessageMode(resource.getBatchMessageMode());
+      resCfgBuilder.schedulerTaskConfig(resource.getSchedulerTaskConfig());
+      resCfgBuilder.rebalancerContext(rebalancerCfg.getRebalancerContext(RebalancerContext.class));
+      resCfgMap.put(resourceId, resCfgBuilder.build());
     }
 
-    // It's important to get partitions from CurrentState as well since the
-    // idealState might be removed.
-    Map<String, LiveInstance> availableInstances = cache.getLiveInstances();
+    event.addAttribute(AttributeName.RESOURCES.toString(), resCfgMap);
+  }
 
-    if (availableInstances != null && availableInstances.size() > 0) {
-      for (LiveInstance instance : availableInstances.values()) {
-        String instanceName = instance.getInstanceName();
-        String clientSessionId = instance.getTypedSessionId().stringify();
+  /**
+   * Get resource config's from current-state
+   * @param cluster
+   * @return resource config map or empty map if not available
+   * @throws StageException
+   */
+  Map<ResourceId, ResourceConfig> getCurStateResourceCfgMap(Cluster cluster) throws StageException {
+    Map<ResourceId, ResourceConfig.Builder> resCfgBuilderMap =
+        new HashMap<ResourceId, ResourceConfig.Builder>();
+
+    Map<ResourceId, PartitionedRebalancerContext.Builder> rebCtxBuilderMap =
+        new HashMap<ResourceId, PartitionedRebalancerContext.Builder>();
+
+    for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) {
+      for (ResourceId resourceId : liveParticipant.getCurrentStateMap().keySet()) {
+        CurrentState currentState = liveParticipant.getCurrentStateMap().get(resourceId);
+
+        if (currentState.getStateModelDefRef() == null) {
+          LOG.error("state model def is null." + "resource:" + currentState.getResourceId()
+              + ", partitions: " + currentState.getPartitionStateMap().keySet()
+              + ", states: " + currentState.getPartitionStateMap().values());
+          throw new StageException("State model def is null for resource:"
+              + currentState.getResourceId());
+        }
 
-        Map<String, CurrentState> currentStateMap =
-            cache.getCurrentState(instanceName, clientSessionId);
-        if (currentStateMap == null || currentStateMap.size() == 0) {
-          continue;
+        if (!resCfgBuilderMap.containsKey(resourceId)) {
+          PartitionedRebalancerContext.Builder rebCtxBuilder =
+              new PartitionedRebalancerContext.Builder(resourceId);
+          rebCtxBuilder.stateModelDefId(currentState.getStateModelDefId());
+          rebCtxBuilder.stateModelFactoryId(StateModelFactoryId.from(currentState
+              .getStateModelFactoryName()));
+          rebCtxBuilderMap.put(resourceId, rebCtxBuilder);
+
+          ResourceConfig.Builder resCfgBuilder = new ResourceConfig.Builder(resourceId);
+          resCfgBuilder.bucketSize(currentState.getBucketSize());
+          resCfgBuilder.batchMessageMode(currentState.getBatchMessageMode());
+          resCfgBuilderMap.put(resourceId, resCfgBuilder);
         }
-        for (CurrentState currentState : currentStateMap.values()) {
-
-          String resourceName = currentState.getResourceName();
-          Map<String, String> resourceStateMap = currentState.getPartitionStateMap();
-
-          // don't overwrite ideal state settings
-          if (!resourceMap.containsKey(resourceName)) {
-            addResource(resourceName, resourceMap);
-            Resource resource = resourceMap.get(resourceName);
-            resource.setStateModelDefRef(currentState.getStateModelDefRef());
-            resource.setStateModelFactoryName(currentState.getStateModelFactoryName());
-            resource.setBucketSize(currentState.getBucketSize());
-            resource.setBatchMessageMode(currentState.getBatchMessageMode());
-          }
-
-          if (currentState.getStateModelDefRef() == null) {
-            LOG.error("state model def is null." + "resource:" + currentState.getResourceName()
-                + ", partitions: " + currentState.getPartitionStateMap().keySet()
-                + ", states: " + currentState.getPartitionStateMap().values());
-            throw new StageException("State model def is null for resource:"
-                + currentState.getResourceName());
-          }
-
-          for (String partition : resourceStateMap.keySet()) {
-            addPartition(partition, resourceName, resourceMap);
-          }
+
+        PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId);
+        for (PartitionId partitionId : currentState.getTypedPartitionStateMap().keySet()) {
+          rebCtxBuilder.addPartition(new Partition(partitionId));
         }
       }
     }
 
-    event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
-  }
-
-  private void addResource(String resource, Map<String, Resource> resourceMap) {
-    if (resource == null || resourceMap == null) {
-      return;
-    }
-    if (!resourceMap.containsKey(resource)) {
-      resourceMap.put(resource, new Resource(resource));
-    }
-  }
-
-  private void addPartition(String partition, String resourceName, Map<String, Resource> resourceMap) {
-    if (resourceName == null || partition == null || resourceMap == null) {
-      return;
-    }
-    if (!resourceMap.containsKey(resourceName)) {
-      resourceMap.put(resourceName, new Resource(resourceName));
+    Map<ResourceId, ResourceConfig> resCfgMap = new HashMap<ResourceId, ResourceConfig>();
+    for (ResourceId resourceId : resCfgBuilderMap.keySet()) {
+      ResourceConfig.Builder resCfgBuilder = resCfgBuilderMap.get(resourceId);
+      PartitionedRebalancerContext.Builder rebCtxBuilder = rebCtxBuilderMap.get(resourceId);
+      resCfgBuilder.rebalancerContext(rebCtxBuilder.build());
+      resCfgMap.put(resourceId, resCfgBuilder.build());
     }
-    Resource resource = resourceMap.get(resourceName);
-    resource.addPartition(partition);
 
+    return resCfgMap;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 c942db9..02188be 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,16 +30,17 @@ 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.Cluster;
+import org.apache.helix.api.Participant;
+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.LiveInstance;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.log4j.Logger;
 
-@Deprecated
 public class TaskAssignmentStage extends AbstractBaseStage {
   private static Logger logger = Logger.getLogger(TaskAssignmentStage.class);
 
@@ -49,30 +50,30 @@ public class TaskAssignmentStage extends AbstractBaseStage {
     logger.info("START TaskAssignmentStage.process()");
 
     HelixManager manager = event.getAttribute("helixmanager");
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
-    MessageThrottleStageOutput messageOutput =
-        event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    Map<String, LiveInstance> liveInstanceMap = cache.getLiveInstances();
-
-    if (manager == null || resourceMap == null || messageOutput == null || cache == null
-        || liveInstanceMap == null) {
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
+    MessageOutput messageOutput = event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    Map<ParticipantId, Participant> liveParticipantMap = cluster.getLiveParticipantMap();
+
+    if (manager == null || resourceMap == null || messageOutput == null || cluster == null
+        || liveParticipantMap == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires HelixManager|RESOURCES|MESSAGES_THROTTLE|DataCache|liveInstanceMap");
     }
 
     HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
     List<Message> messagesToSend = new ArrayList<Message>();
-    for (String resourceName : resourceMap.keySet()) {
-      Resource resource = resourceMap.get(resourceName);
-      for (Partition partition : resource.getPartitions()) {
-        List<Message> messages = messageOutput.getMessages(resourceName, partition);
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      ResourceConfig resource = resourceMap.get(resourceId);
+      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+        List<Message> messages = messageOutput.getMessages(resourceId, partitionId);
         messagesToSend.addAll(messages);
       }
     }
 
     List<Message> outputMessages =
-        batchMessage(dataAccessor.keyBuilder(), messagesToSend, resourceMap, liveInstanceMap,
+        batchMessage(dataAccessor.keyBuilder(), messagesToSend, resourceMap, liveParticipantMap,
             manager.getProperties());
     sendMessages(dataAccessor, outputMessages);
 
@@ -82,8 +83,8 @@ public class TaskAssignmentStage extends AbstractBaseStage {
   }
 
   List<Message> batchMessage(Builder keyBuilder, List<Message> messages,
-      Map<String, Resource> resourceMap, Map<String, LiveInstance> liveInstanceMap,
-      HelixManagerProperties properties) {
+      Map<ResourceId, ResourceConfig> resourceMap,
+      Map<ParticipantId, Participant> liveParticipantMap, HelixManagerProperties properties) {
     // group messages by its CurrentState path + "/" + fromState + "/" + toState
     Map<String, Message> batchMessages = new HashMap<String, Message>();
     List<Message> outputMessages = new ArrayList<Message>();
@@ -92,13 +93,13 @@ public class TaskAssignmentStage extends AbstractBaseStage {
     while (iter.hasNext()) {
       Message message = iter.next();
       ResourceId resourceId = message.getResourceId();
-      Resource resource = resourceMap.get(resourceId.stringify());
+      ResourceConfig resource = resourceMap.get(resourceId);
 
-      String instanceName = message.getTgtName();
-      LiveInstance liveInstance = liveInstanceMap.get(instanceName);
+      ParticipantId participantId = ParticipantId.from(message.getTgtName());
+      Participant liveParticipant = liveParticipantMap.get(participantId);
       String participantVersion = null;
-      if (liveInstance != null) {
-        participantVersion = liveInstance.getTypedHelixVersion().toString();
+      if (liveParticipant != null) {
+        participantVersion = liveParticipant.getRunningInstance().getVersion().toString();
       }
 
       if (resource == null || !resource.getBatchMessageMode() || participantVersion == null
@@ -137,10 +138,10 @@ public class TaskAssignmentStage extends AbstractBaseStage {
           + " transit " + message.getPartitionId() + "|" + message.getPartitionIds() + " from:"
           + message.getTypedFromState() + " to:" + message.getTypedToState());
 
-      // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to " +
-      // message.getTgtName()
-      // + " transit " + message.getPartitionName() + "|" + message.getPartitionNames()
-      // + " from: " + message.getFromState() + " to: " + message.getToState());
+      // System.out.println("[dbg] Sending Message " + message.getMsgId() + " to "
+      // + message.getTgtName() + " transit " + message.getPartitionId() + "|"
+      // + message.getPartitionIds() + " from: " + message.getFromState() + " to: "
+      // + message.getToState());
 
       keys.add(keyBuilder.message(message.getTgtName(), message.getId()));
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 7d84258..805f6bf 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
@@ -38,7 +38,7 @@ 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.helix.api.rebalancer.RebalancerRef;
 import org.apache.log4j.Logger;
 
 import com.google.common.base.Function;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 f591a24..1563769 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
@@ -55,10 +55,10 @@ import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.controller.stages.NewBestPossibleStateCalcStage;
-import org.apache.helix.controller.stages.NewBestPossibleStateOutput;
-import org.apache.helix.controller.stages.NewCurrentStateComputationStage;
-import org.apache.helix.controller.stages.NewResourceComputationStage;
+import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
+import org.apache.helix.controller.stages.BestPossibleStateOutput;
+import org.apache.helix.controller.stages.CurrentStateComputationStage;
+import org.apache.helix.controller.stages.ResourceComputationStage;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
@@ -250,7 +250,7 @@ public class ClusterStateVerifier {
       ClusterAccessor clusterAccessor = new ClusterAccessor(ClusterId.from(clusterName), accessor);
       Cluster cluster = clusterAccessor.readCluster();
       // calculate best possible state
-      NewBestPossibleStateOutput bestPossOutput = ClusterStateVerifier.calcBestPossState(cluster);
+      BestPossibleStateOutput bestPossOutput = ClusterStateVerifier.calcBestPossState(cluster);
 
       // set error states
       if (errStates != null) {
@@ -416,19 +416,19 @@ public class ClusterStateVerifier {
    * @throws Exception
    */
 
-  static NewBestPossibleStateOutput calcBestPossState(Cluster cluster) throws Exception {
+  static BestPossibleStateOutput calcBestPossState(Cluster cluster) throws Exception {
     ClusterEvent event = new ClusterEvent("sampleEvent");
     event.addAttribute("ClusterDataCache", cluster);
 
-    NewResourceComputationStage rcState = new NewResourceComputationStage();
-    NewCurrentStateComputationStage csStage = new NewCurrentStateComputationStage();
-    NewBestPossibleStateCalcStage bpStage = new NewBestPossibleStateCalcStage();
+    ResourceComputationStage rcState = new ResourceComputationStage();
+    CurrentStateComputationStage csStage = new CurrentStateComputationStage();
+    BestPossibleStateCalcStage bpStage = new BestPossibleStateCalcStage();
 
     runStage(event, rcState);
     runStage(event, csStage);
     runStage(event, bpStage);
 
-    NewBestPossibleStateOutput output =
+    BestPossibleStateOutput output =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
 
     return output;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
index 85330be..f48ebbc 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/NewClusterSetup.java
@@ -45,10 +45,10 @@ 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.CustomRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
+import org.apache.helix.api.rebalancer.CustomRebalancerContext;
+import org.apache.helix.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerContext;
+import org.apache.helix.api.rebalancer.SemiAutoRebalancerContext;
 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/9f229c80/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 c478bbb..7fe3314 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
@@ -33,11 +33,11 @@ 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.api.rebalancer.SemiAutoRebalancerContext;
 import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.ClusterEvent;
-import org.apache.helix.controller.stages.NewBestPossibleStateCalcStage;
-import org.apache.helix.controller.stages.NewBestPossibleStateOutput;
+import org.apache.helix.controller.stages.BestPossibleStateCalcStage;
+import org.apache.helix.controller.stages.BestPossibleStateOutput;
 import org.apache.helix.controller.stages.ResourceCurrentState;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -130,13 +130,13 @@ public class TestNewStages extends ZkUnitTestBase {
 
     // Run the stage
     try {
-      new NewBestPossibleStateCalcStage().process(event);
+      new BestPossibleStateCalcStage().process(event);
     } catch (Exception e) {
       Assert.fail(e.toString());
     }
 
     // Verify the result
-    NewBestPossibleStateOutput bestPossibleStateOutput =
+    BestPossibleStateOutput bestPossibleStateOutput =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     Assert.assertNotNull(bestPossibleStateOutput);
     ResourceId resourceId = ResourceId.from("TestDB0");

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
index 74781cd..0a578c1 100644
--- a/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java
+++ b/helix-core/src/test/java/org/apache/helix/api/TestUpdateConfig.java
@@ -9,8 +9,8 @@ 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.apache.helix.api.rebalancer.FullAutoRebalancerContext;
+import org.apache.helix.api.rebalancer.SemiAutoRebalancerContext;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 5bbe54f..8650475 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
@@ -8,6 +8,9 @@ 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.rebalancer.CustomRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 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/9f229c80/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 ecb8151..6279087 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
@@ -39,10 +39,10 @@ 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.api.rebalancer.PartitionedRebalancerContext;
+import org.apache.helix.api.rebalancer.RebalancerContext;
 import org.apache.helix.controller.pipeline.Stage;
 import org.apache.helix.controller.pipeline.StageContext;
-import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
 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/9f229c80/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 18e8f4d..cb60691 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
@@ -62,12 +62,12 @@ public class TestBestPossibleCalcStageCompatibility extends BaseStageTest {
     event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
     event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
 
-    NewReadClusterDataStage stage1 = new NewReadClusterDataStage();
+    ReadClusterDataStage stage1 = new ReadClusterDataStage();
     runStage(event, stage1);
-    NewBestPossibleStateCalcStage stage2 = new NewBestPossibleStateCalcStage();
+    BestPossibleStateCalcStage stage2 = new BestPossibleStateCalcStage();
     runStage(event, stage2);
 
-    NewBestPossibleStateOutput output =
+    BestPossibleStateOutput output =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     for (int p = 0; p < 5; p++) {
       Map<ParticipantId, State> replicaMap =
@@ -100,12 +100,12 @@ public class TestBestPossibleCalcStageCompatibility extends BaseStageTest {
     event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
     event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
 
-    NewReadClusterDataStage stage1 = new NewReadClusterDataStage();
+    ReadClusterDataStage stage1 = new ReadClusterDataStage();
     runStage(event, stage1);
-    NewBestPossibleStateCalcStage stage2 = new NewBestPossibleStateCalcStage();
+    BestPossibleStateCalcStage stage2 = new BestPossibleStateCalcStage();
     runStage(event, stage2);
 
-    NewBestPossibleStateOutput output =
+    BestPossibleStateOutput output =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     for (int p = 0; p < 5; p++) {
       Map<ParticipantId, State> replicaMap =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 d3f348e..d116182 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
@@ -52,12 +52,12 @@ public class TestBestPossibleStateCalcStage extends BaseStageTest {
     event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
     event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
 
-    NewReadClusterDataStage stage1 = new NewReadClusterDataStage();
+    ReadClusterDataStage stage1 = new ReadClusterDataStage();
     runStage(event, stage1);
-    NewBestPossibleStateCalcStage stage2 = new NewBestPossibleStateCalcStage();
+    BestPossibleStateCalcStage stage2 = new BestPossibleStateCalcStage();
     runStage(event, stage2);
 
-    NewBestPossibleStateOutput output =
+    BestPossibleStateOutput output =
         event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
     for (int p = 0; p < 5; p++) {
       Map<ParticipantId, State> replicaMap =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 fb113b9..9d1dd04 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
@@ -78,13 +78,13 @@ public class TestCompatibilityCheckStage extends BaseStageTest {
           .put("minimum_supported_version.participant", minSupportedParticipantVersion);
     }
     event.addAttribute("helixmanager", manager);
-    runStage(event, new NewReadClusterDataStage());
+    runStage(event, new ReadClusterDataStage());
   }
 
   @Test
   public void testCompatible() {
     prepare("0.4.0", "0.4.0");
-    NewCompatibilityCheckStage stage = new NewCompatibilityCheckStage();
+    CompatibilityCheckStage stage = new CompatibilityCheckStage();
     StageContext context = new StageContext();
     stage.init(context);
     stage.preProcess();
@@ -99,7 +99,7 @@ public class TestCompatibilityCheckStage extends BaseStageTest {
   @Test
   public void testNullParticipantVersion() {
     prepare("0.4.0", null);
-    NewCompatibilityCheckStage stage = new NewCompatibilityCheckStage();
+    CompatibilityCheckStage stage = new CompatibilityCheckStage();
     StageContext context = new StageContext();
     stage.init(context);
     stage.preProcess();
@@ -115,7 +115,7 @@ public class TestCompatibilityCheckStage extends BaseStageTest {
   @Test
   public void testNullControllerVersion() {
     prepare(null, "0.4.0");
-    NewCompatibilityCheckStage stage = new NewCompatibilityCheckStage();
+    CompatibilityCheckStage stage = new CompatibilityCheckStage();
     StageContext context = new StageContext();
     stage.init(context);
     stage.preProcess();
@@ -131,7 +131,7 @@ public class TestCompatibilityCheckStage extends BaseStageTest {
   @Test
   public void testIncompatible() {
     prepare("0.6.1-incubating-SNAPSHOT", "0.3.4", "0.4");
-    NewCompatibilityCheckStage stage = new NewCompatibilityCheckStage();
+    CompatibilityCheckStage stage = new CompatibilityCheckStage();
     StageContext context = new StageContext();
     stage.init(context);
     stage.preProcess();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 3412e0a..65d551d 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
@@ -48,8 +48,8 @@ public class TestCurrentStateComputationStage extends BaseStageTest {
     List<IdealState> idealStates = setupIdealState(5, resources, 10, 1, RebalanceMode.SEMI_AUTO);
     Map<ResourceId, ResourceConfig> resourceMap = getResourceMap(idealStates);
     event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
-    NewCurrentStateComputationStage stage = new NewCurrentStateComputationStage();
-    runStage(event, new NewReadClusterDataStage());
+    CurrentStateComputationStage stage = new CurrentStateComputationStage();
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
     ResourceCurrentState output = event.getAttribute(AttributeName.CURRENT_STATE.toString());
     AssertJUnit.assertEquals(
@@ -69,8 +69,8 @@ public class TestCurrentStateComputationStage extends BaseStageTest {
     setupLiveInstances(5);
 
     event.addAttribute(AttributeName.RESOURCES.toString(), resourceMap);
-    NewCurrentStateComputationStage stage = new NewCurrentStateComputationStage();
-    runStage(event, new NewReadClusterDataStage());
+    CurrentStateComputationStage stage = new CurrentStateComputationStage();
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
     ResourceCurrentState output1 = event.getAttribute(AttributeName.CURRENT_STATE.toString());
     AssertJUnit.assertEquals(
@@ -89,7 +89,7 @@ public class TestCurrentStateComputationStage extends BaseStageTest {
     Builder keyBuilder = accessor.keyBuilder();
     accessor.setProperty(keyBuilder.message("localhost_" + 3, message.getId()), message);
 
-    runStage(event, new NewReadClusterDataStage());
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
     ResourceCurrentState output2 = event.getAttribute(AttributeName.CURRENT_STATE.toString());
     State pendingState =
@@ -114,7 +114,7 @@ public class TestCurrentStateComputationStage extends BaseStageTest {
     accessor.setProperty(
         keyBuilder.currentState("localhost_3", "session_dead", "testResourceName"),
         stateWithDeadSession);
-    runStage(event, new NewReadClusterDataStage());
+    runStage(event, new ReadClusterDataStage());
     runStage(event, stage);
     ResourceCurrentState output3 = event.getAttribute(AttributeName.CURRENT_STATE.toString());
     State currentState =

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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 0bd8795..ba61361 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
@@ -78,7 +78,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     ClusterEvent event = new ClusterEvent("testEvent");
     event.addAttribute("helixmanager", manager);
 
-    NewMessageThrottleStage throttleStage = new NewMessageThrottleStage();
+    MessageThrottleStage throttleStage = new MessageThrottleStage();
     try {
       runStage(event, throttleStage);
       Assert.fail("Should throw exception since DATA_CACHE is null");
@@ -87,7 +87,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     }
 
     Pipeline dataRefresh = new Pipeline();
-    dataRefresh.addStage(new NewReadClusterDataStage());
+    dataRefresh.addStage(new ReadClusterDataStage());
     runPipeline(event, dataRefresh);
 
     try {
@@ -96,7 +96,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     } catch (Exception e) {
       // OK
     }
-    runStage(event, new NewResourceComputationStage());
+    runStage(event, new ResourceComputationStage());
 
     try {
       runStage(event, throttleStage);
@@ -104,7 +104,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     } catch (Exception e) {
       // OK
     }
-    NewMessageOutput msgSelectOutput = new NewMessageOutput();
+    MessageOutput msgSelectOutput = new MessageOutput();
     List<Message> selectMessages = new ArrayList<Message>();
     Message msg =
         createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE",
@@ -117,7 +117,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
 
     runStage(event, throttleStage);
 
-    NewMessageOutput msgThrottleOutput =
+    MessageOutput msgThrottleOutput =
         event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
     Assert.assertEquals(
         msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"))
@@ -221,7 +221,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     ClusterConstraints constraint =
         accessor.getProperty(keyBuilder.constraint(ConstraintType.MESSAGE_CONSTRAINT.toString()));
 
-    NewMessageThrottleStage throttleStage = new NewMessageThrottleStage();
+    MessageThrottleStage throttleStage = new MessageThrottleStage();
 
     // test constraintSelection
     // message1: hit contraintSelection rule1 and rule2
@@ -271,10 +271,10 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     event.addAttribute("helixmanager", manager);
 
     Pipeline dataRefresh = new Pipeline();
-    dataRefresh.addStage(new NewReadClusterDataStage());
+    dataRefresh.addStage(new ReadClusterDataStage());
     runPipeline(event, dataRefresh);
-    runStage(event, new NewResourceComputationStage());
-    NewMessageOutput msgSelectOutput = new NewMessageOutput();
+    runStage(event, new ResourceComputationStage());
+    MessageOutput msgSelectOutput = new MessageOutput();
 
     Message msg3 =
         createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-003"), "OFFLINE",
@@ -306,7 +306,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
 
     runStage(event, throttleStage);
 
-    NewMessageOutput msgThrottleOutput =
+    MessageOutput msgThrottleOutput =
         event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
     List<Message> throttleMessages =
         msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"));


[6/9] [HELIX-209] Shuffling around rebalancer code to allow for compatibility

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
new file mode 100644
index 0000000..5d9746b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationStage.java
@@ -0,0 +1,213 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.State;
+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.api.rebalancer.RebalancerContext;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Message.MessageState;
+import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+/**
+ * Compares the currentState, pendingState with IdealState and generate messages
+ */
+public class MessageGenerationStage extends AbstractBaseStage {
+  private static Logger LOG = Logger.getLogger(MessageGenerationStage.class);
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    HelixManager manager = event.getAttribute("helixmanager");
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    Map<StateModelDefId, StateModelDefinition> stateModelDefMap = cluster.getStateModelMap();
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
+    ResourceCurrentState currentStateOutput =
+        event.getAttribute(AttributeName.CURRENT_STATE.toString());
+    BestPossibleStateOutput bestPossibleStateOutput =
+        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
+    if (manager == null || cluster == null || resourceMap == null || currentStateOutput == null
+        || bestPossibleStateOutput == null) {
+      throw new StageException("Missing attributes in event:" + event
+          + ". Requires HelixManager|DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE");
+    }
+
+    MessageOutput output = new MessageOutput();
+
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      ResourceConfig resourceConfig = resourceMap.get(resourceId);
+      int bucketSize = resourceConfig.getBucketSize();
+
+      RebalancerContext rebalancerCtx =
+          resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
+      StateModelDefinition stateModelDef = stateModelDefMap.get(rebalancerCtx.getStateModelDefId());
+
+      ResourceAssignment resourceAssignment =
+          bestPossibleStateOutput.getResourceAssignment(resourceId);
+      for (PartitionId subUnitId : resourceConfig.getSubUnitMap().keySet()) {
+        Map<ParticipantId, State> instanceStateMap = resourceAssignment.getReplicaMap(subUnitId);
+
+        // we should generate message based on the desired-state priority
+        // so keep generated messages in a temp map keyed by state
+        // desired-state->list of generated-messages
+        Map<State, List<Message>> messageMap = new HashMap<State, List<Message>>();
+
+        for (ParticipantId participantId : instanceStateMap.keySet()) {
+          State desiredState = instanceStateMap.get(participantId);
+
+          State currentState =
+              currentStateOutput.getCurrentState(resourceId, subUnitId, participantId);
+          if (currentState == null) {
+            currentState = stateModelDef.getTypedInitialState();
+          }
+
+          if (desiredState.equals(currentState)) {
+            continue;
+          }
+
+          State pendingState =
+              currentStateOutput.getPendingState(resourceId, subUnitId, participantId);
+
+          // TODO fix it
+          State nextState = stateModelDef.getNextStateForTransition(currentState, desiredState);
+          if (nextState == null) {
+            LOG.error("Unable to find a next state for partition: " + subUnitId
+                + " from stateModelDefinition" + stateModelDef.getClass() + " from:" + currentState
+                + " to:" + desiredState);
+            continue;
+          }
+
+          if (pendingState != null) {
+            if (nextState.equals(pendingState)) {
+              LOG.debug("Message already exists for " + participantId + " to transit " + subUnitId
+                  + " from " + currentState + " to " + nextState);
+            } else if (currentState.equals(pendingState)) {
+              LOG.info("Message hasn't been removed for " + participantId + " to transit"
+                  + subUnitId + " to " + pendingState + ", desiredState: " + desiredState);
+            } else {
+              LOG.info("IdealState changed before state transition completes for " + subUnitId
+                  + " on " + participantId + ", pendingState: " + pendingState + ", currentState: "
+                  + currentState + ", nextState: " + nextState);
+            }
+          } else {
+            // TODO check if instance is alive
+            SessionId sessionId =
+                cluster.getLiveParticipantMap().get(participantId).getRunningInstance()
+                    .getSessionId();
+            RebalancerContext rebalancerContext =
+                resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
+            Message message =
+                createMessage(manager, resourceId, subUnitId, participantId, currentState,
+                    nextState, sessionId, StateModelDefId.from(stateModelDef.getId()),
+                    rebalancerContext.getStateModelFactoryId(), bucketSize);
+
+            // TODO refactor get/set timeout/inner-message
+            if (rebalancerContext != null
+                && rebalancerContext.getStateModelDefId().equalsIgnoreCase(
+                    StateModelDefId.SchedulerTaskQueue)) {
+              if (resourceConfig.getSubUnitMap().size() > 0) {
+                // TODO refactor it -- we need a way to read in scheduler tasks a priori
+                Message innerMsg =
+                    resourceConfig.getSchedulerTaskConfig().getInnerMessage(subUnitId);
+                if (innerMsg != null) {
+                  message.setInnerMessage(innerMsg);
+                }
+              }
+            }
+
+            // Set timeout if needed
+            String stateTransition =
+                String.format("%s-%s_%s", currentState, nextState,
+                    Message.Attributes.TIMEOUT.name());
+            SchedulerTaskConfig schedulerTaskConfig = resourceConfig.getSchedulerTaskConfig();
+            if (schedulerTaskConfig != null) {
+              int timeout = schedulerTaskConfig.getTimeout(stateTransition, subUnitId);
+              if (timeout > 0) {
+                message.setExecutionTimeout(timeout);
+              }
+            }
+            message.setClusterEvent(event);
+
+            if (!messageMap.containsKey(desiredState)) {
+              messageMap.put(desiredState, new ArrayList<Message>());
+            }
+            messageMap.get(desiredState).add(message);
+          }
+        }
+
+        // add generated messages to output according to state priority
+        List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
+        for (State state : statesPriorityList) {
+          if (messageMap.containsKey(state)) {
+            for (Message message : messageMap.get(state)) {
+              output.addMessage(resourceId, subUnitId, message);
+            }
+          }
+        }
+
+      } // end of for-each-partition
+    }
+    event.addAttribute(AttributeName.MESSAGES_ALL.toString(), output);
+    // System.out.println("output: " + output);
+  }
+
+  private Message createMessage(HelixManager manager, ResourceId resourceId,
+      PartitionId partitionId, ParticipantId participantId, State currentState, State nextState,
+      SessionId participantSessionId, StateModelDefId stateModelDefId,
+      StateModelFactoryId stateModelFactoryId, int bucketSize) {
+    MessageId uuid = MessageId.from(UUID.randomUUID().toString());
+    Message message = new Message(MessageType.STATE_TRANSITION, uuid);
+    message.setSrcName(manager.getInstanceName());
+    message.setTgtName(participantId.stringify());
+    message.setMsgState(MessageState.NEW);
+    message.setPartitionId(partitionId);
+    message.setResourceId(resourceId);
+    message.setFromState(currentState);
+    message.setToState(nextState);
+    message.setTgtSessionId(participantSessionId);
+    message.setSrcSessionId(SessionId.from(manager.getSessionId()));
+    message.setStateModelDef(stateModelDefId);
+    message.setStateModelFactoryId(stateModelFactoryId);
+    message.setBucketSize(bucketSize);
+
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
new file mode 100644
index 0000000..9c8c154
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
@@ -0,0 +1,79 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.model.Message;
+
+public class MessageOutput {
+
+  private final Map<ResourceId, Map<PartitionId, List<Message>>> _messagesMap;
+
+  public MessageOutput() {
+    _messagesMap = new HashMap<ResourceId, Map<PartitionId, List<Message>>>();
+
+  }
+
+  public void addMessage(ResourceId resourceId, PartitionId partitionId, Message message) {
+    if (!_messagesMap.containsKey(resourceId)) {
+      _messagesMap.put(resourceId, new HashMap<PartitionId, List<Message>>());
+    }
+    if (!_messagesMap.get(resourceId).containsKey(partitionId)) {
+      _messagesMap.get(resourceId).put(partitionId, new ArrayList<Message>());
+
+    }
+    _messagesMap.get(resourceId).get(partitionId).add(message);
+
+  }
+
+  public void setMessages(ResourceId resourceId, PartitionId partitionId,
+      List<Message> selectedMessages) {
+    if (!_messagesMap.containsKey(resourceId)) {
+      _messagesMap.put(resourceId, new HashMap<PartitionId, List<Message>>());
+    }
+    _messagesMap.get(resourceId).put(partitionId, selectedMessages);
+
+  }
+
+  public List<Message> getMessages(ResourceId resourceId, PartitionId partitionId) {
+    Map<PartitionId, List<Message>> map = _messagesMap.get(resourceId);
+    if (map != null) {
+      return map.get(partitionId);
+    }
+    return Collections.emptyList();
+
+  }
+
+  public Map<PartitionId, List<Message>> getMessages(ResourceId resourceId) {
+    return _messagesMap.get(resourceId);
+  }
+
+  @Override
+  public String toString() {
+    return _messagesMap.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
index 1a3f37b..15004a6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStage.java
@@ -26,18 +26,25 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.Participant;
+import org.apache.helix.api.Resource;
+import org.apache.helix.api.Scope;
 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.api.id.StateModelDefId;
+import org.apache.helix.api.rebalancer.RebalancerConfig;
+import org.apache.helix.api.rebalancer.RebalancerContext;
+import org.apache.helix.api.rebalancer.ReplicatedRebalancerContext;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.log4j.Logger;
 
-@Deprecated
 public class MessageSelectionStage extends AbstractBaseStage {
   private static final Logger LOG = Logger.getLogger(MessageSelectionStage.class);
 
@@ -73,41 +80,54 @@ public class MessageSelectionStage extends AbstractBaseStage {
     public int getUpperBound() {
       return upper;
     }
+
+    @Override
+    public String toString() {
+      return String.format("%d-%d", lower, upper);
+    }
   }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
-    CurrentStateOutput currentStateOutput =
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    Map<StateModelDefId, StateModelDefinition> stateModelDefMap = cluster.getStateModelMap();
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
+    ResourceCurrentState currentStateOutput =
         event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    MessageGenerationOutput messageGenOutput =
-        event.getAttribute(AttributeName.MESSAGES_ALL.toString());
-    if (cache == null || resourceMap == null || currentStateOutput == null
+    MessageOutput messageGenOutput = event.getAttribute(AttributeName.MESSAGES_ALL.toString());
+    if (cluster == null || resourceMap == null || currentStateOutput == null
         || messageGenOutput == null) {
       throw new StageException("Missing attributes in event:" + event
           + ". Requires DataCache|RESOURCES|CURRENT_STATE|MESSAGES_ALL");
     }
 
-    MessageSelectionStageOutput output = new MessageSelectionStageOutput();
+    MessageOutput output = new MessageOutput();
 
-    for (String resourceName : resourceMap.keySet()) {
-      Resource resource = resourceMap.get(resourceName);
-      StateModelDefinition stateModelDef = cache.getStateModelDef(resource.getStateModelDefRef());
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      ResourceConfig resource = resourceMap.get(resourceId);
+      StateModelDefinition stateModelDef =
+          stateModelDefMap.get(resource.getRebalancerConfig()
+              .getRebalancerContext(RebalancerContext.class).getStateModelDefId());
 
+      // TODO have a logical model for transition
       Map<String, Integer> stateTransitionPriorities = getStateTransitionPriorityMap(stateModelDef);
-      IdealState idealState = cache.getIdealState(resourceName);
-      Map<String, Bounds> stateConstraints =
-          computeStateConstraints(stateModelDef, idealState, cache);
+      Resource configResource = cluster.getResource(resourceId);
+
+      // if configResource == null, the resource has been dropped
+      Map<State, Bounds> stateConstraints =
+          computeStateConstraints(stateModelDef,
+              configResource == null ? null : configResource.getRebalancerConfig(), cluster);
 
-      for (Partition partition : resource.getPartitions()) {
-        List<Message> messages = messageGenOutput.getMessages(resourceName, partition);
+      // TODO fix it
+      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+        List<Message> messages = messageGenOutput.getMessages(resourceId, partitionId);
         List<Message> selectedMessages =
-            selectMessages(cache.getLiveInstances(),
-                currentStateOutput.getCurrentStateMap(resourceName, partition),
-                currentStateOutput.getPendingStateMap(resourceName, partition), messages,
-                stateConstraints, stateTransitionPriorities, stateModelDef.getInitialState());
-        output.addMessages(resourceName, partition, selectedMessages);
+            selectMessages(cluster.getLiveParticipantMap(),
+                currentStateOutput.getCurrentStateMap(resourceId, partitionId),
+                currentStateOutput.getPendingStateMap(resourceId, partitionId), messages,
+                stateConstraints, stateTransitionPriorities, stateModelDef.getTypedInitialState());
+        output.setMessages(resourceId, partitionId, selectedMessages);
       }
     }
     event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), output);
@@ -132,22 +152,22 @@ public class MessageSelectionStage extends AbstractBaseStage {
    *          : FROME_STATE-TO_STATE -> priority
    * @return: selected messages
    */
-  List<Message> selectMessages(Map<String, LiveInstance> liveInstances,
-      Map<String, String> currentStates, Map<String, String> pendingStates, List<Message> messages,
-      Map<String, Bounds> stateConstraints, final Map<String, Integer> stateTransitionPriorities,
-      String initialState) {
+  List<Message> selectMessages(Map<ParticipantId, Participant> liveParticipants,
+      Map<ParticipantId, State> currentStates, Map<ParticipantId, State> pendingStates,
+      List<Message> messages, Map<State, Bounds> stateConstraints,
+      final Map<String, Integer> stateTransitionPriorities, State initialState) {
     if (messages == null || messages.isEmpty()) {
       return Collections.emptyList();
     }
 
     List<Message> selectedMessages = new ArrayList<Message>();
-    Map<String, Bounds> bounds = new HashMap<String, Bounds>();
+    Map<State, Bounds> bounds = new HashMap<State, Bounds>();
 
     // count currentState, if no currentState, count as in initialState
-    for (String instance : liveInstances.keySet()) {
-      String state = initialState;
-      if (currentStates.containsKey(instance)) {
-        state = currentStates.get(instance);
+    for (ParticipantId liveParticipantId : liveParticipants.keySet()) {
+      State state = initialState;
+      if (currentStates.containsKey(liveParticipantId)) {
+        state = currentStates.get(liveParticipantId);
       }
 
       if (!bounds.containsKey(state)) {
@@ -158,8 +178,8 @@ public class MessageSelectionStage extends AbstractBaseStage {
     }
 
     // count pendingStates
-    for (String instance : pendingStates.keySet()) {
-      String state = pendingStates.get(instance);
+    for (ParticipantId participantId : pendingStates.keySet()) {
+      State state = pendingStates.get(participantId);
       if (!bounds.containsKey(state)) {
         bounds.put(state, new Bounds(0, 0));
       }
@@ -173,7 +193,7 @@ public class MessageSelectionStage extends AbstractBaseStage {
     for (Message message : messages) {
       State fromState = message.getTypedFromState();
       State toState = message.getTypedToState();
-      String transition = fromState + "-" + toState;
+      String transition = fromState.toString() + "-" + toState.toString();
       int priority = Integer.MAX_VALUE;
 
       if (stateTransitionPriorities.containsKey(transition)) {
@@ -198,7 +218,7 @@ public class MessageSelectionStage extends AbstractBaseStage {
         }
 
         if (!bounds.containsKey(toState)) {
-          bounds.put(toState.toString(), new Bounds(0, 0));
+          bounds.put(toState, new Bounds(0, 0));
         }
 
         // check lower bound of fromState
@@ -236,22 +256,35 @@ public class MessageSelectionStage extends AbstractBaseStage {
    * TODO: This code is duplicate in multiple places. Can we do it in to one place in the
    * beginning and compute the stateConstraint instance once and re use at other places.
    * Each IdealState must have a constraint object associated with it
+   * @param stateModelDefinition
+   * @param rebalancerConfig if rebalancerConfig == null, we can't evaluate R thus no constraints
+   * @param cluster
+   * @return
    */
-  private Map<String, Bounds> computeStateConstraints(StateModelDefinition stateModelDefinition,
-      IdealState idealState, ClusterDataCache cache) {
-    Map<String, Bounds> stateConstraints = new HashMap<String, Bounds>();
-
-    List<String> statePriorityList = stateModelDefinition.getStatesPriorityList();
-    for (String state : statePriorityList) {
-      String numInstancesPerState = stateModelDefinition.getNumInstancesPerState(state);
+  private Map<State, Bounds> computeStateConstraints(StateModelDefinition stateModelDefinition,
+      RebalancerConfig rebalancerConfig, Cluster cluster) {
+    ReplicatedRebalancerContext context =
+        (rebalancerConfig != null) ? rebalancerConfig
+            .getRebalancerContext(ReplicatedRebalancerContext.class) : null;
+    Map<State, Bounds> stateConstraints = new HashMap<State, Bounds>();
+
+    List<State> statePriorityList = stateModelDefinition.getTypedStatesPriorityList();
+    for (State state : statePriorityList) {
+      String numInstancesPerState =
+          cluster.getStateUpperBoundConstraint(Scope.cluster(cluster.getId()),
+              stateModelDefinition.getStateModelDefId(), state);
       int max = -1;
       if ("N".equals(numInstancesPerState)) {
-        max = cache.getLiveInstances().size();
+        max = cluster.getLiveParticipantMap().size();
       } else if ("R".equals(numInstancesPerState)) {
         // idealState is null when resource has been dropped,
         // R can't be evaluated and ignore state constraints
-        if (idealState != null) {
-          max = cache.getReplicas(idealState.getResourceName());
+        if (context != null) {
+          if (context.anyLiveParticipant()) {
+            max = cluster.getLiveParticipantMap().size();
+          } else {
+            max = context.getReplicaCount();
+          }
         }
       } else {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java
deleted file mode 100644
index 54ab384..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageSelectionStageOutput.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-
-@Deprecated
-public class MessageSelectionStageOutput {
-  private final Map<String, Map<Partition, List<Message>>> _messagesMap;
-
-  public MessageSelectionStageOutput() {
-    _messagesMap = new HashMap<String, Map<Partition, List<Message>>>();
-  }
-
-  public void addMessages(String resourceName, Partition partition, List<Message> selectedMessages) {
-    if (!_messagesMap.containsKey(resourceName)) {
-      _messagesMap.put(resourceName, new HashMap<Partition, List<Message>>());
-    }
-    _messagesMap.get(resourceName).put(partition, selectedMessages);
-
-  }
-
-  public List<Message> getMessages(String resourceName, Partition partition) {
-    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
-    if (map != null) {
-      return map.get(partition);
-    }
-    return Collections.emptyList();
-
-  }
-
-  @Override
-  public String toString() {
-    return _messagesMap.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
index 62fbafe..a7b75a3 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
@@ -26,6 +26,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.api.Cluster;
+import org.apache.helix.api.Participant;
+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;
@@ -34,11 +40,8 @@ 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;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.log4j.Logger;
 
-@Deprecated
 public class MessageThrottleStage extends AbstractBaseStage {
   private static final Logger LOG = Logger.getLogger(MessageThrottleStage.class.getName());
 
@@ -113,39 +116,43 @@ public class MessageThrottleStage extends AbstractBaseStage {
 
   @Override
   public void process(ClusterEvent event) throws Exception {
-    ClusterDataCache cache = event.getAttribute("ClusterDataCache");
-    MessageSelectionStageOutput msgSelectionOutput =
+    Cluster cluster = event.getAttribute("ClusterDataCache");
+    MessageOutput msgSelectionOutput =
         event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
-    Map<String, Resource> resourceMap = event.getAttribute(AttributeName.RESOURCES.toString());
+    Map<ResourceId, ResourceConfig> resourceMap =
+        event.getAttribute(AttributeName.RESOURCES.toString());
 
-    if (cache == null || resourceMap == null || msgSelectionOutput == null) {
+    if (cluster == null || resourceMap == null || msgSelectionOutput == null) {
       throw new StageException("Missing attributes in event: " + event
           + ". Requires ClusterDataCache|RESOURCES|MESSAGES_SELECTED");
     }
 
-    MessageThrottleStageOutput output = new MessageThrottleStageOutput();
+    MessageOutput output = new MessageOutput();
 
-    ClusterConstraints constraint = cache.getConstraint(ConstraintType.MESSAGE_CONSTRAINT);
+    // TODO fix it
+    ClusterConstraints constraint = cluster.getConstraint(ConstraintType.MESSAGE_CONSTRAINT);
     Map<String, Integer> throttleCounterMap = new HashMap<String, Integer>();
 
     if (constraint != null) {
       // go through all pending messages, they should be counted but not throttled
-      for (String instance : cache.getLiveInstances().keySet()) {
-        throttle(throttleCounterMap, constraint, new ArrayList<Message>(cache.getMessages(instance)
-            .values()), false);
+      for (ParticipantId participantId : cluster.getLiveParticipantMap().keySet()) {
+        Participant liveParticipant = cluster.getLiveParticipantMap().get(participantId);
+        throttle(throttleCounterMap, constraint, new ArrayList<Message>(liveParticipant
+            .getMessageMap().values()), false);
       }
     }
 
     // go through all new messages, throttle if necessary
     // assume messages should be sorted by state transition priority in messageSelection stage
-    for (String resourceName : resourceMap.keySet()) {
-      Resource resource = resourceMap.get(resourceName);
-      for (Partition partition : resource.getPartitions()) {
-        List<Message> messages = msgSelectionOutput.getMessages(resourceName, partition);
+    for (ResourceId resourceId : resourceMap.keySet()) {
+      ResourceConfig resource = resourceMap.get(resourceId);
+      // TODO fix it
+      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
+        List<Message> messages = msgSelectionOutput.getMessages(resourceId, partitionId);
         if (constraint != null && messages != null && messages.size() > 0) {
           messages = throttle(throttleCounterMap, constraint, messages, true);
         }
-        output.addMessages(resourceName, partition, messages);
+        output.setMessages(resourceId, partitionId, messages);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java
deleted file mode 100644
index 5983eff..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStageOutput.java
+++ /dev/null
@@ -1,53 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-
-@Deprecated
-public class MessageThrottleStageOutput {
-  private final Map<String, Map<Partition, List<Message>>> _messagesMap;
-
-  public MessageThrottleStageOutput() {
-    _messagesMap = new HashMap<String, Map<Partition, List<Message>>>();
-  }
-
-  public void addMessages(String resourceName, Partition partition, List<Message> selectedMessages) {
-    if (!_messagesMap.containsKey(resourceName)) {
-      _messagesMap.put(resourceName, new HashMap<Partition, List<Message>>());
-    }
-    _messagesMap.get(resourceName).put(partition, selectedMessages);
-
-  }
-
-  public List<Message> getMessages(String resourceName, Partition partition) {
-    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
-    if (map != null) {
-      return map.get(partition);
-    }
-    return Collections.emptyList();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 8b56bec..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateCalcStage.java
+++ /dev/null
@@ -1,142 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-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.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;
-import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-/**
- * For partition compute best possible (instance,state) pair based on
- * IdealState,StateModel,LiveInstance
- */
-public class NewBestPossibleStateCalcStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(NewBestPossibleStateCalcStage.class.getName());
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-    if (LOG.isInfoEnabled()) {
-      LOG.info("START BestPossibleStateCalcStage.process()");
-    }
-
-    ResourceCurrentState currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-
-    if (currentStateOutput == null || resourceMap == null || cluster == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires CURRENT_STATE|RESOURCES|DataCache");
-    }
-
-    NewBestPossibleStateOutput bestPossibleStateOutput =
-        compute(cluster, event, resourceMap, currentStateOutput);
-    event.addAttribute(AttributeName.BEST_POSSIBLE_STATE.toString(), bestPossibleStateOutput);
-
-    long endTime = System.currentTimeMillis();
-    if (LOG.isInfoEnabled()) {
-      LOG.info("END BestPossibleStateCalcStage.process(). took: " + (endTime - startTime) + " ms");
-    }
-  }
-
-  /**
-   * Fallback for cases when the resource has been dropped, but current state exists
-   * @param cluster cluster snapshot
-   * @param resourceId the resource for which to generate an assignment
-   * @param currentStateOutput full snapshot of the current state
-   * @param stateModelDef state model the resource follows
-   * @return assignment for the dropped resource
-   */
-  private ResourceAssignment mapDroppedResource(Cluster cluster, ResourceId resourceId,
-      ResourceCurrentState currentStateOutput, StateModelDefinition stateModelDef) {
-    ResourceAssignment partitionMapping = new ResourceAssignment(resourceId);
-    Set<? extends PartitionId> mappedPartitions =
-        currentStateOutput.getCurrentStateMappedPartitions(resourceId);
-    if (mappedPartitions == null) {
-      return partitionMapping;
-    }
-    for (PartitionId partitionId : mappedPartitions) {
-      Set<ParticipantId> disabledParticipantsForPartition =
-          NewConstraintBasedAssignment.getDisabledParticipants(cluster.getParticipantMap(),
-              partitionId);
-      Map<State, String> upperBounds =
-          NewConstraintBasedAssignment.stateConstraints(stateModelDef, resourceId,
-              cluster.getConfig());
-      partitionMapping.addReplicaMap(partitionId, NewConstraintBasedAssignment
-          .computeAutoBestStateForPartition(upperBounds, cluster.getLiveParticipantMap().keySet(),
-              stateModelDef, null, currentStateOutput.getCurrentStateMap(resourceId, partitionId),
-              disabledParticipantsForPartition));
-    }
-    return partitionMapping;
-  }
-
-  private NewBestPossibleStateOutput compute(Cluster cluster, ClusterEvent event,
-      Map<ResourceId, ResourceConfig> resourceMap, ResourceCurrentState currentStateOutput) {
-    NewBestPossibleStateOutput output = new NewBestPossibleStateOutput();
-    Map<StateModelDefId, StateModelDefinition> stateModelDefs = cluster.getStateModelMap();
-
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Processing resource:" + resourceId);
-      }
-      ResourceConfig resourceConfig = resourceMap.get(resourceId);
-      RebalancerConfig rebalancerConfig = resourceConfig.getRebalancerConfig();
-      ResourceAssignment resourceAssignment = null;
-      if (rebalancerConfig != null) {
-        Rebalancer rebalancer = rebalancerConfig.getRebalancer();
-        if (rebalancer != null) {
-          HelixManager manager = event.getAttribute("helixmanager");
-          rebalancer.init(manager);
-          resourceAssignment =
-              rebalancer.computeResourceMapping(rebalancerConfig, cluster, currentStateOutput);
-        }
-      }
-      if (resourceAssignment == null) {
-        RebalancerContext context = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
-        StateModelDefinition stateModelDef = stateModelDefs.get(context.getStateModelDefId());
-        resourceAssignment =
-            mapDroppedResource(cluster, resourceId, currentStateOutput, stateModelDef);
-      }
-
-      output.setResourceAssignment(resourceId, resourceAssignment);
-    }
-
-    return output;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 7720143..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewBestPossibleStateOutput.java
+++ /dev/null
@@ -1,49 +0,0 @@
-package org.apache.helix.controller.stages;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.helix.api.id.ResourceId;
-import org.apache.helix.model.ResourceAssignment;
-
-import com.google.common.collect.Maps;
-
-public class NewBestPossibleStateOutput {
-
-  Map<ResourceId, ResourceAssignment> _resourceAssignmentMap;
-
-  public NewBestPossibleStateOutput() {
-    _resourceAssignmentMap = Maps.newHashMap();
-  }
-
-  /**
-   * Set the computed resource assignment for a resource
-   * @param resourceId the resource to set
-   * @param resourceAssignment the computed assignment
-   */
-  public void setResourceAssignment(ResourceId resourceId, ResourceAssignment resourceAssignment) {
-    _resourceAssignmentMap.put(resourceId, resourceAssignment);
-  }
-
-  /**
-   * Get the resource assignment computed for a resource
-   * @param resourceId resource to look up
-   * @return ResourceAssignment computed by the best possible state calculation
-   */
-  public ResourceAssignment getResourceAssignment(ResourceId resourceId) {
-    return _resourceAssignmentMap.get(resourceId);
-  }
-
-  /**
-   * Get all of the resources currently assigned
-   * @return set of assigned resource ids
-   */
-  public Set<ResourceId> getAssignedResources() {
-    return _resourceAssignmentMap.keySet();
-  }
-
-  @Override
-  public String toString() {
-    return _resourceAssignmentMap.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index ea1a507..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCompatibilityCheckStage.java
+++ /dev/null
@@ -1,68 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Map;
-
-import org.apache.helix.HelixManager;
-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.id.ParticipantId;
-import org.apache.helix.controller.pipeline.AbstractBaseStage;
-import org.apache.helix.controller.pipeline.StageException;
-import org.apache.log4j.Logger;
-
-/**
- * controller checks if participant version is compatible
- */
-public class NewCompatibilityCheckStage extends AbstractBaseStage {
-  private static final Logger LOG = Logger.getLogger(NewCompatibilityCheckStage.class.getName());
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    HelixManager manager = event.getAttribute("helixmanager");
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    if (manager == null || cluster == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires HelixManager | DataCache");
-    }
-
-    HelixManagerProperties properties = manager.getProperties();
-    // Map<String, LiveInstance> liveInstanceMap = cache.getLiveInstances();
-    Map<ParticipantId, Participant> liveParticipants = cluster.getLiveParticipantMap();
-    for (Participant liveParticipant : liveParticipants.values()) {
-      HelixVersion version = liveParticipant.getRunningInstance().getVersion();
-      String participantVersion = (version != null) ? version.toString() : null;
-      if (!properties.isParticipantCompatible(participantVersion)) {
-        String errorMsg =
-            "incompatible participant. pipeline will not continue. " + "controller: "
-                + manager.getInstanceName() + ", controllerVersion: " + properties.getVersion()
-                + ", minimumSupportedParticipantVersion: "
-                + properties.getProperty("minimum_supported_version.participant")
-                + ", participant: " + liveParticipant.getId() + ", participantVersion: "
-                + participantVersion;
-        LOG.error(errorMsg);
-        throw new StageException(errorMsg);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index f7f2a5f..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewCurrentStateComputationStage.java
+++ /dev/null
@@ -1,142 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.List;
-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.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.api.id.StateModelDefId;
-import org.apache.helix.controller.pipeline.AbstractBaseStage;
-import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageType;
-
-/**
- * For each LiveInstances select currentState and message whose sessionId matches
- * sessionId from LiveInstance Get Partition,State for all the resources computed in
- * previous State [ResourceComputationStage]
- */
-public class NewCurrentStateComputationStage extends AbstractBaseStage {
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-
-    if (cluster == null || resourceMap == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires DataCache|RESOURCE");
-    }
-
-    ResourceCurrentState currentStateOutput = new ResourceCurrentState();
-
-    for (Participant liveParticipant : cluster.getLiveParticipantMap().values()) {
-      ParticipantId participantId = liveParticipant.getId();
-
-      // add pending messages
-      Map<MessageId, Message> instanceMsgs = liveParticipant.getMessageMap();
-      for (Message message : instanceMsgs.values()) {
-        if (!MessageType.STATE_TRANSITION.toString().equalsIgnoreCase(message.getMsgType())) {
-          continue;
-        }
-
-        if (!liveParticipant.getRunningInstance().getSessionId().equals(message.getTypedTgtSessionId())) {
-          continue;
-        }
-
-        ResourceId resourceId = message.getResourceId();
-        ResourceConfig resource = resourceMap.get(resourceId);
-        if (resource == null) {
-          continue;
-        }
-
-        if (!message.getBatchMessageMode()) {
-          PartitionId partitionId = message.getPartitionId();
-          Partition partition = resource.getSubUnit(partitionId);
-          if (partition != null) {
-            currentStateOutput.setPendingState(resourceId, partitionId, participantId,
-                message.getTypedToState());
-          } else {
-            // log
-          }
-        } else {
-          List<PartitionId> partitionNames = message.getPartitionIds();
-          if (!partitionNames.isEmpty()) {
-            for (PartitionId partitionId : partitionNames) {
-              Partition partition = resource.getSubUnit(partitionId);
-              if (partition != null) {
-                currentStateOutput.setPendingState(resourceId, partitionId, participantId,
-                    message.getTypedToState());
-              } else {
-                // log
-              }
-            }
-          }
-        }
-      }
-
-      // add current state
-      SessionId sessionId = liveParticipant.getRunningInstance().getSessionId();
-      Map<ResourceId, CurrentState> curStateMap = liveParticipant.getCurrentStateMap();
-      for (CurrentState curState : curStateMap.values()) {
-        if (!sessionId.equals(curState.getTypedSessionId())) {
-          continue;
-        }
-
-        ResourceId resourceId = curState.getResourceId();
-        StateModelDefId stateModelDefId = curState.getStateModelDefId();
-        ResourceConfig resource = resourceMap.get(resourceId);
-        if (resource == null) {
-          continue;
-        }
-
-        if (stateModelDefId != null) {
-          currentStateOutput.setResourceStateModelDef(resourceId, stateModelDefId);
-        }
-
-        currentStateOutput.setBucketSize(resourceId, curState.getBucketSize());
-
-        Map<PartitionId, State> partitionStateMap = curState.getTypedPartitionStateMap();
-        for (PartitionId partitionId : partitionStateMap.keySet()) {
-          Partition partition = resource.getSubUnit(partitionId);
-          if (partition != null) {
-            currentStateOutput.setCurrentState(resourceId, partitionId, participantId,
-                curState.getState(partitionId));
-          } else {
-            // log
-          }
-        }
-      }
-    }
-
-    event.addAttribute(AttributeName.CURRENT_STATE.toString(), currentStateOutput);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index d67931d..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
+++ /dev/null
@@ -1,281 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.helix.HelixDataAccessor;
-import org.apache.helix.HelixDefinedState;
-import org.apache.helix.HelixManager;
-import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
-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.State;
-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;
-import org.apache.helix.controller.rebalancer.context.RebalancerContext;
-import org.apache.helix.model.ExternalView;
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.StatusUpdate;
-import org.apache.log4j.Logger;
-
-public class NewExternalViewComputeStage extends AbstractBaseStage {
-  private static Logger LOG = Logger.getLogger(NewExternalViewComputeStage.class);
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    long startTime = System.currentTimeMillis();
-    LOG.info("START ExternalViewComputeStage.process()");
-
-    HelixManager manager = event.getAttribute("helixmanager");
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-
-    if (manager == null || resourceMap == null || cluster == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires ClusterManager|RESOURCES|DataCache");
-    }
-
-    HelixDataAccessor dataAccessor = manager.getHelixDataAccessor();
-    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
-
-    ResourceCurrentState currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.toString());
-
-    List<ExternalView> newExtViews = new ArrayList<ExternalView>();
-    List<PropertyKey> keys = new ArrayList<PropertyKey>();
-
-    // TODO use external-view accessor
-    Map<String, ExternalView> curExtViews =
-        dataAccessor.getChildValuesMap(keyBuilder.externalViews());
-
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      ExternalView view = new ExternalView(resourceId.stringify());
-      // view.setBucketSize(currentStateOutput.getBucketSize(resourceName));
-      // if resource ideal state has bucket size, set it
-      // otherwise resource has been dropped, use bucket size from current state instead
-      ResourceConfig resource = resourceMap.get(resourceId);
-      RebalancerConfig rebalancerConfig = resource.getRebalancerConfig();
-      SchedulerTaskConfig schedulerTaskConfig = resource.getSchedulerTaskConfig();
-
-      if (resource.getBucketSize() > 0) {
-        view.setBucketSize(resource.getBucketSize());
-      } else {
-        view.setBucketSize(currentStateOutput.getBucketSize(resourceId));
-      }
-      for (PartitionId partitionId : resource.getSubUnitMap().keySet()) {
-        Map<ParticipantId, State> currentStateMap =
-            currentStateOutput.getCurrentStateMap(resourceId, partitionId);
-        if (currentStateMap != null && currentStateMap.size() > 0) {
-          // Set<String> disabledInstances
-          // = cache.getDisabledInstancesForResource(resource.toString());
-          for (ParticipantId participantId : currentStateMap.keySet()) {
-            // if (!disabledInstances.contains(instance))
-            // {
-            view.setState(partitionId.stringify(), participantId.stringify(),
-                currentStateMap.get(participantId).toString());
-            // }
-          }
-        }
-      }
-
-      // TODO fix this
-      // Update cluster status monitor mbean
-      // ClusterStatusMonitor clusterStatusMonitor =
-      // (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
-      // IdealState idealState = cache._idealStateMap.get(view.getResourceName());
-      // if (idealState != null) {
-      // if (clusterStatusMonitor != null
-      // && !idealState.getStateModelDefRef().equalsIgnoreCase(
-      // DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE)) {
-      // clusterStatusMonitor.onExternalViewChange(view,
-      // cache._idealStateMap.get(view.getResourceName()));
-      // }
-      // }
-
-      // compare the new external view with current one, set only on different
-      ExternalView curExtView = curExtViews.get(resourceId.stringify());
-      if (curExtView == null || !curExtView.getRecord().equals(view.getRecord())) {
-        keys.add(keyBuilder.externalView(resourceId.stringify()));
-        newExtViews.add(view);
-
-        // For SCHEDULER_TASK_RESOURCE resource group (helix task queue), we need to find out which
-        // task
-        // partitions are finished (COMPLETED or ERROR), update the status update of the original
-        // scheduler
-        // message, and then remove the partitions from the ideal state
-        RebalancerContext rebalancerContext =
-            (rebalancerConfig != null) ? rebalancerConfig
-                .getRebalancerContext(RebalancerContext.class) : null;
-        if (rebalancerContext != null
-            && rebalancerContext.getStateModelDefId().equalsIgnoreCase(
-                StateModelDefId.SchedulerTaskQueue)) {
-          updateScheduledTaskStatus(resourceId, view, manager, schedulerTaskConfig);
-        }
-      }
-    }
-    // TODO: consider not setting the externalview of SCHEDULER_TASK_QUEUE at all.
-    // Are there any entity that will be interested in its change?
-
-    // add/update external-views
-    if (newExtViews.size() > 0) {
-      dataAccessor.setChildren(keys, newExtViews);
-    }
-
-    // remove dead external-views
-    for (String resourceName : curExtViews.keySet()) {
-      if (!resourceMap.containsKey(ResourceId.from(resourceName))) {
-        dataAccessor.removeProperty(keyBuilder.externalView(resourceName));
-      }
-    }
-
-    long endTime = System.currentTimeMillis();
-    LOG.info("END ExternalViewComputeStage.process(). took: " + (endTime - startTime) + " ms");
-  }
-
-  // TODO fix it
-  private void updateScheduledTaskStatus(ResourceId resourceId, ExternalView ev,
-      HelixManager manager, SchedulerTaskConfig schedulerTaskConfig) {
-    HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    Builder keyBuilder = accessor.keyBuilder();
-
-    ZNRecord finishedTasks = new ZNRecord(ev.getResourceName());
-
-    // Place holder for finished partitions
-    Map<String, String> emptyMap = new HashMap<String, String>();
-    List<String> emptyList = new LinkedList<String>();
-
-    Map<String, Integer> controllerMsgIdCountMap = new HashMap<String, Integer>();
-    Map<String, Map<String, String>> controllerMsgUpdates =
-        new HashMap<String, Map<String, String>>();
-
-    for (String taskPartitionName : ev.getPartitionSet()) {
-      for (String taskState : ev.getStateMap(taskPartitionName).values()) {
-        if (taskState.equalsIgnoreCase(HelixDefinedState.ERROR.toString())
-            || taskState.equalsIgnoreCase("COMPLETED")) {
-          LOG.info(taskPartitionName + " finished as " + taskState);
-          finishedTasks.setListField(taskPartitionName, emptyList);
-          finishedTasks.setMapField(taskPartitionName, emptyMap);
-
-          // Update original scheduler message status update
-          Message innerMessage =
-              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
-          if (innerMessage != null) {
-            String controllerMsgId = innerMessage.getControllerMessagId();
-            if (controllerMsgId != null) {
-              LOG.info(taskPartitionName + " finished with controllerMsg " + controllerMsgId);
-              if (!controllerMsgUpdates.containsKey(controllerMsgId)) {
-                controllerMsgUpdates.put(controllerMsgId, new HashMap<String, String>());
-              }
-              controllerMsgUpdates.get(controllerMsgId).put(taskPartitionName, taskState);
-            }
-          }
-        }
-      }
-    }
-    // fill the controllerMsgIdCountMap
-    for (PartitionId taskId : schedulerTaskConfig.getPartitionSet()) {
-      Message innerMessage = schedulerTaskConfig.getInnerMessage(taskId);
-      String controllerMsgId = innerMessage.getControllerMessagId();
-
-      if (controllerMsgId != null) {
-        Integer curCnt = controllerMsgIdCountMap.get(controllerMsgId);
-        if (curCnt == null) {
-          curCnt = 0;
-        }
-        controllerMsgIdCountMap.put(controllerMsgId, curCnt + 1);
-      }
-    }
-
-    if (controllerMsgUpdates.size() > 0) {
-      for (String controllerMsgId : controllerMsgUpdates.keySet()) {
-        PropertyKey controllerStatusUpdateKey =
-            keyBuilder.controllerTaskStatus(MessageType.SCHEDULER_MSG.toString(), controllerMsgId);
-        StatusUpdate controllerStatusUpdate = accessor.getProperty(controllerStatusUpdateKey);
-        for (String taskPartitionName : controllerMsgUpdates.get(controllerMsgId).keySet()) {
-          Message innerMessage =
-              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
-
-          Map<String, String> result = new HashMap<String, String>();
-          result.put("Result", controllerMsgUpdates.get(controllerMsgId).get(taskPartitionName));
-          controllerStatusUpdate.getRecord().setMapField(
-              "MessageResult " + innerMessage.getTgtName() + " " + taskPartitionName + " "
-                  + innerMessage.getMessageId(), result);
-        }
-
-        // All done for the scheduled tasks that came from controllerMsgId, add summary for it
-        if (controllerMsgUpdates.get(controllerMsgId).size() == controllerMsgIdCountMap.get(
-            controllerMsgId).intValue()) {
-          int finishedTasksNum = 0;
-          int completedTasksNum = 0;
-          for (String key : controllerStatusUpdate.getRecord().getMapFields().keySet()) {
-            if (key.startsWith("MessageResult ")) {
-              finishedTasksNum++;
-            }
-            if (controllerStatusUpdate.getRecord().getMapField(key).get("Result") != null) {
-              if (controllerStatusUpdate.getRecord().getMapField(key).get("Result")
-                  .equalsIgnoreCase("COMPLETED")) {
-                completedTasksNum++;
-              }
-            }
-          }
-          Map<String, String> summary = new TreeMap<String, String>();
-          summary.put("TotalMessages:", "" + finishedTasksNum);
-          summary.put("CompletedMessages", "" + completedTasksNum);
-
-          controllerStatusUpdate.getRecord().setMapField("Summary", summary);
-        }
-        // Update the statusUpdate of controllerMsgId
-        accessor.updateProperty(controllerStatusUpdateKey, controllerStatusUpdate);
-      }
-    }
-
-    if (finishedTasks.getListFields().size() > 0) {
-      ZNRecordDelta znDelta = new ZNRecordDelta(finishedTasks, MergeOperation.SUBTRACT);
-      List<ZNRecordDelta> deltaList = new LinkedList<ZNRecordDelta>();
-      deltaList.add(znDelta);
-      IdealState delta = new IdealState(resourceId);
-      delta.setDeltaList(deltaList);
-
-      // Remove the finished (COMPLETED or ERROR) tasks from the SCHEDULER_TASK_RESOURCE idealstate
-      keyBuilder = accessor.keyBuilder();
-      accessor.updateProperty(keyBuilder.idealState(resourceId.stringify()), delta);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 3d51bd0..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
+++ /dev/null
@@ -1,213 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.helix.HelixManager;
-import org.apache.helix.api.Cluster;
-import org.apache.helix.api.State;
-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;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Message.MessageState;
-import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.ResourceAssignment;
-import org.apache.helix.model.StateModelDefinition;
-import org.apache.log4j.Logger;
-
-/**
- * Compares the currentState, pendingState with IdealState and generate messages
- */
-public class NewMessageGenerationStage extends AbstractBaseStage {
-  private static Logger LOG = Logger.getLogger(NewMessageGenerationStage.class);
-
-  @Override
-  public void process(ClusterEvent event) throws Exception {
-    HelixManager manager = event.getAttribute("helixmanager");
-    Cluster cluster = event.getAttribute("ClusterDataCache");
-    Map<StateModelDefId, StateModelDefinition> stateModelDefMap = cluster.getStateModelMap();
-    Map<ResourceId, ResourceConfig> resourceMap =
-        event.getAttribute(AttributeName.RESOURCES.toString());
-    ResourceCurrentState currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.toString());
-    NewBestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.toString());
-    if (manager == null || cluster == null || resourceMap == null || currentStateOutput == null
-        || bestPossibleStateOutput == null) {
-      throw new StageException("Missing attributes in event:" + event
-          + ". Requires HelixManager|DataCache|RESOURCES|CURRENT_STATE|BEST_POSSIBLE_STATE");
-    }
-
-    NewMessageOutput output = new NewMessageOutput();
-
-    for (ResourceId resourceId : resourceMap.keySet()) {
-      ResourceConfig resourceConfig = resourceMap.get(resourceId);
-      int bucketSize = resourceConfig.getBucketSize();
-
-      RebalancerContext rebalancerCtx =
-          resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
-      StateModelDefinition stateModelDef = stateModelDefMap.get(rebalancerCtx.getStateModelDefId());
-
-      ResourceAssignment resourceAssignment =
-          bestPossibleStateOutput.getResourceAssignment(resourceId);
-      for (PartitionId subUnitId : resourceConfig.getSubUnitMap().keySet()) {
-        Map<ParticipantId, State> instanceStateMap = resourceAssignment.getReplicaMap(subUnitId);
-
-        // we should generate message based on the desired-state priority
-        // so keep generated messages in a temp map keyed by state
-        // desired-state->list of generated-messages
-        Map<State, List<Message>> messageMap = new HashMap<State, List<Message>>();
-
-        for (ParticipantId participantId : instanceStateMap.keySet()) {
-          State desiredState = instanceStateMap.get(participantId);
-
-          State currentState =
-              currentStateOutput.getCurrentState(resourceId, subUnitId, participantId);
-          if (currentState == null) {
-            currentState = stateModelDef.getTypedInitialState();
-          }
-
-          if (desiredState.equals(currentState)) {
-            continue;
-          }
-
-          State pendingState =
-              currentStateOutput.getPendingState(resourceId, subUnitId, participantId);
-
-          // TODO fix it
-          State nextState = stateModelDef.getNextStateForTransition(currentState, desiredState);
-          if (nextState == null) {
-            LOG.error("Unable to find a next state for partition: " + subUnitId
-                + " from stateModelDefinition" + stateModelDef.getClass() + " from:" + currentState
-                + " to:" + desiredState);
-            continue;
-          }
-
-          if (pendingState != null) {
-            if (nextState.equals(pendingState)) {
-              LOG.debug("Message already exists for " + participantId + " to transit " + subUnitId
-                  + " from " + currentState + " to " + nextState);
-            } else if (currentState.equals(pendingState)) {
-              LOG.info("Message hasn't been removed for " + participantId + " to transit"
-                  + subUnitId + " to " + pendingState + ", desiredState: " + desiredState);
-            } else {
-              LOG.info("IdealState changed before state transition completes for " + subUnitId
-                  + " on " + participantId + ", pendingState: " + pendingState + ", currentState: "
-                  + currentState + ", nextState: " + nextState);
-            }
-          } else {
-            // TODO check if instance is alive
-            SessionId sessionId =
-                cluster.getLiveParticipantMap().get(participantId).getRunningInstance()
-                    .getSessionId();
-            RebalancerContext rebalancerContext =
-                resourceConfig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
-            Message message =
-                createMessage(manager, resourceId, subUnitId, participantId, currentState,
-                    nextState, sessionId, StateModelDefId.from(stateModelDef.getId()),
-                    rebalancerContext.getStateModelFactoryId(), bucketSize);
-
-            // TODO refactor get/set timeout/inner-message
-            if (rebalancerContext != null
-                && rebalancerContext.getStateModelDefId().equalsIgnoreCase(
-                    StateModelDefId.SchedulerTaskQueue)) {
-              if (resourceConfig.getSubUnitMap().size() > 0) {
-                // TODO refactor it -- we need a way to read in scheduler tasks a priori
-                Message innerMsg =
-                    resourceConfig.getSchedulerTaskConfig().getInnerMessage(subUnitId);
-                if (innerMsg != null) {
-                  message.setInnerMessage(innerMsg);
-                }
-              }
-            }
-
-            // Set timeout if needed
-            String stateTransition =
-                String.format("%s-%s_%s", currentState, nextState,
-                    Message.Attributes.TIMEOUT.name());
-            SchedulerTaskConfig schedulerTaskConfig = resourceConfig.getSchedulerTaskConfig();
-            if (schedulerTaskConfig != null) {
-              int timeout = schedulerTaskConfig.getTimeout(stateTransition, subUnitId);
-              if (timeout > 0) {
-                message.setExecutionTimeout(timeout);
-              }
-            }
-            message.setClusterEvent(event);
-
-            if (!messageMap.containsKey(desiredState)) {
-              messageMap.put(desiredState, new ArrayList<Message>());
-            }
-            messageMap.get(desiredState).add(message);
-          }
-        }
-
-        // add generated messages to output according to state priority
-        List<State> statesPriorityList = stateModelDef.getTypedStatesPriorityList();
-        for (State state : statesPriorityList) {
-          if (messageMap.containsKey(state)) {
-            for (Message message : messageMap.get(state)) {
-              output.addMessage(resourceId, subUnitId, message);
-            }
-          }
-        }
-
-      } // end of for-each-partition
-    }
-    event.addAttribute(AttributeName.MESSAGES_ALL.toString(), output);
-    // System.out.println("output: " + output);
-  }
-
-  private Message createMessage(HelixManager manager, ResourceId resourceId,
-      PartitionId partitionId, ParticipantId participantId, State currentState, State nextState,
-      SessionId participantSessionId, StateModelDefId stateModelDefId,
-      StateModelFactoryId stateModelFactoryId, int bucketSize) {
-    MessageId uuid = MessageId.from(UUID.randomUUID().toString());
-    Message message = new Message(MessageType.STATE_TRANSITION, uuid);
-    message.setSrcName(manager.getInstanceName());
-    message.setTgtName(participantId.stringify());
-    message.setMsgState(MessageState.NEW);
-    message.setPartitionId(partitionId);
-    message.setResourceId(resourceId);
-    message.setFromState(currentState);
-    message.setToState(nextState);
-    message.setTgtSessionId(participantSessionId);
-    message.setSrcSessionId(SessionId.from(manager.getSessionId()));
-    message.setStateModelDef(stateModelDefId);
-    message.setStateModelFactoryId(stateModelFactoryId);
-    message.setBucketSize(bucketSize);
-
-    return message;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/9f229c80/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
deleted file mode 100644
index 89231c2..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageOutput.java
+++ /dev/null
@@ -1,79 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.api.id.PartitionId;
-import org.apache.helix.api.id.ResourceId;
-import org.apache.helix.model.Message;
-
-public class NewMessageOutput {
-
-  private final Map<ResourceId, Map<PartitionId, List<Message>>> _messagesMap;
-
-  public NewMessageOutput() {
-    _messagesMap = new HashMap<ResourceId, Map<PartitionId, List<Message>>>();
-
-  }
-
-  public void addMessage(ResourceId resourceId, PartitionId partitionId, Message message) {
-    if (!_messagesMap.containsKey(resourceId)) {
-      _messagesMap.put(resourceId, new HashMap<PartitionId, List<Message>>());
-    }
-    if (!_messagesMap.get(resourceId).containsKey(partitionId)) {
-      _messagesMap.get(resourceId).put(partitionId, new ArrayList<Message>());
-
-    }
-    _messagesMap.get(resourceId).get(partitionId).add(message);
-
-  }
-
-  public void setMessages(ResourceId resourceId, PartitionId partitionId,
-      List<Message> selectedMessages) {
-    if (!_messagesMap.containsKey(resourceId)) {
-      _messagesMap.put(resourceId, new HashMap<PartitionId, List<Message>>());
-    }
-    _messagesMap.get(resourceId).put(partitionId, selectedMessages);
-
-  }
-
-  public List<Message> getMessages(ResourceId resourceId, PartitionId partitionId) {
-    Map<PartitionId, List<Message>> map = _messagesMap.get(resourceId);
-    if (map != null) {
-      return map.get(partitionId);
-    }
-    return Collections.emptyList();
-
-  }
-
-  public Map<PartitionId, List<Message>> getMessages(ResourceId resourceId) {
-    return _messagesMap.get(resourceId);
-  }
-
-  @Override
-  public String toString() {
-    return _messagesMap.toString();
-  }
-}