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

[14/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/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
index d50ddb2..e2c8854 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
@@ -26,9 +26,12 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.helix.HelixManager;
-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.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
@@ -191,18 +194,18 @@ public class MessageGenerationPhase extends AbstractBaseStage {
   private Message createMessage(HelixManager manager, String resourceName, String partitionName,
       String instanceName, String currentState, String nextState, String sessionId,
       String stateModelDefName, String stateModelFactoryName, int bucketSize) {
-    MessageId uuid = Id.message(UUID.randomUUID().toString());
+    MessageId uuid = MessageId.from(UUID.randomUUID().toString());
     Message message = new Message(MessageType.STATE_TRANSITION, uuid);
     message.setSrcName(manager.getInstanceName());
     message.setTgtName(instanceName);
     message.setMsgState(MessageState.NEW);
-    message.setPartitionId(Id.partition(partitionName));
-    message.setResourceId(Id.resource(resourceName));
+    message.setPartitionId(PartitionId.from(partitionName));
+    message.setResourceId(ResourceId.from(resourceName));
     message.setFromState(State.from(currentState));
     message.setToState(State.from(nextState));
-    message.setTgtSessionId(Id.session(sessionId));
-    message.setSrcSessionId(Id.session(manager.getSessionId()));
-    message.setStateModelDef(Id.stateModelDef(stateModelDefName));
+    message.setTgtSessionId(SessionId.from(sessionId));
+    message.setSrcSessionId(SessionId.from(manager.getSessionId()));
+    message.setStateModelDef(StateModelDefId.from(stateModelDefName));
     message.setStateModelFactoryName(stateModelFactoryName);
     message.setBucketSize(bucketSize);
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
index 70e0a07..62fbafe 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageThrottleStage.java
@@ -29,13 +29,13 @@ import java.util.Set;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.ClusterConstraints;
-import org.apache.helix.model.Message;
-import org.apache.helix.model.Partition;
-import org.apache.helix.model.Resource;
 import org.apache.helix.model.ClusterConstraints.ConstraintAttribute;
-import org.apache.helix.model.ConstraintItem;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
 import org.apache.helix.model.ClusterConstraints.ConstraintValue;
+import org.apache.helix.model.ConstraintItem;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
 import org.apache.log4j.Logger;
 
 @Deprecated

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
index 8c151e3..95c76e6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewExternalViewComputeStage.java
@@ -35,7 +35,6 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.ZNRecordDelta;
 import org.apache.helix.ZNRecordDelta.MergeOperation;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.RebalancerConfig;
@@ -46,13 +45,11 @@ import org.apache.helix.api.State;
 import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
 import org.apache.helix.model.StatusUpdate;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
 import org.apache.log4j.Logger;
 
 public class NewExternalViewComputeStage extends AbstractBaseStage {
@@ -118,8 +115,8 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
 
       // TODO fix this
       // Update cluster status monitor mbean
-      ClusterStatusMonitor clusterStatusMonitor =
-          (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
+      // ClusterStatusMonitor clusterStatusMonitor =
+      // (ClusterStatusMonitor) event.getAttribute("clusterStatusMonitor");
       // IdealState idealState = cache._idealStateMap.get(view.getResourceName());
       // if (idealState != null) {
       // if (clusterStatusMonitor != null
@@ -158,7 +155,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
 
     // remove dead external-views
     for (String resourceName : curExtViews.keySet()) {
-      if (!resourceMap.containsKey(Id.resource(resourceName))) {
+      if (!resourceMap.containsKey(ResourceId.from(resourceName))) {
         dataAccessor.removeProperty(keyBuilder.externalView(resourceName));
       }
     }
@@ -169,8 +166,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
 
   // TODO fix it
   private void updateScheduledTaskStatus(ResourceId resourceId, ExternalView ev,
-      HelixManager manager,
-      SchedulerTaskConfig schedulerTaskConfig) {
+      HelixManager manager, SchedulerTaskConfig schedulerTaskConfig) {
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
     Builder keyBuilder = accessor.keyBuilder();
 
@@ -194,7 +190,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
 
           // Update original scheduler message status update
           Message innerMessage =
-              schedulerTaskConfig.getInnerMessage(Id.partition(taskPartitionName));
+              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
           if (innerMessage != null) {
             String controllerMsgId = innerMessage.getControllerMessagId();
             if (controllerMsgId != null) {
@@ -229,7 +225,7 @@ public class NewExternalViewComputeStage extends AbstractBaseStage {
         StatusUpdate controllerStatusUpdate = accessor.getProperty(controllerStatusUpdateKey);
         for (String taskPartitionName : controllerMsgUpdates.get(controllerMsgId).keySet()) {
           Message innerMessage =
-              schedulerTaskConfig.getInnerMessage(Id.partition(taskPartitionName));
+              schedulerTaskConfig.getInnerMessage(PartitionId.from(taskPartitionName));
 
           Map<String, String> result = new HashMap<String, String>();
           result.put("Result", controllerMsgUpdates.get(controllerMsgId).get(taskPartitionName));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
index 76362bc..c0bde54 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewMessageGenerationStage.java
@@ -27,12 +27,10 @@ import java.util.UUID;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.RebalancerConfig;
-import org.apache.helix.api.Resource;
 import org.apache.helix.api.ResourceConfig;
 import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.SchedulerTaskConfig;
@@ -42,7 +40,6 @@ import org.apache.helix.api.StateModelDefId;
 import org.apache.helix.api.StateModelFactoryId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
-import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageState;
 import org.apache.helix.model.Message.MessageType;
@@ -136,7 +133,7 @@ public class NewMessageGenerationStage extends AbstractBaseStage {
                     .getSessionId();
             Message message =
                 createMessage(manager, resourceId, partitionId, participantId, currentState,
-                    nextState, sessionId, new StateModelDefId(stateModelDef.getId()),
+                    nextState, sessionId, StateModelDefId.from(stateModelDef.getId()),
                     resourceConfig.getRebalancerConfig().getStateModelFactoryId(), bucketSize);
 
             // TODO refactor get/set timeout/inner-message
@@ -194,7 +191,7 @@ public class NewMessageGenerationStage extends AbstractBaseStage {
       PartitionId partitionId, ParticipantId participantId, State currentState, State nextState,
       SessionId participantSessionId, StateModelDefId stateModelDefId,
       StateModelFactoryId stateModelFactoryId, int bucketSize) {
-    MessageId uuid = Id.message(UUID.randomUUID().toString());
+    MessageId uuid = MessageId.from(UUID.randomUUID().toString());
     Message message = new Message(MessageType.STATE_TRANSITION, uuid);
     message.setSrcName(manager.getInstanceName());
     message.setTgtName(participantId.stringify());
@@ -204,7 +201,7 @@ public class NewMessageGenerationStage extends AbstractBaseStage {
     message.setFromState(currentState);
     message.setToState(nextState);
     message.setTgtSessionId(participantSessionId);
-    message.setSrcSessionId(Id.session(manager.getSessionId()));
+    message.setSrcSessionId(SessionId.from(manager.getSessionId()));
     message.setStateModelDef(stateModelDefId);
     message.setStateModelFactoryId(stateModelFactoryId);
     message.setBucketSize(bucketSize);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
index 4200fba..d10e155 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewReadClusterDataStage.java
@@ -24,7 +24,6 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.api.Cluster;
 import org.apache.helix.api.ClusterAccessor;
 import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.Id;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
@@ -43,7 +42,7 @@ public class NewReadClusterDataStage extends AbstractBaseStage {
       throw new StageException("HelixManager attribute value is null");
     }
     HelixDataAccessor accessor = manager.getHelixDataAccessor();
-    ClusterId clusterId = Id.cluster(manager.getClusterName());
+    ClusterId clusterId = ClusterId.from(manager.getClusterName());
     ClusterAccessor clusterAccessor = new ClusterAccessor(clusterId, accessor);
 
     Cluster cluster = clusterAccessor.readCluster();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
index 62768e0..0cc1ea1 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewResourceComputationStage.java
@@ -24,7 +24,6 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.Participant;
 import org.apache.helix.api.Partition;
 import org.apache.helix.api.PartitionId;
@@ -32,10 +31,10 @@ import org.apache.helix.api.RebalancerConfig;
 import org.apache.helix.api.Resource;
 import org.apache.helix.api.ResourceConfig;
 import org.apache.helix.api.ResourceId;
+import org.apache.helix.api.StateModelFactoryId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.CurrentState;
-import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.log4j.Logger;
 
 /**
@@ -90,12 +89,11 @@ public class NewResourceComputationStage extends AbstractBaseStage {
           if (!rebalancerConfigBuilderMap.containsKey(resourceId)) {
             RebalancerConfig.SimpleBuilder rebalancerConfigBuilder =
                 new RebalancerConfig.SimpleBuilder(resourceId);
-            rebalancerConfigBuilder.stateModelDefId(currentState.getStateModelDefId());
-            rebalancerConfigBuilder.stateModelFactoryId(Id.stateModelFactory(currentState
+            rebalancerConfigBuilder.stateModelDef(currentState.getStateModelDefId());
+            rebalancerConfigBuilder.stateModelFactoryId(StateModelFactoryId.from(currentState
                 .getStateModelFactoryName()));
             rebalancerConfigBuilderMap.put(resourceId, rebalancerConfigBuilder);
           }
-
           ResourceConfig.Builder resourceBuilder = new ResourceConfig.Builder(resourceId);
           resourceBuilder.bucketSize(currentState.getBucketSize());
           resourceBuilder.batchMessageMode(currentState.getBatchMessageMode());
@@ -104,7 +102,8 @@ public class NewResourceComputationStage extends AbstractBaseStage {
 
         // add all partitions in current-state
         if (rebalancerConfigBuilderMap.containsKey(resourceId)) {
-          RebalancerConfig.SimpleBuilder rebalancerConfigBuilder = rebalancerConfigBuilderMap.get(resourceId);
+          RebalancerConfig.SimpleBuilder rebalancerConfigBuilder =
+              rebalancerConfigBuilderMap.get(resourceId);
           for (PartitionId partitionId : currentState.getPartitionStateMap().keySet()) {
             rebalancerConfigBuilder.addPartition(new Partition(partitionId));
           }
@@ -118,7 +117,8 @@ public class NewResourceComputationStage extends AbstractBaseStage {
     for (ResourceId resourceId : resourceBuilderMap.keySet()) {
       ResourceConfig.Builder resourceConfigBuilder = resourceBuilderMap.get(resourceId);
       if (rebalancerConfigBuilderMap.containsKey(resourceId)) {
-        RebalancerConfig.SimpleBuilder rebalancerConfigBuilder = rebalancerConfigBuilderMap.get(resourceId);
+        RebalancerConfig.SimpleBuilder rebalancerConfigBuilder =
+            rebalancerConfigBuilderMap.get(resourceId);
         resourceConfigBuilder.rebalancerConfig(rebalancerConfigBuilder.build());
       }
       resourceMap.put(resourceId, resourceConfigBuilder.build());

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
index b259847..1bbfc15 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/NewTaskAssignmentStage.java
@@ -31,7 +31,6 @@ import org.apache.helix.HelixManagerProperties;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.api.Cluster;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.Participant;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
@@ -96,7 +95,7 @@ public class NewTaskAssignmentStage extends AbstractBaseStage {
       ResourceId resourceId = message.getResourceId();
       ResourceConfig resource = resourceMap.get(resourceId);
 
-      ParticipantId participantId = Id.participant(message.getTgtName());
+      ParticipantId participantId = ParticipantId.from(message.getTgtName());
       Participant liveParticipant = liveParticipantMap.get(participantId);
       String participantVersion = null;
       if (liveParticipant != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
index e531c88..00612dd 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/StatsAggregationStage.java
@@ -29,9 +29,9 @@ import java.util.Map;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.alerts.AlertParser;
 import org.apache.helix.alerts.AlertProcessor;
 import org.apache.helix.alerts.AlertValueAndStatus;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java
index a522352..85741ed 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/AutoRebalanceStrategy.java
@@ -678,7 +678,7 @@ public class AutoRebalanceStrategy {
     @Override
     public int compareTo(Replica that) {
       if (that instanceof Replica) {
-        return this.format.compareTo(((Replica) that).format);
+        return this.format.compareTo(that.format);
       }
       return -1;
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java
index e3972ff..2314ded 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/strategy/RUSHrHash.java
@@ -19,7 +19,9 @@ package org.apache.helix.controller.strategy;
  * under the License.
  */
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Random;
 import java.util.zip.CRC32;
 
 public class RUSHrHash {
@@ -299,7 +301,7 @@ public class RUSHrHash {
     for (int i = 0; i < replicas; i++) {
       if (totalDisks != 0) {
         ranInt = ran.nextInt((int) (ranMax + 1));
-        z = ((float) ranInt / ranMax);
+        z = (ranInt / ranMax);
         prob = ((float) disksInCurrentCluster / (float) totalDisks);
         if (z <= prob) {
           found++;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
index 34e449a..c3252e9 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/DecayAggregationType.java
@@ -53,6 +53,6 @@ public class DecayAggregationType implements AggregationType {
     // decays
     double oldDecayCoeff = Math.pow((1 - _decayFactor), minutesOld);
     return String
-        .valueOf((double) (oldDecayCoeff * existingVal + (1 - oldDecayCoeff) * incomingVal));
+        .valueOf((oldDecayCoeff * existingVal + (1 - oldDecayCoeff) * incomingVal));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
index b7502f4..619667c 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/DefaultHealthReportProvider.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.helix.ZNRecord;
 import org.apache.log4j.Logger;
 
 class DefaultHealthReportProvider extends HealthReportProvider {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
index a300a02..9023641 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/ParticipantHealthReportCollectorImpl.java
@@ -24,8 +24,8 @@ import java.util.Map;
 
 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.alerts.StatsHolder;
 import org.apache.helix.model.HealthStat;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
index 3a151a7..6bc33d3 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/PerformanceHealthReportProvider.java
@@ -57,7 +57,7 @@ public class PerformanceHealthReportProvider extends HealthReportProvider {
     Map<String, String> result = new TreeMap<String, String>();
 
     result.put(_testStat, "" + testStat);
-    result.put(_readLatencyStat, "" + readLatencySum / (double) readLatencyCount);
+    result.put(_readLatencyStat, "" + readLatencySum / readLatencyCount);
     result.put(_requestCountStat, "" + requestCount);
 
     return result;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java b/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
index d5a31d4..e675792 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/Stat.java
@@ -19,10 +19,10 @@ package org.apache.helix.healthcheck;
  * under the License.
  */
 
-import org.apache.log4j.Logger;
-
 import java.util.Map;
 
+import org.apache.log4j.Logger;
+
 public class Stat {
 
   private static final Logger _logger = Logger.getLogger(Stat.class);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java b/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
index 04bb943..82b2d31 100644
--- a/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/healthcheck/StatHealthReportProvider.java
@@ -19,13 +19,13 @@ package org.apache.helix.healthcheck;
  * under the License.
  */
 
-import org.apache.log4j.Logger;
-
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.log4j.Logger;
+
 public class StatHealthReportProvider extends HealthReportProvider {
 
   private static final Logger _logger = Logger.getLogger(StatHealthReportProvider.class);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java b/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java
index 8f11de5..bac2b15 100644
--- a/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java
+++ b/helix-core/src/main/java/org/apache/helix/josql/ClusterJosqlQueryProcessor.java
@@ -28,9 +28,9 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.LiveInstance.LiveInstanceProperty;
 import org.apache.log4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
index 4f549e4..f623ca5 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/AbstractManager.java
@@ -20,8 +20,6 @@ package org.apache.helix.manager.zk;
  */
 
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -37,6 +35,7 @@ import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HealthStateChangeListener;
 import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
@@ -50,12 +49,11 @@ import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
 import org.apache.helix.PropertyKey;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.HelixConstants.ChangeType;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.healthcheck.ParticipantHealthReportCollector;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java
index 354a9f6..1935c5f 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/Cache.java
@@ -19,7 +19,6 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.io.File;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
index b59976d..b844926 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
@@ -49,13 +49,11 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
-import org.apache.helix.InstanceType;
 import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.MessageListener;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.ScopedConfigChangeListener;
 import org.apache.helix.ZNRecord;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
index 9f551c1..55ace7a 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManager.java
@@ -19,18 +19,17 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Timer;
 
 import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixTimerTask;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyPathConfig;
 import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.controller.GenericHelixController;
 import org.apache.helix.healthcheck.HealthStatsAggregationTask;
 import org.apache.helix.healthcheck.HealthStatsAggregator;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
index ff3a264..df90f6e 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ControllerManagerHelper.java
@@ -22,7 +22,6 @@ package org.apache.helix.manager.zk;
 import java.util.List;
 
 import org.I0Itec.zkclient.exception.ZkInterruptedException;
-import org.apache.helix.HelixManager;
 import org.apache.helix.HelixTimerTask;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.controller.GenericHelixController;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
index b45ea98..482d3cf 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CurStateCarryOverUpdater.java
@@ -21,8 +21,8 @@ package org.apache.helix.manager.zk;
 
 import org.I0Itec.zkclient.DataUpdater;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.PartitionId;
+import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
 import org.apache.helix.model.CurrentState;
 
@@ -55,7 +55,7 @@ class CurStateCarryOverUpdater implements DataUpdater<ZNRecord> {
       curState = new CurrentState(_lastCurState.getId());
       // copy all simple fields settings and overwrite session-id to current session
       curState.getRecord().setSimpleFields(_lastCurState.getRecord().getSimpleFields());
-      curState.setSessionId(Id.session(_curSessionId));
+      curState.setSessionId(SessionId.from(_curSessionId));
     } else {
       curState = new CurrentState(currentData);
     }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
index cd18401..0a223fb 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/DefaultSchedulerMessageHandlerFactory.java
@@ -20,9 +20,6 @@ package org.apache.helix.manager.zk;
  */
 
 import java.io.StringReader;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -37,10 +34,11 @@ import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
 import org.apache.helix.NotificationContext;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.Id;
+import org.apache.helix.ZNRecord;
 import org.apache.helix.api.MessageId;
+import org.apache.helix.api.ParticipantId;
+import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.State;
 import org.apache.helix.messaging.AsyncCallback;
 import org.apache.helix.messaging.handling.HelixTaskResult;
@@ -48,8 +46,8 @@ import org.apache.helix.messaging.handling.MessageHandler;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.StatusUpdate;
 import org.apache.helix.model.Message.MessageType;
+import org.apache.helix.model.StatusUpdate;
 import org.apache.helix.util.StatusUpdateUtil;
 import org.apache.log4j.Logger;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -198,8 +196,8 @@ public class DefaultSchedulerMessageHandlerFactory implements MessageHandlerFact
             String partitionId = taskQueueName + "_" + existingTopPartitionId;
             existingTopPartitionId++;
             String instanceName = task.getTgtName();
-            newAddedScheduledTasks.setPartitionState(Id.partition(partitionId),
-                Id.participant(instanceName), State.from("COMPLETED"));
+            newAddedScheduledTasks.setPartitionState(PartitionId.from(partitionId),
+                ParticipantId.from(instanceName), State.from("COMPLETED"));
             task.getRecord().setSimpleField(instanceName, "COMPLETED");
             task.getRecord().setSimpleField(CONTROLLER_MSG_ID, controllerMsgId.stringify());
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java
index 8dd9d77..48f0647 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/HelixGroupCommit.java
@@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.I0Itec.zkclient.DataUpdater;
 import org.I0Itec.zkclient.exception.ZkBadVersionException;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
-import org.apache.helix.AccessOption;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.data.Stat;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
index 0af7e77..ab618fe 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ParticipantManager.java
@@ -19,9 +19,7 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.List;
 
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.HelixException;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java
index 4ad0f0f..7c991ae 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/WriteThroughCache.java
@@ -19,7 +19,6 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.io.File;
 import java.util.List;
 
 import org.I0Itec.zkclient.exception.ZkNoNodeException;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
index 89c412c..61b1cd1 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixAdmin.java
@@ -53,8 +53,9 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.alerts.AlertsHolder;
 import org.apache.helix.alerts.StatsHolder;
-import org.apache.helix.api.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;
@@ -378,13 +379,13 @@ public class ZKHelixAdmin implements HelixAdmin {
     List<PropertyKey> messageKeys = new ArrayList<PropertyKey>();
     for (String partitionName : resetPartitionNames) {
       // send ERROR to initialState message
-      MessageId msgId = Id.message(UUID.randomUUID().toString());
+      MessageId msgId = MessageId.from(UUID.randomUUID().toString());
       Message message = new Message(MessageType.STATE_TRANSITION, msgId);
       message.setSrcName(adminName);
       message.setTgtName(instanceName);
       message.setMsgState(MessageState.NEW);
-      message.setPartitionId(Id.partition(partitionName));
-      message.setResourceId(Id.resource(resourceName));
+      message.setPartitionId(PartitionId.from(partitionName));
+      message.setResourceId(ResourceId.from(resourceName));
       message.setTgtSessionId(sessionId);
       message.setStateModelDef(stateModelDef);
       message.setFromState(State.from(HelixDefinedState.ERROR.toString()));

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
index 715d683..7d37b68 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -30,11 +30,11 @@ import java.util.Timer;
 import java.util.concurrent.TimeUnit;
 
 import org.I0Itec.zkclient.ZkConnection;
+import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
 import org.apache.helix.ClusterMessagingService;
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.ConfigChangeListener;
-import org.apache.helix.AccessOption;
 import org.apache.helix.ControllerChangeListener;
 import org.apache.helix.CurrentStateChangeListener;
 import org.apache.helix.ExternalViewChangeListener;
@@ -44,6 +44,7 @@ import org.apache.helix.HelixConstants.ChangeType;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerProperties;
 import org.apache.helix.HelixTimerTask;
 import org.apache.helix.IdealStateChangeListener;
 import org.apache.helix.InstanceConfigChangeListener;
@@ -52,7 +53,6 @@ import org.apache.helix.LiveInstanceChangeListener;
 import org.apache.helix.LiveInstanceInfoProvider;
 import org.apache.helix.MessageListener;
 import org.apache.helix.PreConnectCallback;
-import org.apache.helix.HelixManagerProperties;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.PropertyPathConfig;
@@ -67,16 +67,14 @@ import org.apache.helix.healthcheck.ParticipantHealthReportCollectorImpl;
 import org.apache.helix.healthcheck.ParticipantHealthReportTask;
 import org.apache.helix.messaging.DefaultMessagingService;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
-import org.apache.helix.model.ConfigScope;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.HelixConfigScope.ConfigScopeProperty;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.model.builder.ConfigScopeBuilder;
-import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.monitoring.ZKPathDataDumpTask;
 import org.apache.helix.participant.DistClusterControllerElection;
 import org.apache.helix.participant.HelixStateMachineEngine;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
index 3b0f0fb..0b112cd 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBaseDataAccessor.java
@@ -22,7 +22,6 @@ package org.apache.helix.manager.zk;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
-import java.util.Iterator;
 import java.util.List;
 
 import org.I0Itec.zkclient.DataUpdater;
@@ -34,7 +33,6 @@ import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.DeleteCallbackHandler;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.ExistsCallbackHandler;
@@ -46,7 +44,6 @@ import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.server.DataTree;
 
 public class ZkBaseDataAccessor<T> implements BaseDataAccessor<T> {
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
index 8a00127..a36c49a 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCacheBaseDataAccessor.java
@@ -35,16 +35,12 @@ import org.I0Itec.zkclient.IZkDataListener;
 import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.AccessOption;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.manager.zk.ZkAsyncCallbacks.CreateCallbackHandler;
-import org.apache.helix.manager.zk.ZkBaseDataAccessor.AccessResult;
 import org.apache.helix.manager.zk.ZkBaseDataAccessor.RetCode;
 import org.apache.helix.store.HelixPropertyListener;
 import org.apache.helix.store.HelixPropertyStore;
 import org.apache.helix.store.zk.ZNode;
 import org.apache.helix.util.PathUtils;
 import org.apache.log4j.Logger;
-import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.DataTree;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java
index a02cedf..d1e3af3 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkCallbackCache.java
@@ -19,7 +19,6 @@ package org.apache.helix.manager.zk;
  * under the License.
  */
 
-import java.io.File;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
index 5b3af6d..56580f9 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java
@@ -37,7 +37,6 @@ import org.apache.helix.manager.zk.ZkAsyncCallbacks.GetDataCallbackHandler;
 import org.apache.helix.manager.zk.ZkAsyncCallbacks.SetDataCallbackHandler;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java b/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java
index 13609f1..4147012 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/serializer/JacksonPayloadSerializer.java
@@ -23,7 +23,6 @@ import java.io.ByteArrayInputStream;
 import java.io.StringWriter;
 
 import org.apache.helix.HelixException;
-import org.apache.helix.manager.zk.serializer.PayloadSerializer;
 import org.apache.log4j.Logger;
 import org.codehaus.jackson.map.DeserializationConfig;
 import org.codehaus.jackson.map.ObjectMapper;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
index 45a6c9a..f5afc59 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/DefaultMessagingService.java
@@ -33,8 +33,10 @@ import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.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.messaging.handling.AsyncCallbackService;
 import org.apache.helix.messaging.handling.HelixTaskExecutor;
 import org.apache.helix.messaging.handling.MessageHandlerFactory;
@@ -164,7 +166,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
           }
         }
         for (Map<String, String> map : matchedList) {
-          MessageId id = Id.message(UUID.randomUUID().toString());
+          MessageId id = MessageId.from(UUID.randomUUID().toString());
           Message newMessage = new Message(message.getRecord(), id);
           String srcInstanceName = _manager.getInstanceName();
           String tgtInstanceName = map.get("instanceName");
@@ -175,10 +177,10 @@ public class DefaultMessagingService implements ClusterMessagingService {
           }
           newMessage.setSrcName(srcInstanceName);
           newMessage.setTgtName(tgtInstanceName);
-          newMessage.setResourceId(Id.resource(map.get("resourceName")));
-          newMessage.setPartitionId(Id.partition(map.get("partitionName")));
+          newMessage.setResourceId(ResourceId.from(map.get("resourceName")));
+          newMessage.setPartitionId(PartitionId.from(map.get("partitionName")));
           if (recipientCriteria.isSessionSpecific()) {
-            newMessage.setTgtSessionId(Id.session(sessionIdMap.get(tgtInstanceName)));
+            newMessage.setTgtSessionId(SessionId.from(sessionIdMap.get(tgtInstanceName)));
           }
           messages.add(newMessage);
         }
@@ -190,7 +192,7 @@ public class DefaultMessagingService implements ClusterMessagingService {
 
   private List<Message> generateMessagesForController(Message message) {
     List<Message> messages = new ArrayList<Message>();
-    MessageId id = Id.message(UUID.randomUUID().toString());
+    MessageId id = MessageId.from(UUID.randomUUID().toString());
     Message newMessage = new Message(message.getRecord(), id);
     newMessage.setMsgId(id);
     newMessage.setSrcName(_manager.getInstanceName());
@@ -265,7 +267,8 @@ public class DefaultMessagingService implements ClusterMessagingService {
   public void sendNopMessage() {
     if (_manager.isConnected()) {
       try {
-        Message nopMsg = new Message(MessageType.NO_OP, Id.message(UUID.randomUUID().toString()));
+        Message nopMsg =
+            new Message(MessageType.NO_OP, MessageId.from(UUID.randomUUID().toString()));
         nopMsg.setSrcName(_manager.getInstanceName());
 
         HelixDataAccessor accessor = _manager.getHelixDataAccessor();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
index c218a15..46c595d 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/AsyncCallbackService.java
@@ -19,22 +19,13 @@ package org.apache.helix.messaging.handling;
  * under the License.
  */
 
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.Random;
-import java.util.Timer;
-import java.util.TimerTask;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.messaging.AsyncCallback;
-import org.apache.helix.messaging.handling.MessageHandler.ErrorCode;
-import org.apache.helix.messaging.handling.MessageHandler.ErrorType;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.Message.MessageType;
-import org.apache.helix.participant.HelixStateMachineEngine;
 import org.apache.log4j.Logger;
 
 public class AsyncCallbackService implements MessageHandlerFactory {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
index 1c1951a..ab8fe8a 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/BatchMessageHandler.java
@@ -35,7 +35,7 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.PropertyKey;
-import org.apache.helix.api.Id;
+import org.apache.helix.api.MessageId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.Message;
@@ -68,7 +68,8 @@ public class BatchMessageHandler extends MessageHandler {
     List<PartitionId> partitionKeys = _message.getPartitionIds();
     for (PartitionId partitionKey : partitionKeys) {
       // assign a new message id, put batch-msg-id to parent-id field
-      Message subMsg = new Message(_message.getRecord(), Id.message(UUID.randomUUID().toString()));
+      Message subMsg =
+          new Message(_message.getRecord(), MessageId.from(UUID.randomUUID().toString()));
       subMsg.setPartitionId(partitionKey);
       subMsg.setAttribute(Attributes.PARENT_MSG_ID, _message.getId());
       subMsg.setBatchMessageMode(false);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
index b4de571..6c6cf8e 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixStateTransitionHandler.java
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.helix.HelixAdmin;
-import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixException;
@@ -36,9 +35,9 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.PropertyKey;
+import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecordBucketizer;
 import org.apache.helix.ZNRecordDelta;
-import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecordDelta.MergeOperation;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
index f9f96ac..af58c98 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/HelixTaskExecutor.java
@@ -45,7 +45,6 @@ import org.apache.helix.NotificationContext.MapKey;
 import org.apache.helix.NotificationContext.Type;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.SessionId;
 import org.apache.helix.model.ConfigScope;
@@ -484,7 +483,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
       // update msgState to read
       message.setMsgState(MessageState.READ);
       message.setReadTimeStamp(new Date().getTime());
-      message.setExecuteSessionId(Id.session(changeContext.getManager().getSessionId()));
+      message.setExecuteSessionId(SessionId.from(changeContext.getManager().getSessionId()));
 
       _statusUpdateUtil.logInfo(message, HelixStateMachineEngine.class, "New Message", accessor);
 
@@ -504,7 +503,7 @@ public class HelixTaskExecutor implements MessageListener, TaskExecutor {
           CurrentState metaCurState = new CurrentState(resourceId.stringify());
           metaCurState.setBucketSize(message.getBucketSize());
           metaCurState.setStateModelDefRef(message.getStateModelDef());
-          metaCurState.setSessionId(Id.session(sessionId));
+          metaCurState.setSessionId(SessionId.from(sessionId));
           metaCurState.setBatchMessageMode(message.getBatchMessageMode());
           String ftyName = message.getStateModelFactoryName();
           if (ftyName != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java
index 96784c2..5715571 100644
--- a/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/messaging/handling/MessageHandler.java
@@ -20,8 +20,6 @@ package org.apache.helix.messaging.handling;
  */
 
 import org.apache.helix.NotificationContext;
-import org.apache.helix.messaging.handling.MessageHandler.ErrorCode;
-import org.apache.helix.messaging.handling.MessageHandler.ErrorType;
 import org.apache.helix.model.Message;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
index 5bd1e86..8104bce 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfiguration.java
@@ -3,7 +3,6 @@ package org.apache.helix.model;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.ClusterId;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.UserConfig;
 
 /*
@@ -52,7 +51,7 @@ public class ClusterConfiguration extends HelixProperty {
    */
   public static ClusterConfiguration from(UserConfig userConfig) {
     ClusterConfiguration clusterConfiguration =
-        new ClusterConfiguration(Id.cluster(userConfig.getId()));
+        new ClusterConfiguration(ClusterId.from(userConfig.getId()));
     clusterConfiguration.addNamespacedConfig(userConfig);
     return clusterConfiguration;
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
index 240939e..a828117 100644
--- a/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/CurrentState.java
@@ -26,7 +26,6 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;
 import org.apache.helix.api.SessionId;
@@ -89,7 +88,7 @@ public class CurrentState extends HelixProperty {
    * @return ResourceId
    */
   public ResourceId getResourceId() {
-    return Id.resource(getResourceName());
+    return ResourceId.from(getResourceName());
   }
 
   /**
@@ -117,7 +116,7 @@ public class CurrentState extends HelixProperty {
     for (String partitionName : _record.getMapFields().keySet()) {
       Map<String, String> stateMap = _record.getMapField(partitionName);
       if (stateMap != null) {
-        map.put(Id.partition(partitionName),
+        map.put(PartitionId.from(partitionName),
             State.from(stateMap.get(CurrentStateProperty.CURRENT_STATE.toString())));
       }
     }
@@ -129,7 +128,7 @@ public class CurrentState extends HelixProperty {
    * @return session identifier
    */
   public SessionId getSessionId() {
-    return Id.session(_record.getSimpleField(CurrentStateProperty.SESSION_ID.toString()));
+    return SessionId.from(_record.getSimpleField(CurrentStateProperty.SESSION_ID.toString()));
   }
 
   /**
@@ -193,7 +192,7 @@ public class CurrentState extends HelixProperty {
    * @return an identifier of the state model
    */
   public StateModelDefId getStateModelDefId() {
-    return Id.stateModelDef(getStateModelDefRef());
+    return StateModelDefId.from(getStateModelDefRef());
   }
 
   /**
@@ -279,7 +278,7 @@ public class CurrentState extends HelixProperty {
     }
     Map<PartitionId, State> partitionStateMap = new HashMap<PartitionId, State>();
     for (String partitionId : rawMap.keySet()) {
-      partitionStateMap.put(Id.partition(partitionId), State.from(rawMap.get(partitionId)));
+      partitionStateMap.put(PartitionId.from(partitionId), State.from(rawMap.get(partitionId)));
     }
     return partitionStateMap;
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
index 16dba05..b8f4ee5 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ExternalView.java
@@ -25,7 +25,6 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;
@@ -126,7 +125,7 @@ public class ExternalView extends HelixProperty {
   public Set<PartitionId> getPartitionSet() {
     ImmutableSet.Builder<PartitionId> builder = new ImmutableSet.Builder<PartitionId>();
     for (String partitionName : getPartitionStringSet()) {
-      builder.add(Id.partition(partitionName));
+      builder.add(PartitionId.from(partitionName));
     }
     return builder.build();
   }
@@ -150,7 +149,8 @@ public class ExternalView extends HelixProperty {
     ImmutableMap.Builder<ParticipantId, State> builder =
         new ImmutableMap.Builder<ParticipantId, State>();
     for (String participantName : rawStateMap.keySet()) {
-      builder.put(Id.participant(participantName), State.from(rawStateMap.get(participantName)));
+      builder
+          .put(ParticipantId.from(participantName), State.from(rawStateMap.get(participantName)));
     }
     return builder.build();
   }
@@ -168,7 +168,7 @@ public class ExternalView extends HelixProperty {
    * @return resource id
    */
   public ResourceId getResourceId() {
-    return Id.resource(getResourceName());
+    return ResourceId.from(getResourceName());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/IdealState.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/IdealState.java b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
index b2eccad..a0400f2 100644
--- a/helix-core/src/main/java/org/apache/helix/model/IdealState.java
+++ b/helix-core/src/main/java/org/apache/helix/model/IdealState.java
@@ -32,7 +32,6 @@ import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.RebalancerRef;
@@ -138,7 +137,7 @@ public class IdealState extends HelixProperty {
    * @return the id of the resource
    */
   public ResourceId getResourceId() {
-    return Id.resource(getResourceName());
+    return ResourceId.from(getResourceName());
   }
 
   /**
@@ -297,7 +296,7 @@ public class IdealState extends HelixProperty {
   public Set<PartitionId> getPartitionSet() {
     ImmutableSet.Builder<PartitionId> partitionSetBuilder = new ImmutableSet.Builder<PartitionId>();
     for (String partitionName : getPartitionStringSet()) {
-      partitionSetBuilder.add(Id.partition(partitionName));
+      partitionSetBuilder.add(PartitionId.from(partitionName));
     }
     return partitionSetBuilder.build();
   }
@@ -345,7 +344,8 @@ public class IdealState extends HelixProperty {
         new ImmutableMap.Builder<ParticipantId, State>();
     if (instanceStateMap != null) {
       for (String participantId : instanceStateMap.keySet()) {
-        builder.put(Id.participant(participantId), State.from(instanceStateMap.get(participantId)));
+        builder.put(ParticipantId.from(participantId),
+            State.from(instanceStateMap.get(participantId)));
       }
       return builder.build();
     }
@@ -390,7 +390,7 @@ public class IdealState extends HelixProperty {
   public Set<ParticipantId> getParticipantSet(PartitionId partitionId) {
     ImmutableSet.Builder<ParticipantId> builder = new ImmutableSet.Builder<ParticipantId>();
     for (String participantName : getInstanceSet(partitionId.stringify())) {
-      builder.add(Id.participant(participantName));
+      builder.add(ParticipantId.from(participantName));
     }
     return builder.build();
   }
@@ -442,7 +442,7 @@ public class IdealState extends HelixProperty {
     List<String> preferenceStringList = getPreferenceList(partitionId.stringify());
     if (preferenceStringList != null) {
       for (String participantName : preferenceStringList) {
-        builder.add(Id.participant(participantName));
+        builder.add(ParticipantId.from(participantName));
       }
       return builder.build();
     }
@@ -462,7 +462,7 @@ public class IdealState extends HelixProperty {
    * @return an identifier of the state model
    */
   public StateModelDefId getStateModelDefId() {
-    return Id.stateModelDef(getStateModelDefRef());
+    return StateModelDefId.from(getStateModelDefRef());
   }
 
   /**
@@ -584,7 +584,7 @@ public class IdealState extends HelixProperty {
    * @return state model factory id
    */
   public StateModelFactoryId getStateModelFactoryId() {
-    return Id.stateModelFactory(getStateModelFactoryName());
+    return StateModelFactoryId.from(getStateModelFactoryName());
   }
 
   /**
@@ -743,7 +743,7 @@ public class IdealState extends HelixProperty {
         new Function<String, ParticipantId>() {
           @Override
           public ParticipantId apply(String participantName) {
-            return Id.participant(participantName);
+            return ParticipantId.from(participantName);
           }
         });
   }
@@ -761,7 +761,7 @@ public class IdealState extends HelixProperty {
     Map<PartitionId, List<ParticipantId>> preferenceLists =
         new HashMap<PartitionId, List<ParticipantId>>();
     for (String partitionId : rawPreferenceLists.keySet()) {
-      preferenceLists.put(Id.partition(partitionId),
+      preferenceLists.put(PartitionId.from(partitionId),
           preferenceListFromStringList(rawPreferenceLists.get(partitionId)));
     }
     return preferenceLists;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
index 2dd2c4b..89b043b 100644
--- a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
@@ -27,7 +27,6 @@ import java.util.Set;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.log4j.Logger;
@@ -263,7 +262,7 @@ public class InstanceConfig extends HelixProperty {
    * @return the participant id
    */
   public ParticipantId getParticipantId() {
-    return Id.participant(getInstanceName());
+    return ParticipantId.from(getInstanceName());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
index 8a4b49b..70822ac 100644
--- a/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
+++ b/helix-core/src/main/java/org/apache/helix/model/LiveInstance.java
@@ -22,7 +22,6 @@ package org.apache.helix.model;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.HelixVersion;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.ProcId;
 import org.apache.helix.api.SessionId;
@@ -81,7 +80,7 @@ public class LiveInstance extends HelixProperty {
    * @return session identifier
    */
   public SessionId getSessionId() {
-    return Id.session(_record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString()));
+    return SessionId.from(_record.getSimpleField(LiveInstanceProperty.SESSION_ID.toString()));
   }
 
   /**
@@ -97,7 +96,7 @@ public class LiveInstance extends HelixProperty {
    * @return participant id
    */
   public ParticipantId getParticipantId() {
-    return Id.participant(getInstanceName());
+    return ParticipantId.from(getInstanceName());
   }
 
   /**
@@ -129,7 +128,7 @@ public class LiveInstance extends HelixProperty {
    * @return process identifier
    */
   public ProcId getProcessId() {
-    return Id.process(getLiveInstance());
+    return ProcId.from(getLiveInstance());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/Message.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/Message.java b/helix-core/src/main/java/org/apache/helix/model/Message.java
index 5704b83..29ac173 100644
--- a/helix-core/src/main/java/org/apache/helix/model/Message.java
+++ b/helix-core/src/main/java/org/apache/helix/model/Message.java
@@ -34,7 +34,6 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.MessageId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;
@@ -225,7 +224,7 @@ public class Message extends HelixProperty {
    * @return session identifier
    */
   public SessionId getTgtSessionId() {
-    return Id.session(_record.getSimpleField(Attributes.TGT_SESSION_ID.toString()));
+    return SessionId.from(_record.getSimpleField(Attributes.TGT_SESSION_ID.toString()));
   }
 
   /**
@@ -241,7 +240,7 @@ public class Message extends HelixProperty {
    * @return session identifier
    */
   public SessionId getSrcSessionId() {
-    return Id.session(_record.getSimpleField(Attributes.SRC_SESSION_ID.toString()));
+    return SessionId.from(_record.getSimpleField(Attributes.SRC_SESSION_ID.toString()));
   }
 
   /**
@@ -257,7 +256,7 @@ public class Message extends HelixProperty {
    * @return session identifier
    */
   public SessionId getExecutionSessionId() {
-    return Id.session(_record.getSimpleField(Attributes.EXE_SESSION_ID.toString()));
+    return SessionId.from(_record.getSimpleField(Attributes.EXE_SESSION_ID.toString()));
   }
 
   /**
@@ -341,7 +340,7 @@ public class Message extends HelixProperty {
    * @return message identifier
    */
   public MessageId getMsgId() {
-    return Id.message(_record.getSimpleField(Attributes.MSG_ID.toString()));
+    return MessageId.from(_record.getSimpleField(Attributes.MSG_ID.toString()));
   }
 
   /**
@@ -421,7 +420,7 @@ public class Message extends HelixProperty {
    * @return resource name
    */
   public ResourceId getResourceId() {
-    return Id.resource(_record.getSimpleField(Attributes.RESOURCE_NAME.toString()));
+    return ResourceId.from(_record.getSimpleField(Attributes.RESOURCE_NAME.toString()));
   }
 
   /**
@@ -429,7 +428,7 @@ public class Message extends HelixProperty {
    * @return partition id
    */
   public PartitionId getPartitionId() {
-    return Id.partition(_record.getSimpleField(Attributes.PARTITION_NAME.toString()));
+    return PartitionId.from(_record.getSimpleField(Attributes.PARTITION_NAME.toString()));
   }
 
   /**
@@ -445,7 +444,7 @@ public class Message extends HelixProperty {
    * @return a reference to the state model definition
    */
   public StateModelDefId getStateModelDefId() {
-    return Id.stateModelDef(getStateModelDef());
+    return StateModelDefId.from(getStateModelDef());
   }
 
   /**
@@ -633,10 +632,10 @@ public class Message extends HelixProperty {
           + " does not contain correlation id");
     }
     Message replyMessage =
-        new Message(MessageType.TASK_REPLY, Id.message(UUID.randomUUID().toString()));
+        new Message(MessageType.TASK_REPLY, MessageId.from(UUID.randomUUID().toString()));
     replyMessage.setCorrelationId(srcMessage.getCorrelationId());
     replyMessage.setResultMap(taskResultMap);
-    replyMessage.setTgtSessionId(Id.session("*"));
+    replyMessage.setTgtSessionId(SessionId.from("*"));
     replyMessage.setMsgState(MessageState.NEW);
     replyMessage.setSrcName(instanceName);
     if (srcMessage.getSrcInstanceType() == InstanceType.CONTROLLER) {
@@ -673,7 +672,7 @@ public class Message extends HelixProperty {
     }
     ImmutableList.Builder<PartitionId> builder = new ImmutableList.Builder<PartitionId>();
     for (String partitionName : partitionNames) {
-      builder.add(Id.partition(partitionName));
+      builder.add(PartitionId.from(partitionName));
     }
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
index d5b9627..45a78c0 100644
--- a/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/PartitionConfiguration.java
@@ -2,7 +2,6 @@ package org.apache.helix.model;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.UserConfig;
 
@@ -52,7 +51,7 @@ public class PartitionConfiguration extends HelixProperty {
    */
   public static PartitionConfiguration from(UserConfig userConfig) {
     PartitionConfiguration partitionConfiguration =
-        new PartitionConfiguration(Id.partition(userConfig.getId()));
+        new PartitionConfiguration(PartitionId.from(userConfig.getId()));
     partitionConfiguration.addNamespacedConfig(userConfig);
     return partitionConfiguration;
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
index 2b06c2b..65fa14d 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceAssignment.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.HelixProperty;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.ParticipantId;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.ResourceId;
@@ -68,7 +67,7 @@ public class ResourceAssignment extends HelixProperty {
    * @return resource id
    */
   public ResourceId getResourceId() {
-    return Id.resource(getId());
+    return ResourceId.from(getId());
   }
 
   /**
@@ -78,7 +77,7 @@ public class ResourceAssignment extends HelixProperty {
   public List<PartitionId> getMappedPartitions() {
     ImmutableList.Builder<PartitionId> builder = new ImmutableList.Builder<PartitionId>();
     for (String partitionName : _record.getMapFields().keySet()) {
-      builder.add(Id.partition(partitionName));
+      builder.add(PartitionId.from(partitionName));
     }
     return builder.build();
   }
@@ -104,7 +103,8 @@ public class ResourceAssignment extends HelixProperty {
     ImmutableMap.Builder<ParticipantId, State> builder =
         new ImmutableMap.Builder<ParticipantId, State>();
     for (String participantName : rawReplicaMap.keySet()) {
-      builder.put(Id.participant(participantName), State.from(rawReplicaMap.get(participantName)));
+      builder.put(ParticipantId.from(participantName),
+          State.from(rawReplicaMap.get(participantName)));
     }
     return builder.build();
   }
@@ -134,7 +134,7 @@ public class ResourceAssignment extends HelixProperty {
     }
     Map<ParticipantId, State> replicaMap = new HashMap<ParticipantId, State>();
     for (String participantName : rawMap.keySet()) {
-      replicaMap.put(Id.participant(participantName), State.from(rawMap.get(participantName)));
+      replicaMap.put(ParticipantId.from(participantName), State.from(rawMap.get(participantName)));
     }
     return replicaMap;
   }
@@ -152,7 +152,7 @@ public class ResourceAssignment extends HelixProperty {
     Map<PartitionId, Map<ParticipantId, State>> participantStateMaps =
         new HashMap<PartitionId, Map<ParticipantId, State>>();
     for (String partitionId : rawMaps.keySet()) {
-      participantStateMaps.put(Id.partition(partitionId),
+      participantStateMaps.put(PartitionId.from(partitionId),
           replicaMapFromStringMap(rawMaps.get(partitionId)));
     }
     return participantStateMaps;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
index 307ab0f..00f8472 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceConfiguration.java
@@ -5,7 +5,6 @@ import java.util.List;
 
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
-import org.apache.helix.api.Id;
 import org.apache.helix.api.NamespacedConfig;
 import org.apache.helix.api.PartitionId;
 import org.apache.helix.api.RebalancerConfig;
@@ -55,7 +54,7 @@ public class ResourceConfiguration extends HelixProperty {
    * @return resource id
    */
   public ResourceId getResourceId() {
-    return Id.resource(getId());
+    return ResourceId.from(getId());
   }
 
   /**
@@ -85,7 +84,7 @@ public class ResourceConfiguration extends HelixProperty {
       return Lists.transform(partitionNames, new Function<String, PartitionId>() {
         @Override
         public PartitionId apply(String partitionName) {
-          return Id.partition(partitionName);
+          return PartitionId.from(partitionName);
         }
       });
     }
@@ -108,7 +107,7 @@ public class ResourceConfiguration extends HelixProperty {
    */
   public static ResourceConfiguration from(NamespacedConfig namespacedConfig) {
     ResourceConfiguration resourceConfiguration =
-        new ResourceConfiguration(Id.resource(namespacedConfig.getId()));
+        new ResourceConfiguration(ResourceId.from(namespacedConfig.getId()));
     resourceConfiguration.addNamespacedConfig(namespacedConfig);
     return resourceConfiguration;
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
index 3c4de68..41aa929 100644
--- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
+++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
@@ -30,7 +30,6 @@ import java.util.TreeMap;
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixProperty;
 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.model.builder.StateTransitionTableBuilder;
@@ -147,7 +146,7 @@ public class StateModelDefinition extends HelixProperty {
    * @return StateModelDefId
    */
   public StateModelDefId getStateModelDefId() {
-    return Id.stateModelDef(getId());
+    return StateModelDefId.from(getId());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/fb96a138/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java b/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java
index 535f25b..72b2bc9 100644
--- a/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java
+++ b/helix-core/src/main/java/org/apache/helix/model/builder/AutoModeISBuilder.java
@@ -19,11 +19,11 @@ package org.apache.helix.model.builder;
  * under the License.
  */
 
-import org.apache.helix.model.IdealState.RebalanceMode;
-
 import java.util.ArrayList;
 import java.util.Arrays;
 
+import org.apache.helix.model.IdealState.RebalanceMode;
+
 public class AutoModeISBuilder extends IdealStateBuilder {
   public AutoModeISBuilder(String resourceName) {
     super(resourceName);