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/11/07 02:19:46 UTC

[38/53] [abbrv] [HELIX-209] Backward compatible function naming in the model package

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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/142c9248/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;
     }