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:20 UTC

[12/53] [abbrv] [HELIX-234] Convert-all-Id.x-to-x.from-and-assorted-code-cleanup

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 bcd8f4a..b0c3e58 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMessageThrottleStage.java
@@ -31,7 +31,9 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.MessageId;
+import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.ResourceId;
 import org.apache.helix.controller.pipeline.Pipeline;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
@@ -102,11 +104,12 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     NewMessageOutput msgSelectOutput = new NewMessageOutput();
     List<Message> selectMessages = new ArrayList<Message>();
     Message msg =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-001"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_0");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_0");
     selectMessages.add(msg);
 
-    msgSelectOutput.setMessages(Id.resource("TestDB"), Id.partition("TestDB_0"), selectMessages);
+    msgSelectOutput.setMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"),
+        selectMessages);
     event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), msgSelectOutput);
 
     runStage(event, throttleStage);
@@ -114,7 +117,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     NewMessageOutput msgThrottleOutput =
         event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
     Assert.assertEquals(
-        msgThrottleOutput.getMessages(Id.resource("TestDB"), Id.partition("TestDB_0")).size(), 1);
+        msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"))
+            .size(), 1);
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
 
@@ -216,8 +220,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     // test constraintSelection
     // message1: hit contraintSelection rule1 and rule2
     Message msg1 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-001"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_0");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-001"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_0");
 
     Map<ConstraintAttribute, String> msgAttr = ClusterConstraints.toConstraintAttributes(msg1);
     Set<ConstraintItem> matches = constraint.match(msgAttr);
@@ -237,8 +241,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
 
     // message2: hit contraintSelection rule1, rule2, and rule3
     Message msg2 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-002"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_1");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-002"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_1");
 
     msgAttr = ClusterConstraints.toConstraintAttributes(msg2);
     matches = constraint.match(msgAttr);
@@ -267,20 +271,20 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     NewMessageOutput msgSelectOutput = new NewMessageOutput();
 
     Message msg3 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-003"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_0");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-003"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_0");
 
     Message msg4 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-004"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_0");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-004"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_0");
 
     Message msg5 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-005"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_0");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-005"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_0");
 
     Message msg6 =
-        createMessage(MessageType.STATE_TRANSITION, Id.message("msgId-006"), "OFFLINE", "SLAVE",
-            "TestDB", "localhost_1");
+        createMessage(MessageType.STATE_TRANSITION, MessageId.from("msgId-006"), "OFFLINE",
+            "SLAVE", "TestDB", "localhost_1");
 
     List<Message> selectMessages = new ArrayList<Message>();
     selectMessages.add(msg1);
@@ -290,7 +294,8 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     selectMessages.add(msg5); // should be throttled
     selectMessages.add(msg6); // should be throttled
 
-    msgSelectOutput.setMessages(Id.resource("TestDB"), Id.partition("TestDB_0"), selectMessages);
+    msgSelectOutput.setMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"),
+        selectMessages);
     event.addAttribute(AttributeName.MESSAGES_SELECTED.toString(), msgSelectOutput);
 
     runStage(event, throttleStage);
@@ -298,7 +303,7 @@ public class TestMessageThrottleStage extends ZkUnitTestBase {
     NewMessageOutput msgThrottleOutput =
         event.getAttribute(AttributeName.MESSAGES_THROTTLE.toString());
     List<Message> throttleMessages =
-        msgThrottleOutput.getMessages(Id.resource("TestDB"), Id.partition("TestDB_0"));
+        msgThrottleOutput.getMessages(ResourceId.from("TestDB"), PartitionId.from("TestDB_0"));
     Assert.assertEquals(throttleMessages.size(), 4);
     Assert.assertTrue(throttleMessages.contains(msg1));
     Assert.assertTrue(throttleMessages.contains(msg2));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 28b1461..ff2e292 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestMsgSelectionStage.java
@@ -29,13 +29,15 @@ import java.util.Set;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.api.HelixVersion;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
 import org.apache.helix.api.Participant;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.ProcId;
 import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.RunningInstance;
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
 import org.apache.helix.api.UserConfig;
 import org.apache.helix.controller.stages.NewMessageSelectionStage.Bounds;
@@ -55,27 +57,31 @@ public class TestMsgSelectionStage {
     Map<ResourceId, CurrentState> currentStateMap = Collections.emptyMap();
     Map<MessageId, Message> messageMap = Collections.emptyMap();
     RunningInstance runningInstance0 =
-        new RunningInstance(Id.session("session_0"), HelixVersion.from("1.2.3.4"), Id.process("0"));
+        new RunningInstance(SessionId.from("session_0"), HelixVersion.from("1.2.3.4"),
+            ProcId.from("0"));
     RunningInstance runningInstance1 =
-        new RunningInstance(Id.session("session_1"), HelixVersion.from("1.2.3.4"), Id.process("1"));
-    liveInstances.put(Id.participant("localhost_0"), new Participant(Id.participant("localhost_0"),
-        "localhost", 0, true, disabledPartitions, tags, runningInstance0, currentStateMap,
-        messageMap, new UserConfig(Id.participant("localhost_0"))));
-    liveInstances.put(Id.participant("localhost_1"), new Participant(Id.participant("localhost_1"),
-        "localhost", 1, true, disabledPartitions, tags, runningInstance1, currentStateMap,
-        messageMap, new UserConfig(Id.participant("localhost_1"))));
+        new RunningInstance(SessionId.from("session_1"), HelixVersion.from("1.2.3.4"),
+            ProcId.from("1"));
+    liveInstances.put(ParticipantId.from("localhost_0"),
+        new Participant(ParticipantId.from("localhost_0"), "localhost", 0, true,
+            disabledPartitions, tags, runningInstance0, currentStateMap, messageMap,
+            new UserConfig(Scope.participant(ParticipantId.from("localhost_0")))));
+    liveInstances.put(ParticipantId.from("localhost_1"),
+        new Participant(ParticipantId.from("localhost_1"), "localhost", 1, true,
+            disabledPartitions, tags, runningInstance1, currentStateMap, messageMap,
+            new UserConfig(Scope.participant(ParticipantId.from("localhost_1")))));
 
     Map<ParticipantId, State> currentStates = new HashMap<ParticipantId, State>();
-    currentStates.put(Id.participant("localhost_0"), State.from("SLAVE"));
-    currentStates.put(Id.participant("localhost_1"), State.from("MASTER"));
+    currentStates.put(ParticipantId.from("localhost_0"), State.from("SLAVE"));
+    currentStates.put(ParticipantId.from("localhost_1"), State.from("MASTER"));
 
     Map<ParticipantId, State> pendingStates = new HashMap<ParticipantId, State>();
 
     List<Message> messages = new ArrayList<Message>();
-    messages.add(TestHelper.createMessage(Id.message("msgId_0"), "SLAVE", "MASTER", "localhost_0",
-        "TestDB", "TestDB_0"));
-    messages.add(TestHelper.createMessage(Id.message("msgId_1"), "MASTER", "SLAVE", "localhost_1",
-        "TestDB", "TestDB_0"));
+    messages.add(TestHelper.createMessage(MessageId.from("msgId_0"), "SLAVE", "MASTER",
+        "localhost_0", "TestDB", "TestDB_0"));
+    messages.add(TestHelper.createMessage(MessageId.from("msgId_1"), "MASTER", "SLAVE",
+        "localhost_1", "TestDB", "TestDB_0"));
 
     Map<State, Bounds> stateConstraints = new HashMap<State, Bounds>();
     stateConstraints.put(State.from("MASTER"), new Bounds(0, 1));
@@ -90,7 +96,7 @@ public class TestMsgSelectionStage {
             messages, stateConstraints, stateTransitionPriorities, State.from("OFFLINE"));
 
     Assert.assertEquals(selectedMsg.size(), 1);
-    Assert.assertEquals(selectedMsg.get(0).getMsgId(), Id.message("msgId_1"));
+    Assert.assertEquals(selectedMsg.get(0).getMsgId(), MessageId.from("msgId_1"));
     System.out.println("END testMasterXfer at " + new Date(System.currentTimeMillis()));
   }
 
@@ -105,26 +111,30 @@ public class TestMsgSelectionStage {
     Map<ResourceId, CurrentState> currentStateMap = Collections.emptyMap();
     Map<MessageId, Message> messageMap = Collections.emptyMap();
     RunningInstance runningInstance0 =
-        new RunningInstance(Id.session("session_0"), HelixVersion.from("1.2.3.4"), Id.process("0"));
+        new RunningInstance(SessionId.from("session_0"), HelixVersion.from("1.2.3.4"),
+            ProcId.from("0"));
     RunningInstance runningInstance1 =
-        new RunningInstance(Id.session("session_1"), HelixVersion.from("1.2.3.4"), Id.process("1"));
-    liveInstances.put(Id.participant("localhost_0"), new Participant(Id.participant("localhost_0"),
-        "localhost", 0, true, disabledPartitions, tags, runningInstance0, currentStateMap,
-        messageMap, new UserConfig(Id.participant("localhost_0"))));
-    liveInstances.put(Id.participant("localhost_1"), new Participant(Id.participant("localhost_1"),
-        "localhost", 1, true, disabledPartitions, tags, runningInstance1, currentStateMap,
-        messageMap, new UserConfig(Id.participant("localhost_1"))));
+        new RunningInstance(SessionId.from("session_1"), HelixVersion.from("1.2.3.4"),
+            ProcId.from("1"));
+    liveInstances.put(ParticipantId.from("localhost_0"),
+        new Participant(ParticipantId.from("localhost_0"), "localhost", 0, true,
+            disabledPartitions, tags, runningInstance0, currentStateMap, messageMap,
+            new UserConfig(Scope.participant(ParticipantId.from("localhost_0")))));
+    liveInstances.put(ParticipantId.from("localhost_1"),
+        new Participant(ParticipantId.from("localhost_1"), "localhost", 1, true,
+            disabledPartitions, tags, runningInstance1, currentStateMap, messageMap,
+            new UserConfig(Scope.participant(ParticipantId.from("localhost_1")))));
 
     Map<ParticipantId, State> currentStates = new HashMap<ParticipantId, State>();
-    currentStates.put(Id.participant("localhost_0"), State.from("SLAVE"));
-    currentStates.put(Id.participant("localhost_1"), State.from("SLAVE"));
+    currentStates.put(ParticipantId.from("localhost_0"), State.from("SLAVE"));
+    currentStates.put(ParticipantId.from("localhost_1"), State.from("SLAVE"));
 
     Map<ParticipantId, State> pendingStates = new HashMap<ParticipantId, State>();
-    pendingStates.put(Id.participant("localhost_1"), State.from("MASTER"));
+    pendingStates.put(ParticipantId.from("localhost_1"), State.from("MASTER"));
 
     List<Message> messages = new ArrayList<Message>();
-    messages.add(TestHelper.createMessage(Id.message("msgId_0"), "SLAVE", "MASTER", "localhost_0",
-        "TestDB", "TestDB_0"));
+    messages.add(TestHelper.createMessage(MessageId.from("msgId_0"), "SLAVE", "MASTER",
+        "localhost_0", "TestDB", "TestDB_0"));
 
     Map<State, Bounds> stateConstraints = new HashMap<State, Bounds>();
     stateConstraints.put(State.from("MASTER"), new Bounds(0, 1));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
index d191345..ec5d503 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestParseInfoFromAlert.java
@@ -20,7 +20,6 @@ package org.apache.helix.controller.stages;
  */
 
 import org.apache.helix.HelixManager;
-import org.apache.helix.controller.stages.StatsAggregationStage;
 import org.apache.helix.integration.ZkStandAloneCMTestBase;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 44c6510..e5674dc 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestRebalancePipeline.java
@@ -29,7 +29,9 @@ import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.pipeline.Pipeline;
@@ -101,7 +103,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            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");
@@ -117,7 +120,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node1");
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
@@ -248,7 +252,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            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");
@@ -264,7 +269,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 1,
         "Should output only 1 message: OFFLINE->DROPPED for localhost_1");
 
@@ -282,7 +288,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 1,
         "Should output 1 message: OFFLINE->DROPPED for localhost_0");
     message = messages.get(0);
@@ -345,7 +352,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     NewMessageOutput msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     List<Message> messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            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");
@@ -367,7 +375,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     runPipeline(event, rebalancePipeline);
     msgSelOutput = event.getAttribute(AttributeName.MESSAGES_SELECTED.toString());
     messages =
-        msgSelOutput.getMessages(Id.resource(resourceName), Id.partition(resourceName + "_0"));
+        msgSelOutput.getMessages(ResourceId.from(resourceName),
+            PartitionId.from(resourceName + "_0"));
     Assert.assertEquals(messages.size(), 0, "Should NOT output 1 message: SLAVE-MASTER for node0");
 
     System.out.println("END " + clusterName + " at " + new Date(System.currentTimeMillis()));
@@ -381,8 +390,8 @@ public class TestRebalancePipeline extends ZkUnitTestBase {
     Builder keyBuilder = accessor.keyBuilder();
 
     CurrentState curState = new CurrentState(resourceGroupName);
-    curState.setState(Id.partition(resourceKey), State.from(state));
-    curState.setSessionId(Id.session(sessionId));
+    curState.setState(PartitionId.from(resourceKey), State.from(state));
+    curState.setSessionId(SessionId.from(sessionId));
     curState.setStateModelDefRef("MasterSlave");
     accessor.setProperty(keyBuilder.currentState(instance, sessionId, resourceGroupName), curState);
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 de7a6b8..b9cba1f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/TestResourceComputationStage.java
@@ -27,7 +27,7 @@ import java.util.UUID;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceConfig;
 import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.State;
@@ -73,9 +73,9 @@ public class TestResourceComputationStage extends BaseStageTest {
         event.getAttribute(AttributeName.RESOURCES.toString());
     AssertJUnit.assertEquals(1, resource.size());
 
-    AssertJUnit.assertEquals(resource.keySet().iterator().next(), Id.resource(resourceName));
-    AssertJUnit
-        .assertEquals(resource.values().iterator().next().getId(), Id.resource(resourceName));
+    AssertJUnit.assertEquals(resource.keySet().iterator().next(), ResourceId.from(resourceName));
+    AssertJUnit.assertEquals(resource.values().iterator().next().getId(),
+        ResourceId.from(resourceName));
     AssertJUnit.assertEquals(resource.values().iterator().next().getRebalancerConfig()
         .getStateModelDefId(), idealState.getStateModelDefId());
     AssertJUnit.assertEquals(resource.values().iterator().next().getPartitionSet().size(),
@@ -99,7 +99,7 @@ public class TestResourceComputationStage extends BaseStageTest {
 
     for (int i = 0; i < resources.length; i++) {
       String resourceName = resources[i];
-      ResourceId resourceId = Id.resource(resourceName);
+      ResourceId resourceId = ResourceId.from(resourceName);
       IdealState idealState = idealStates.get(i);
       AssertJUnit.assertTrue(resourceMap.containsKey(resourceId));
       AssertJUnit.assertEquals(resourceMap.get(resourceId).getId(), resourceId);
@@ -155,9 +155,9 @@ public class TestResourceComputationStage extends BaseStageTest {
 
     String oldResource = "testResourceOld";
     CurrentState currentState = new CurrentState(oldResource);
-    currentState.setState(Id.partition("testResourceOld_0"), State.from("OFFLINE"));
-    currentState.setState(Id.partition("testResourceOld_1"), State.from("SLAVE"));
-    currentState.setState(Id.partition("testResourceOld_2"), State.from("MASTER"));
+    currentState.setState(PartitionId.from("testResourceOld_0"), State.from("OFFLINE"));
+    currentState.setState(PartitionId.from("testResourceOld_1"), State.from("SLAVE"));
+    currentState.setState(PartitionId.from("testResourceOld_2"), State.from("MASTER"));
     currentState.setStateModelDefRef("MasterSlave");
     accessor.setProperty(keyBuilder.currentState(instanceName, sessionId, oldResource),
         currentState);
@@ -173,7 +173,7 @@ public class TestResourceComputationStage extends BaseStageTest {
 
     for (int i = 0; i < resources.length; i++) {
       String resourceName = resources[i];
-      ResourceId resourceId = Id.resource(resourceName);
+      ResourceId resourceId = ResourceId.from(resourceName);
       IdealState idealState = idealStates.get(i);
       AssertJUnit.assertTrue(resourceMap.containsKey(resourceId));
       AssertJUnit.assertEquals(resourceMap.get(resourceId).getId(), resourceId);
@@ -183,7 +183,7 @@ public class TestResourceComputationStage extends BaseStageTest {
           idealState.getNumPartitions());
     }
     // Test the data derived from CurrentState
-    ResourceId oldResourceId = Id.resource(oldResource);
+    ResourceId oldResourceId = ResourceId.from(oldResource);
     AssertJUnit.assertTrue(resourceMap.containsKey(oldResourceId));
     AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getId(), oldResourceId);
     AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getRebalancerConfig()
@@ -191,11 +191,11 @@ public class TestResourceComputationStage extends BaseStageTest {
     AssertJUnit.assertEquals(resourceMap.get(oldResourceId).getPartitionSet().size(), currentState
         .getPartitionStateMap().size());
     AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getPartition(
-        Id.partition("testResourceOld_0")));
+        PartitionId.from("testResourceOld_0")));
     AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getPartition(
-        Id.partition("testResourceOld_1")));
+        PartitionId.from("testResourceOld_1")));
     AssertJUnit.assertNotNull(resourceMap.get(oldResourceId).getPartition(
-        Id.partition("testResourceOld_2")));
+        PartitionId.from("testResourceOld_2")));
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 bd6cd13..3c1ee13 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
@@ -37,8 +37,8 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.Mocks.MockAccessor;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.State;
+import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
@@ -125,7 +125,7 @@ public class TestAutoRebalanceStrategy {
   private StateModelDefinition getIncompleteStateModelDef(String modelName, String initialState,
       LinkedHashMap<String, Integer> states) {
     StateModelDefinition.Builder builder =
-        new StateModelDefinition.Builder(Id.stateModelDef(modelName));
+        new StateModelDefinition.Builder(StateModelDefId.from(modelName));
     builder.initialState(State.from(initialState));
     int i = 0;
     for (String state : states.keySet()) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 6db3e6c..02e7e29 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/strategy/TestNewAutoRebalanceStrategy.java
@@ -35,13 +35,14 @@ import java.util.TreeSet;
 
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
 import org.apache.helix.api.Participant;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.Scope;
 import org.apache.helix.api.State;
+import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.api.UserConfig;
 import org.apache.helix.controller.rebalancer.util.NewConstraintBasedAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
@@ -133,7 +134,7 @@ public class TestNewAutoRebalanceStrategy {
   private StateModelDefinition getIncompleteStateModelDef(String modelName, String initialState,
       LinkedHashMap<String, Integer> states) {
     StateModelDefinition.Builder builder =
-        new StateModelDefinition.Builder(Id.stateModelDef(modelName));
+        new StateModelDefinition.Builder(StateModelDefId.from(modelName));
     builder.initialState(State.from(initialState));
     int i = 0;
     for (String state : states.keySet()) {
@@ -220,7 +221,7 @@ public class TestNewAutoRebalanceStrategy {
       final Map<PartitionId, Map<ParticipantId, State>> mapResult =
           new HashMap<PartitionId, Map<ParticipantId, State>>();
       for (String partition : _partitions) {
-        PartitionId partitionId = Id.partition(partition);
+        PartitionId partitionId = PartitionId.from(partition);
         Set<ParticipantId> disabledParticipantsForPartition = Collections.emptySet();
         Set<PartitionId> disabledPartitionIdSet = Collections.emptySet();
         Set<String> tags = Collections.emptySet();
@@ -230,17 +231,18 @@ public class TestNewAutoRebalanceStrategy {
             new HashMap<ParticipantId, Participant>();
         // set up some participants
         for (String nodeName : _liveNodes) {
-          ParticipantId participantId = Id.participant(nodeName);
+          ParticipantId participantId = ParticipantId.from(nodeName);
           Participant participant =
               new Participant(participantId, "hostname", 0, true, disabledPartitionIdSet, tags,
-                  null, currentStateMap, messageMap, new UserConfig(participantId));
+                  null, currentStateMap, messageMap, new UserConfig(
+                      Scope.participant(participantId)));
           liveParticipantMap.put(participantId, participant);
         }
         List<ParticipantId> participantPreferenceList =
             Lists.transform(listResult.get(partition), new Function<String, ParticipantId>() {
               @Override
               public ParticipantId apply(String participantId) {
-                return Id.participant(participantId);
+                return ParticipantId.from(participantId);
               }
             });
         // compute the mapping

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 69c2df6..a29d604 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
@@ -26,15 +26,12 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
-import org.apache.helix.api.PartitionId;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
index 7f004d3..f1d2ba6 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertActionTriggering.java
@@ -24,13 +24,10 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.helix.ConfigAccessor;
-import org.apache.helix.model.ConfigScope;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.HealthStat;
@@ -38,6 +35,7 @@ import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.InstanceConfig.InstanceConfigProperty;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
index 125f61f..24595d0 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestAlertFireHistory.java
@@ -24,19 +24,17 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.helix.model.ConfigScope;
-import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.integration.ZkStandAloneCMTestBaseWithPropertyServerCheck;
 import org.apache.helix.model.AlertHistory;
 import org.apache.helix.model.HealthStat;
 import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
index c5f373c..efbd3b4 100644
--- a/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
+++ b/helix-core/src/test/java/org/apache/helix/healthcheck/TestDummyAlerts.java
@@ -25,9 +25,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 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/fb96a138/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 6b545ed..c181e2c 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
@@ -26,16 +26,13 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
-import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 d10ed53..003232f 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
@@ -26,15 +26,13 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 bd0e3ff..bca492e 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
@@ -25,15 +25,13 @@ import java.util.Map;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 fdf39a6..9d7ac9a 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
@@ -26,15 +26,13 @@ import java.util.Set;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 67049c9..b4ed5d2 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
@@ -41,15 +41,13 @@ import javax.management.ReflectionException;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertValueAndStatus;
 import org.apache.helix.api.State;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.healthcheck.HealthStatsAggregationTask;
-import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.integration.ZkIntegrationTestBase;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZNRecordSerializer;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
index 8547666..82e45cc 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddClusterV2.java
@@ -25,11 +25,9 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.model.PauseSignal;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
index b135d92..d1014ed 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddNodeAfterControllerStart.java
@@ -21,7 +21,6 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyPathConfig;
@@ -31,7 +30,6 @@ import org.apache.helix.ZkHelixTestManager;
 import org.apache.helix.ZkTestHelper;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.CallbackHandler;
-import org.apache.helix.manager.zk.ZKHelixManager;
 import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
index 0592883..f38f3b4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestAddStateModelFactoryAfterConnect.java
@@ -19,20 +19,17 @@ package org.apache.helix.integration;
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockMSModelFactory;
-import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.ExternalView;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.tools.ClusterSetup;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 3497290..2a0c645 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
@@ -25,10 +25,10 @@ import java.util.Map;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.TestHelper.StartCMResult;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
index 5c2c68d..833f85e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBatchMessage.java
@@ -26,18 +26,16 @@ import java.util.Map;
 import java.util.Set;
 
 import org.I0Itec.zkclient.IZkChildListener;
-import org.apache.helix.TestHelper;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.HelixProperty.HelixPropertyAttribute;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.ZkTestHelper;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
-import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.tools.ClusterSetup;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
index 155af87..b568801 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestBucketizedResource.java
@@ -21,9 +21,9 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.controller.HelixControllerMain;
 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/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
index c63e03a..a96dff5 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDisable.java
@@ -23,10 +23,10 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.ZkTestHelper;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.controller.ClusterController;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
index 29627a2..3155dd4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedCMMain.java
@@ -21,9 +21,9 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.controller.HelixControllerMain;
 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/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
index ed60971..672df10 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestDistributedClusterController.java
@@ -21,9 +21,9 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.controller.HelixControllerMain;
 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/fb96a138/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 faf8504..7f9d461 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
@@ -25,27 +25,21 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.I0Itec.zkclient.DataUpdater;
-import org.I0Itec.zkclient.IZkChildListener;
-import org.I0Itec.zkclient.IZkDataListener;
-import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.builder.CustomModeISBuilder;
-import org.apache.helix.model.builder.IdealStateBuilder;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
-import org.apache.zookeeper.data.Stat;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 1b3b317..a3dea87 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestEnablePartitionDuringDisable.java
@@ -24,7 +24,6 @@ import java.util.Date;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.TestHelper;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
 import org.apache.helix.mock.controller.ClusterController;
@@ -55,7 +54,7 @@ public class TestEnablePartitionDuringDisable extends ZkIntegrationTestBase {
       PartitionId partitionId = message.getPartitionId();
       State fromState = message.getFromState();
       State toState = message.getToState();
-      if (instance.equals("localhost_12919") && partitionId.equals(Id.partition("TestDB0_0"))) {
+      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/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
index bebefd1..7e01c89 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestErrorPartition.java
@@ -28,8 +28,8 @@ import java.util.Set;
 import org.apache.helix.TestHelper;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;
 import org.testng.annotations.Test;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
index 26da639..9f4c596 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestExternalViewUpdates.java
@@ -23,9 +23,9 @@ import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.participant.MockParticipant;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java b/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java
index 464d827..962cecf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestFailOverPerf1kp.java
@@ -25,7 +25,6 @@ import org.apache.helix.ExternalCommand;
 import org.apache.helix.ScriptTestHelper;
 import org.apache.helix.TestHelper;
 import org.testng.Assert;
-import org.testng.annotations.Test;
 
 public class TestFailOverPerf1kp {
   // TODO: renable this test. disable it because the script is not running properly on apache

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
index 9c0def6..b24f511 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestHelixCustomCodeRunner.java
@@ -21,12 +21,12 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 
+import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.TestHelper;
-import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey.Builder;
+import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
index 837d0a1..fa4d5a8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestInvalidAutoIdealState.java
@@ -25,9 +25,9 @@ import java.util.Map;
 
 import org.apache.helix.HelixAdmin;
 import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZkUnitTestBase;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.mock.controller.ClusterController;
 import org.apache.helix.mock.participant.MockParticipant;
@@ -37,8 +37,8 @@ import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.tools.ClusterStateVerifier;
-import org.apache.helix.tools.StateModelConfigGenerator;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;
+import org.apache.helix.tools.StateModelConfigGenerator;
 import org.testng.Assert;
 
 // Helix-50: integration test for generate message based on state priority

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 2119642..e09b7e8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagePartitionStateMismatch.java
@@ -26,9 +26,12 @@ import java.util.UUID;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
+import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
+import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
@@ -53,32 +56,32 @@ public class TestMessagePartitionStateMismatch extends ZkStandAloneCMTestBase {
       String sessionid = liveinstanceMap.get(instanceName).getSessionId().stringify();
       for (String partition : ev.getPartitionStringSet()) {
         if (ev.getStateMap(partition).containsKey(instanceName)) {
-          MessageId uuid = Id.message(UUID.randomUUID().toString());
+          MessageId uuid = MessageId.from(UUID.randomUUID().toString());
           Message message = new Message(MessageType.STATE_TRANSITION, uuid);
           boolean rand = new Random().nextInt(10) > 5;
           if (ev.getStateMap(partition).get(instanceName).equals("MASTER")) {
             message.setSrcName(manager.getInstanceName());
             message.setTgtName(instanceName);
             message.setMsgState(MessageState.NEW);
-            message.setPartitionId(Id.partition(partition));
-            message.setResourceId(Id.resource(TEST_DB));
+            message.setPartitionId(PartitionId.from(partition));
+            message.setResourceId(ResourceId.from(TEST_DB));
             message.setFromState(State.from(rand ? "SLAVE" : "OFFLINE"));
             message.setToState(State.from(rand ? "MASTER" : "SLAVE"));
-            message.setTgtSessionId(Id.session(sessionid));
-            message.setSrcSessionId(Id.session(manager.getSessionId()));
-            message.setStateModelDef(Id.stateModelDef("MasterSlave"));
+            message.setTgtSessionId(SessionId.from(sessionid));
+            message.setSrcSessionId(SessionId.from(manager.getSessionId()));
+            message.setStateModelDef(StateModelDefId.from("MasterSlave"));
             message.setStateModelFactoryName("DEFAULT");
           } else if (ev.getStateMap(partition).get(instanceName).equals("SLAVE")) {
             message.setSrcName(manager.getInstanceName());
             message.setTgtName(instanceName);
             message.setMsgState(MessageState.NEW);
-            message.setPartitionId(Id.partition(partition));
-            message.setResourceId(Id.resource(TEST_DB));
+            message.setPartitionId(PartitionId.from(partition));
+            message.setResourceId(ResourceId.from(TEST_DB));
             message.setFromState(State.from(rand ? "MASTER" : "OFFLINE"));
             message.setToState(State.from(rand ? "SLAVE" : "SLAVE"));
-            message.setTgtSessionId(Id.session(sessionid));
-            message.setSrcSessionId(Id.session(manager.getSessionId()));
-            message.setStateModelDef(Id.stateModelDef("MasterSlave"));
+            message.setTgtSessionId(SessionId.from(sessionid));
+            message.setSrcSessionId(SessionId.from(manager.getSessionId()));
+            message.setStateModelDef(StateModelDefId.from("MasterSlave"));
             message.setStateModelFactoryName("DEFAULT");
           }
           accessor.setProperty(

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
index fbff4e1..8cbe55b 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessageThrottle.java
@@ -21,8 +21,6 @@ package org.apache.helix.integration;
 
 import java.util.Date;
 import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.I0Itec.zkclient.IZkChildListener;
@@ -31,17 +29,13 @@ import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.TestHelper;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.mock.participant.MockParticipant;
-import org.apache.helix.model.ClusterConstraints;
-import org.apache.helix.model.ConstraintItem;
-import org.apache.helix.model.Message;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
-import org.apache.helix.model.builder.ClusterConstraintsBuilder;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.builder.ConstraintItemBuilder;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.apache.helix.tools.ClusterStateVerifier.BestPossAndExtViewZkVerifier;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/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 4c60768..7809fcb 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestMessagingService.java
@@ -26,8 +26,8 @@ import org.apache.helix.Criteria;
 import org.apache.helix.Criteria.DataSource;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;
 import org.apache.helix.messaging.handling.MessageHandler;
@@ -92,11 +92,11 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
         factory.getMessageType(), factory);
 
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -187,12 +187,12 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     _startCMResultMap.get(hostSrc)._manager.getMessagingService().registerMessageHandlerFactory(
         factory.getMessageType(), factory);
 
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
 
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -251,12 +251,12 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
     _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
         factory.getMessageType(), factory);
 
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(factory.getMessageType(), msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
 
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -294,12 +294,12 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
       _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
     }
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
 
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -363,12 +363,12 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
       _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
     }
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(new TestMessagingHandlerFactory().getMessageType(), msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
 
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -399,12 +399,12 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
       _startCMResultMap.get(hostDest)._manager.getMessagingService().registerMessageHandlerFactory(
           factory.getMessageType(), factory);
     }
-    MessageId msgId = Id.message(new UUID(123, 456).toString());
+    MessageId msgId = MessageId.from(new UUID(123, 456).toString());
     Message msg = new Message(MessageType.CONTROLLER_MSG, msgId);
     msg.setMsgId(msgId);
     msg.setSrcName(hostSrc);
 
-    msg.setTgtSessionId(Id.session("*"));
+    msg.setTgtSessionId(SessionId.from("*"));
     msg.setMsgState(MessageState.NEW);
     String para = "Testing messaging para";
     msg.getRecord().setSimpleField("TestMessagingPara", para);
@@ -424,7 +424,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
         .indexOf(hostSrc) != -1);
     AssertJUnit.assertTrue(callback1.getMessageReplied().size() == 1);
 
-    msgId = Id.message(UUID.randomUUID().toString());
+    msgId = MessageId.from(UUID.randomUUID().toString());
     msg.setMsgId(msgId);
     cr.setPartition("TestDB_17");
     AsyncCallback callback2 = new MockAsyncCallback();
@@ -437,7 +437,7 @@ public class TestMessagingService extends ZkStandAloneCMTestBaseWithPropertyServ
 
     AssertJUnit.assertTrue(callback2.getMessageReplied().size() == 1);
 
-    msgId = Id.message(UUID.randomUUID().toString());
+    msgId = MessageId.from(UUID.randomUUID().toString());
     msg.setMsgId(msgId);
     cr.setPartitionState("SLAVE");
     AsyncCallback callback3 = new MockAsyncCallback();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
index 5a6e5e6..c5558ca 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestNonOfflineInitState.java
@@ -24,8 +24,8 @@ import java.util.Date;
 import org.apache.helix.TestHelper;
 import org.apache.helix.controller.HelixControllerMain;
 import org.apache.helix.manager.zk.ZkClient;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.MockBootstrapModelFactory;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
index f26377c..aded9b3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestParticipantErrorMessage.java
@@ -24,7 +24,10 @@ import java.util.UUID;
 import org.apache.helix.Criteria;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.MessageId;
+import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory;
 import org.apache.helix.manager.zk.DefaultParticipantErrorMessageHandlerFactory.ActionOnError;
 import org.apache.helix.model.ExternalView;
@@ -42,8 +45,9 @@ public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase {
     String participant2 = "localhost_" + (START_PORT + 1);
 
     Message errorMessage1 =
-        new Message(MessageType.PARTICIPANT_ERROR_REPORT, Id.message(UUID.randomUUID().toString()));
-    errorMessage1.setTgtSessionId(Id.session("*"));
+        new Message(MessageType.PARTICIPANT_ERROR_REPORT, MessageId.from(UUID.randomUUID()
+            .toString()));
+    errorMessage1.setTgtSessionId(SessionId.from("*"));
     errorMessage1.getRecord().setSimpleField(
         DefaultParticipantErrorMessageHandlerFactory.ACTIONKEY,
         ActionOnError.DISABLE_INSTANCE.toString());
@@ -54,10 +58,11 @@ public class TestParticipantErrorMessage extends ZkStandAloneCMTestBase {
         errorMessage1);
 
     Message errorMessage2 =
-        new Message(MessageType.PARTICIPANT_ERROR_REPORT, Id.message(UUID.randomUUID().toString()));
-    errorMessage2.setTgtSessionId(Id.session("*"));
-    errorMessage2.setResourceId(Id.resource("TestDB"));
-    errorMessage2.setPartitionId(Id.partition("TestDB_14"));
+        new Message(MessageType.PARTICIPANT_ERROR_REPORT, MessageId.from(UUID.randomUUID()
+            .toString()));
+    errorMessage2.setTgtSessionId(SessionId.from("*"));
+    errorMessage2.setResourceId(ResourceId.from("TestDB"));
+    errorMessage2.setPartitionId(PartitionId.from("TestDB_14"));
     errorMessage2.getRecord().setSimpleField(
         DefaultParticipantErrorMessageHandlerFactory.ACTIONKEY,
         ActionOnError.DISABLE_PARTITION.toString());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
index 2715932..72920a8 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetInstance.java
@@ -26,8 +26,8 @@ import java.util.Set;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
index de4ad1a..f8b4dc9 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestResetResource.java
@@ -26,8 +26,8 @@ import java.util.Set;
 
 import org.apache.helix.TestHelper;
 import org.apache.helix.mock.controller.ClusterController;
-import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.mock.participant.ErrTransition;
+import org.apache.helix.mock.participant.MockParticipant;
 import org.apache.helix.tools.ClusterSetup;
 import org.apache.helix.tools.ClusterStateVerifier;
 import org.testng.Assert;