You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2013/09/27 01:05:12 UTC

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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
new file mode 100644
index 0000000..c6f9d19
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/NamespacedConfig.java
@@ -0,0 +1,228 @@
+package org.apache.helix.api.config;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.api.Scope;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Generic configuration of Helix components prefixed with a namespace
+ */
+public class NamespacedConfig extends ZNRecord {
+  private static final char PREFIX_CHAR = '!';
+  private final String _prefix;
+
+  /**
+   * Instantiate a NamespacedConfig. It is intended for use only by entities that can be identified
+   * @param scope scope object
+   */
+  public NamespacedConfig(Scope<?> scope, String prefix) {
+    super(scope.getScopedId().stringify());
+    _prefix = prefix + PREFIX_CHAR;
+  }
+
+  /**
+   * Instantiate a NamespacedConfig from an existing HelixProperty
+   * @param property property wrapping a configuration
+   */
+  public NamespacedConfig(HelixProperty property, String prefix) {
+    super(property.getRecord());
+    _prefix = prefix + PREFIX_CHAR;
+    filterNonPrefixedFields();
+  }
+
+  /**
+   * Instantiate a NamespacedConfig as a copy of another NamedspacedConfig
+   * @param config populated NamespacedConfig
+   */
+  public NamespacedConfig(NamespacedConfig config) {
+    super(config.getId());
+    _prefix = config.getPrefix() + PREFIX_CHAR;
+    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
+      setRawPayload(config.getRawPayload());
+      setPayloadSerializer(config.getPayloadSerializer());
+    }
+    super.setSimpleFields(config.getPrefixedSimpleFields());
+    super.setListFields(config.getPrefixedListFields());
+    super.setMapFields(config.getPrefixedMapFields());
+  }
+
+  @Override
+  public void setMapField(String k, Map<String, String> v) {
+    super.setMapField(_prefix + k, v);
+  }
+
+  @Override
+  public Map<String, String> getMapField(String k) {
+    return super.getMapField(_prefix + k);
+  }
+
+  @Override
+  public void setMapFields(Map<String, Map<String, String>> mapFields) {
+    for (String k : mapFields.keySet()) {
+      super.setMapField(_prefix + k, mapFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of map fields
+   */
+  @Override
+  public Map<String, Map<String, String>> getMapFields() {
+    return convertToPrefixlessMap(super.getMapFields(), _prefix);
+  }
+
+  @Override
+  public void setListField(String k, List<String> v) {
+    super.setListField(_prefix + k, v);
+  }
+
+  @Override
+  public List<String> getListField(String k) {
+    return super.getListField(_prefix + k);
+  }
+
+  @Override
+  public void setListFields(Map<String, List<String>> listFields) {
+    for (String k : listFields.keySet()) {
+      super.setListField(_prefix + k, listFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of list fields
+   */
+  @Override
+  public Map<String, List<String>> getListFields() {
+    return convertToPrefixlessMap(super.getListFields(), _prefix);
+  }
+
+  @Override
+  public void setSimpleField(String k, String v) {
+    super.setSimpleField(_prefix + k, v);
+  }
+
+  @Override
+  public String getSimpleField(String k) {
+    return super.getSimpleField(_prefix + k);
+  }
+
+  @Override
+  public void setSimpleFields(Map<String, String> simpleFields) {
+    for (String k : simpleFields.keySet()) {
+      super.setSimpleField(_prefix + k, simpleFields.get(k));
+    }
+  }
+
+  /**
+   * Returns an immutable map of simple fields
+   */
+  @Override
+  public Map<String, String> getSimpleFields() {
+    return convertToPrefixlessMap(super.getSimpleFields(), _prefix);
+  }
+
+  /**
+   * Get the prefix used to distinguish these config properties
+   * @return string prefix, not including the underscore
+   */
+  public String getPrefix() {
+    return _prefix.substring(0, _prefix.indexOf(PREFIX_CHAR));
+  }
+
+  /**
+   * Remove all fields from this config that are not prefixed
+   */
+  private void filterNonPrefixedFields() {
+    // filter out any configuration that isn't user-defined
+    Predicate<String> keyFilter = new Predicate<String>() {
+      @Override
+      public boolean apply(String key) {
+        return key.contains(_prefix);
+      }
+    };
+    super.setMapFields(Maps.filterKeys(super.getMapFields(), keyFilter));
+    super.setListFields(Maps.filterKeys(super.getListFields(), keyFilter));
+    super.setSimpleFields(Maps.filterKeys(super.getSimpleFields(), keyFilter));
+  }
+
+  /**
+   * Get all map fields with prefixed keys
+   * @return prefixed map fields
+   */
+  private Map<String, Map<String, String>> getPrefixedMapFields() {
+    return super.getMapFields();
+  }
+
+  /**
+   * Get all list fields with prefixed keys
+   * @return prefixed list fields
+   */
+  private Map<String, List<String>> getPrefixedListFields() {
+    return super.getListFields();
+  }
+
+  /**
+   * Get all simple fields with prefixed keys
+   * @return prefixed simple fields
+   */
+  private Map<String, String> getPrefixedSimpleFields() {
+    return super.getSimpleFields();
+  }
+
+  /**
+   * Add user configuration to an existing helix property.
+   * @param property the property to update
+   * @param config the user config
+   */
+  public static void addConfigToProperty(HelixProperty property, NamespacedConfig config) {
+    ZNRecord record = property.getRecord();
+    record.getMapFields().putAll(config.getPrefixedMapFields());
+    record.getListFields().putAll(config.getPrefixedListFields());
+    record.getSimpleFields().putAll(config.getPrefixedSimpleFields());
+    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
+      record.setPayloadSerializer(config.getPayloadSerializer());
+      record.setRawPayload(config.getRawPayload());
+    }
+  }
+
+  /**
+   * Get a copy of a map with the key prefix stripped. The resulting map is immutable
+   * @param rawMap map of key, value pairs where the key is prefixed
+   * @return map of key, value pairs where the key is not prefixed
+   */
+  private static <T> Map<String, T> convertToPrefixlessMap(Map<String, T> rawMap, String prefix) {
+    Map<String, T> convertedMap = new HashMap<String, T>();
+    for (String rawKey : rawMap.keySet()) {
+      String k = rawKey.substring(prefix.length());
+      convertedMap.put(k, rawMap.get(rawKey));
+    }
+    return ImmutableMap.copyOf(convertedMap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
new file mode 100644
index 0000000..1d7b23e
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
@@ -0,0 +1,382 @@
+package org.apache.helix.api.config;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.id.ParticipantId;
+import org.apache.helix.api.id.PartitionId;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Configuration properties of a Helix participant
+ */
+public class ParticipantConfig {
+  private final ParticipantId _id;
+  private final String _hostName;
+  private final int _port;
+  private final boolean _isEnabled;
+  private final Set<PartitionId> _disabledPartitions;
+  private final Set<String> _tags;
+  private final UserConfig _userConfig;
+
+  /**
+   * Initialize a participant configuration. Also see ParticipantConfig.Builder
+   * @param id participant id
+   * @param hostName host where participant can be reached
+   * @param port port to use to contact participant
+   * @param isEnabled true if enabled, false if disabled
+   * @param disabledPartitions set of partitions, if any to disable on this participant
+   * @param tags tags to set for the participant
+   */
+  public ParticipantConfig(ParticipantId id, String hostName, int port, boolean isEnabled,
+      Set<PartitionId> disabledPartitions, Set<String> tags, UserConfig userConfig) {
+    _id = id;
+    _hostName = hostName;
+    _port = port;
+    _isEnabled = isEnabled;
+    _disabledPartitions = ImmutableSet.copyOf(disabledPartitions);
+    _tags = ImmutableSet.copyOf(tags);
+    _userConfig = userConfig;
+  }
+
+  /**
+   * Get the host name of the participant
+   * @return host name, or null if not applicable
+   */
+  public String getHostName() {
+    return _hostName;
+  }
+
+  /**
+   * Get the port of the participant
+   * @return port number, or -1 if not applicable
+   */
+  public int getPort() {
+    return _port;
+  }
+
+  /**
+   * Get if the participant is enabled
+   * @return true if enabled or false otherwise
+   */
+  public boolean isEnabled() {
+    return _isEnabled;
+  }
+
+  /**
+   * Get disabled partition id's
+   * @return set of disabled partition id's, or empty set if none
+   */
+  public Set<PartitionId> getDisabledPartitions() {
+    return _disabledPartitions;
+  }
+
+  /**
+   * Get tags
+   * @return set of tags
+   */
+  public Set<String> getTags() {
+    return _tags;
+  }
+
+  /**
+   * Check if participant has a tag
+   * @param tag tag to check
+   * @return true if tagged, false otherwise
+   */
+  public boolean hasTag(String tag) {
+    return _tags.contains(tag);
+  }
+
+  /**
+   * Get user-specified configuration properties of this participant
+   * @return UserConfig properties
+   */
+  public UserConfig getUserConfig() {
+    return _userConfig;
+  }
+
+  /**
+   * Get the participant id
+   * @return ParticipantId
+   */
+  public ParticipantId getId() {
+    return _id;
+  }
+
+  /**
+   * Update context for a ParticipantConfig
+   */
+  public static class Delta {
+    private enum Fields {
+      HOST_NAME,
+      PORT,
+      ENABLED,
+      USER_CONFIG
+    }
+
+    private Set<Fields> _updateFields;
+    private Set<String> _removedTags;
+    private Set<PartitionId> _removedDisabledPartitions;
+    private Builder _builder;
+
+    /**
+     * Instantiate the delta for a participant config
+     * @param participantId the participant to update
+     */
+    public Delta(ParticipantId participantId) {
+      _updateFields = Sets.newHashSet();
+      _removedTags = Sets.newHashSet();
+      _removedDisabledPartitions = Sets.newHashSet();
+      _builder = new Builder(participantId);
+    }
+
+    /**
+     * Set the participant host name
+     * @param hostName reachable host when live
+     * @return Delta
+     */
+    public Delta setHostName(String hostName) {
+      _builder.hostName(hostName);
+      _updateFields.add(Fields.HOST_NAME);
+      return this;
+    }
+
+    /**
+     * Set the participant port
+     * @param port port number
+     * @return Delta
+     */
+    public Delta setPort(int port) {
+      _builder.port(port);
+      _updateFields.add(Fields.PORT);
+      return this;
+    }
+
+    /**
+     * <<<<<<< HEAD:helix-core/src/main/java/org/apache/helix/api/config/ParticipantConfig.java
+     * Set the enabled status of the participant
+     * @param isEnabled true if enabled, false if disabled
+     *          =======
+     *          Set whether or not the participant is enabled
+     * @param isEnabled true if enabled, false otherwise
+     *          >>>>>>> helix-logical-model:helix-core/src/main/java/org/apache/helix/api/
+     *          ParticipantConfig.java
+     * @return Delta
+     */
+    public Delta setEnabled(boolean isEnabled) {
+      _builder.enabled(isEnabled);
+      _updateFields.add(Fields.ENABLED);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Delta
+     */
+    public Delta setUserConfig(UserConfig userConfig) {
+      _builder.userConfig(userConfig);
+      _updateFields.add(Fields.USER_CONFIG);
+      return this;
+    }
+
+    /**
+     * Add an new tag for this participant
+     * @param tag the tag to add
+     * @return Delta
+     */
+    public Delta addTag(String tag) {
+      _builder.addTag(tag);
+      return this;
+    }
+
+    /**
+     * Remove a tag for this participant
+     * @param tag the tag to remove
+     * @return Delta
+     */
+    public Delta removeTag(String tag) {
+      _removedTags.add(tag);
+      return this;
+    }
+
+    /**
+     * Add a partition to disable for this participant
+     * @param partitionId the partition to disable
+     * @return Delta
+     */
+    public Delta addDisabledPartition(PartitionId partitionId) {
+      _builder.addDisabledPartition(partitionId);
+      return this;
+    }
+
+    /**
+     * Remove a partition from the disabled set for this participant
+     * @param partitionId the partition to enable
+     * @return Delta
+     */
+    public Delta removeDisabledPartition(PartitionId partitionId) {
+      _removedDisabledPartitions.add(partitionId);
+      return this;
+    }
+
+    /**
+     * Create a ParticipantConfig that is the combination of an existing ParticipantConfig and this
+     * delta
+     * @param orig the original ParticipantConfig
+     * @return updated ParticipantConfig
+     */
+    public ParticipantConfig mergeInto(ParticipantConfig orig) {
+      ParticipantConfig deltaConfig = _builder.build();
+      Builder builder =
+          new Builder(orig.getId()).hostName(orig.getHostName()).port(orig.getPort())
+              .enabled(orig.isEnabled()).userConfig(orig.getUserConfig());
+      for (Fields field : _updateFields) {
+        switch (field) {
+        case HOST_NAME:
+          builder.hostName(deltaConfig.getHostName());
+          break;
+        case PORT:
+          builder.port(deltaConfig.getPort());
+          break;
+        case ENABLED:
+          builder.enabled(deltaConfig.isEnabled());
+          break;
+        case USER_CONFIG:
+          builder.userConfig(deltaConfig.getUserConfig());
+          break;
+        }
+      }
+      Set<String> tags = Sets.newHashSet(orig.getTags());
+      tags.addAll(deltaConfig.getTags());
+      tags.removeAll(_removedTags);
+      for (String tag : tags) {
+        builder.addTag(tag);
+      }
+      Set<PartitionId> disabledPartitions = Sets.newHashSet(orig.getDisabledPartitions());
+      disabledPartitions.addAll(deltaConfig.getDisabledPartitions());
+      disabledPartitions.removeAll(_removedDisabledPartitions);
+      for (PartitionId partitionId : disabledPartitions) {
+        builder.addDisabledPartition(partitionId);
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Assemble a participant
+   */
+  public static class Builder {
+    private final ParticipantId _id;
+    private String _hostName;
+    private int _port;
+    private boolean _isEnabled;
+    private final Set<PartitionId> _disabledPartitions;
+    private final Set<String> _tags;
+    private UserConfig _userConfig;
+
+    /**
+     * Build a participant with a given id
+     * @param id participant id
+     */
+    public Builder(ParticipantId id) {
+      _id = id;
+      _disabledPartitions = new HashSet<PartitionId>();
+      _tags = new HashSet<String>();
+      _isEnabled = true;
+      _userConfig = new UserConfig(Scope.participant(id));
+    }
+
+    /**
+     * Set the participant host name
+     * @param hostName reachable host when live
+     * @return Builder
+     */
+    public Builder hostName(String hostName) {
+      _hostName = hostName;
+      return this;
+    }
+
+    /**
+     * Set the participant port
+     * @param port port number
+     * @return Builder
+     */
+    public Builder port(int port) {
+      _port = port;
+      return this;
+    }
+
+    /**
+     * Set whether or not the participant is enabled
+     * @param isEnabled true if enabled, false otherwise
+     * @return Builder
+     */
+    public Builder enabled(boolean isEnabled) {
+      _isEnabled = isEnabled;
+      return this;
+    }
+
+    /**
+     * Add a partition to disable for this participant
+     * @param partitionId the partition to disable
+     * @return Builder
+     */
+    public Builder addDisabledPartition(PartitionId partitionId) {
+      _disabledPartitions.add(partitionId);
+      return this;
+    }
+
+    /**
+     * Add an arbitrary tag for this participant
+     * @param tag the tag to add
+     * @return Builder
+     */
+    public Builder addTag(String tag) {
+      _tags.add(tag);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Builder userConfig(UserConfig userConfig) {
+      _userConfig = userConfig;
+      return this;
+    }
+
+    /**
+     * Assemble the participant
+     * @return instantiated Participant
+     */
+    public ParticipantConfig build() {
+      return new ParticipantConfig(_id, _hostName, _port, _isEnabled, _disabledPartitions, _tags,
+          _userConfig);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
new file mode 100644
index 0000000..38d48ab
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/ResourceConfig.java
@@ -0,0 +1,373 @@
+package org.apache.helix.api.config;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.api.Partition;
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+
+import com.google.common.collect.Sets;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Full configuration of a Helix resource. Typically used to add or modify resources on a cluster
+ */
+public class ResourceConfig {
+  /**
+   * Type of a resource. A resource is any entity that can be managed by Helix.
+   */
+  public enum ResourceType {
+    /**
+     * A resource that is persistent, and potentially partitioned and replicated.
+     */
+    DATA
+  }
+
+  private final ResourceId _id;
+  private final RebalancerConfig _rebalancerConfig;
+  private final SchedulerTaskConfig _schedulerTaskConfig;
+  private final UserConfig _userConfig;
+  private final int _bucketSize;
+  private final boolean _batchMessageMode;
+  private final ResourceType _resourceType;
+
+  /**
+   * Instantiate a configuration. Consider using ResourceConfig.Builder
+   * @param id resource id
+   * @param partitionMap map of partition identifiers to partition objects
+   * @param schedulerTaskConfig configuration for scheduler tasks associated with the resource
+   * @param rebalancerConfig configuration for rebalancing the resource
+   * @param userConfig user-defined resource properties
+   * @param bucketSize bucket size for this resource
+   * @param batchMessageMode whether or not batch messaging is allowed
+   */
+  public ResourceConfig(ResourceId id, ResourceType resourceType,
+      SchedulerTaskConfig schedulerTaskConfig, RebalancerConfig rebalancerConfig,
+      UserConfig userConfig, int bucketSize, boolean batchMessageMode) {
+    _id = id;
+    _resourceType = resourceType;
+    _schedulerTaskConfig = schedulerTaskConfig;
+    _rebalancerConfig = rebalancerConfig;
+    _userConfig = userConfig;
+    _bucketSize = bucketSize;
+    _batchMessageMode = batchMessageMode;
+  }
+
+  /**
+   * Get the subunits of the resource
+   * @return map of subunit id to subunit or empty map if none
+   */
+  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap() {
+    return _rebalancerConfig.getRebalancerContext(RebalancerContext.class).getSubUnitMap();
+  }
+
+  /**
+   * Get a subunit that the resource contains
+   * @param subUnitId the subunit id to look up
+   * @return Partition or null if none is present with the given id
+   */
+  public Partition getSubUnit(PartitionId subUnitId) {
+    return getSubUnitMap().get(subUnitId);
+  }
+
+  /**
+   * Get the set of subunit ids that the resource contains
+   * @return subunit id set, or empty if none
+   */
+  public Set<? extends PartitionId> getSubUnitSet() {
+    return getSubUnitMap().keySet();
+  }
+
+  /**
+   * Get the resource properties configuring rebalancing
+   * @return RebalancerConfig properties
+   */
+  public RebalancerConfig getRebalancerConfig() {
+    return _rebalancerConfig;
+  }
+
+  /**
+   * Get the resource id
+   * @return ResourceId
+   */
+  public ResourceId getId() {
+    return _id;
+  }
+
+  /**
+   * Get the resource type
+   * @return ResourceType
+   */
+  public ResourceType getType() {
+    return _resourceType;
+  }
+
+  /**
+   * Get the properties configuring scheduler tasks
+   * @return SchedulerTaskConfig properties
+   */
+  public SchedulerTaskConfig getSchedulerTaskConfig() {
+    return _schedulerTaskConfig;
+  }
+
+  /**
+   * Get user-specified configuration properties of this resource
+   * @return UserConfig properties
+   */
+  public UserConfig getUserConfig() {
+    return _userConfig;
+  }
+
+  /**
+   * Get the bucket size for this resource
+   * @return bucket size
+   */
+  public int getBucketSize() {
+    return _bucketSize;
+  }
+
+  /**
+   * Get the batch message mode
+   * @return true if enabled, false if disabled
+   */
+  public boolean getBatchMessageMode() {
+    return _batchMessageMode;
+  }
+
+  @Override
+  public String toString() {
+    return getSubUnitMap().toString();
+  }
+
+  /**
+   * Update context for a ResourceConfig
+   */
+  public static class Delta {
+    private enum Fields {
+      TYPE,
+      REBALANCER_CONTEXT,
+      USER_CONFIG,
+      BUCKET_SIZE,
+      BATCH_MESSAGE_MODE
+    }
+
+    private Set<Fields> _updateFields;
+    private Builder _builder;
+
+    /**
+     * Instantiate the delta for a resource config
+     * @param resourceId the resource to update
+     */
+    public Delta(ResourceId resourceId) {
+      _builder = new Builder(resourceId);
+      _updateFields = Sets.newHashSet();
+    }
+
+    /**
+     * Set the type of this resource
+     * @param type ResourceType
+     * @return Delta
+     */
+    public Delta setType(ResourceType type) {
+      _builder.type(type);
+      _updateFields.add(Fields.TYPE);
+      return this;
+    }
+
+    /**
+     * Set the rebalancer configuration
+     * @param context properties of interest for rebalancing
+     * @return Delta
+     */
+    public Delta setRebalancerContext(RebalancerContext context) {
+      _builder.rebalancerContext(context);
+      _updateFields.add(Fields.REBALANCER_CONTEXT);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Delta
+     */
+    public Delta setUserConfig(UserConfig userConfig) {
+      _builder.userConfig(userConfig);
+      _updateFields.add(Fields.USER_CONFIG);
+      return this;
+    }
+
+    /**
+     * Set the bucket size
+     * @param bucketSize the size to use
+     * @return Delta
+     */
+    public Delta setBucketSize(int bucketSize) {
+      _builder.bucketSize(bucketSize);
+      _updateFields.add(Fields.BUCKET_SIZE);
+      return this;
+    }
+
+    /**
+     * Set the batch message mode
+     * @param batchMessageMode true to enable, false to disable
+     * @return Delta
+     */
+    public Delta setBatchMessageMode(boolean batchMessageMode) {
+      _builder.batchMessageMode(batchMessageMode);
+      _updateFields.add(Fields.BATCH_MESSAGE_MODE);
+      return this;
+    }
+
+    /**
+     * Create a ResourceConfig that is the combination of an existing ResourceConfig and this delta
+     * @param orig the original ResourceConfig
+     * @return updated ResourceConfig
+     */
+    public ResourceConfig mergeInto(ResourceConfig orig) {
+      ResourceConfig deltaConfig = _builder.build();
+      Builder builder =
+          new Builder(orig.getId())
+              .type(orig.getType())
+              .rebalancerContext(
+                  orig.getRebalancerConfig().getRebalancerContext(RebalancerContext.class))
+              .schedulerTaskConfig(orig.getSchedulerTaskConfig()).userConfig(orig.getUserConfig())
+              .bucketSize(orig.getBucketSize()).batchMessageMode(orig.getBatchMessageMode());
+      for (Fields field : _updateFields) {
+        switch (field) {
+        case TYPE:
+          builder.type(deltaConfig.getType());
+          break;
+        case REBALANCER_CONTEXT:
+          builder.rebalancerContext(deltaConfig.getRebalancerConfig().getRebalancerContext(
+              RebalancerContext.class));
+          break;
+        case USER_CONFIG:
+          builder.userConfig(deltaConfig.getUserConfig());
+          break;
+        case BUCKET_SIZE:
+          builder.bucketSize(deltaConfig.getBucketSize());
+          break;
+        case BATCH_MESSAGE_MODE:
+          builder.batchMessageMode(deltaConfig.getBatchMessageMode());
+          break;
+        }
+      }
+      return builder.build();
+    }
+  }
+
+  /**
+   * Assembles a ResourceConfig
+   */
+  public static class Builder {
+    private final ResourceId _id;
+    private ResourceType _type;
+    private RebalancerConfig _rebalancerConfig;
+    private SchedulerTaskConfig _schedulerTaskConfig;
+    private UserConfig _userConfig;
+    private int _bucketSize;
+    private boolean _batchMessageMode;
+
+    /**
+     * Build a Resource with an id
+     * @param id resource id
+     */
+    public Builder(ResourceId id) {
+      _id = id;
+      _type = ResourceType.DATA;
+      _bucketSize = 0;
+      _batchMessageMode = false;
+      _userConfig = new UserConfig(Scope.resource(id));
+    }
+
+    /**
+     * Set the type of this resource
+     * @param type ResourceType
+     * @return Builder
+     */
+    public Builder type(ResourceType type) {
+      _type = type;
+      return this;
+    }
+
+    /**
+     * Set the rebalancer configuration
+     * @param rebalancerContext properties of interest for rebalancing
+     * @return Builder
+     */
+    public Builder rebalancerContext(RebalancerContext rebalancerContext) {
+      _rebalancerConfig = new RebalancerConfig(rebalancerContext);
+      return this;
+    }
+
+    /**
+     * Set the user configuration
+     * @param userConfig user-specified properties
+     * @return Builder
+     */
+    public Builder userConfig(UserConfig userConfig) {
+      _userConfig = userConfig;
+      return this;
+    }
+
+    /**
+     * @param schedulerTaskConfig
+     * @return
+     */
+    public Builder schedulerTaskConfig(SchedulerTaskConfig schedulerTaskConfig) {
+      _schedulerTaskConfig = schedulerTaskConfig;
+      return this;
+    }
+
+    /**
+     * Set the bucket size
+     * @param bucketSize the size to use
+     * @return Builder
+     */
+    public Builder bucketSize(int bucketSize) {
+      _bucketSize = bucketSize;
+      return this;
+    }
+
+    /**
+     * Set the batch message mode
+     * @param batchMessageMode true to enable, false to disable
+     * @return Builder
+     */
+    public Builder batchMessageMode(boolean batchMessageMode) {
+      _batchMessageMode = batchMessageMode;
+      return this;
+    }
+
+    /**
+     * Create a Resource object
+     * @return instantiated Resource
+     */
+    public ResourceConfig build() {
+      return new ResourceConfig(_id, _type, _schedulerTaskConfig, _rebalancerConfig, _userConfig,
+          _bucketSize, _batchMessageMode);
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
new file mode 100644
index 0000000..dbf70ea
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/config/UserConfig.java
@@ -0,0 +1,53 @@
+package org.apache.helix.api.config;
+
+import org.apache.helix.HelixProperty;
+import org.apache.helix.api.Scope;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Generic user-defined configuration of Helix components
+ */
+public class UserConfig extends NamespacedConfig {
+  /**
+   * Instantiate a UserConfig. It is intended for use only by entities that can be identified
+   * @param scope scope of the configuration, e.g. cluster, resource, partition, participant, etc
+   */
+  public UserConfig(Scope<?> scope) {
+    super(scope, UserConfig.class.getSimpleName());
+  }
+
+  /**
+   * Instantiate a UserConfig from an existing HelixProperty
+   * @param property property wrapping a configuration
+   */
+  private UserConfig(HelixProperty property) {
+    super(property, UserConfig.class.getSimpleName());
+  }
+
+  /**
+   * Get a UserConfig that filters out the user-specific configurations in a property
+   * @param property the property to extract from
+   * @return UserConfig
+   */
+  public static UserConfig from(HelixProperty property) {
+    return new UserConfig(property);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
new file mode 100644
index 0000000..f05bb5d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ClusterId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Identifies a cluster
+ */
+public class ClusterId extends Id {
+  @JsonProperty("id")
+  final private String _id;
+
+  /**
+   * Create a cluster id
+   * @param id string representation of the id
+   */
+  @JsonCreator
+  public ClusterId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete cluster id for a string name
+   * @param clusterId string cluster identifier
+   * @return ClusterId
+   */
+  public static ClusterId from(String clusterId) {
+    if (clusterId == null) {
+      return null;
+    }
+    return new ClusterId(clusterId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
new file mode 100644
index 0000000..26a7610
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ConstraintId.java
@@ -0,0 +1,80 @@
+package org.apache.helix.api.id;
+
+import org.apache.helix.api.Scope;
+import org.apache.helix.api.State;
+import org.apache.helix.model.Transition;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Identifies a constraint item on the cluster
+ */
+public class ConstraintId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a constraint id
+   * @param constraintId string representing the constraint id
+   */
+  @JsonCreator
+  public ConstraintId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a constraint id from a string
+   * @param constraintId string representing the constraint id
+   * @return ConstraintId
+   */
+  public static ConstraintId from(String constraintId) {
+    return new ConstraintId(constraintId);
+  }
+
+  /**
+   * Get a state constraint id based on the state model definition and state
+   * @param scope the scope of the constraint
+   * @param stateModelDefId the state model
+   * @param state the constrained state
+   * @return ConstraintId
+   */
+  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId, State state) {
+    return new ConstraintId(scope + "|" + stateModelDefId + "|" + state);
+  }
+
+  /**
+   * Get a state constraint id based on the state model definition and transition
+   * @param scope the scope of the constraint
+   * @param stateModelDefId the state model
+   * @param transition the constrained transition
+   * @return ConstraintId
+   */
+  public static ConstraintId from(Scope<?> scope, StateModelDefId stateModelDefId,
+      Transition transition) {
+    return new ConstraintId(scope + "|" + stateModelDefId + "|" + transition);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
new file mode 100644
index 0000000..1130afd
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ControllerId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Identifies Helix nodes that take on the CONTROLLER role
+ */
+public class ControllerId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a controller id
+   * @param id string representation of a controller id
+   */
+  @JsonCreator
+  public ControllerId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a ControllerId from a string
+   * @param controllerId string representing the id
+   * @return ControllerId
+   */
+  public static ControllerId from(String controllerId) {
+    return new ControllerId(controllerId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/Id.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/Id.java b/helix-core/src/main/java/org/apache/helix/api/id/Id.java
new file mode 100644
index 0000000..ea9c6cc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/Id.java
@@ -0,0 +1,55 @@
+package org.apache.helix.api.id;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Generic identifier for Helix constructs
+ */
+public abstract class Id implements Comparable<Id> {
+  public abstract String stringify();
+
+  @Override
+  public String toString() {
+    return stringify();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that instanceof Id) {
+      return this.stringify().equals(((Id) that).stringify());
+    } else if (that instanceof String) {
+      return this.stringify().equals(that);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return this.stringify().hashCode();
+  }
+
+  @Override
+  public int compareTo(Id that) {
+    if (that instanceof Id) {
+      return this.stringify().compareTo(that.stringify());
+    }
+    return -1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
new file mode 100644
index 0000000..a59976d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/MessageId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class MessageId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a message id
+   * @param id string representation of a message id
+   */
+  @JsonCreator
+  public MessageId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete message id
+   * @param messageId string message identifier
+   * @return MsgId
+   */
+  public static MessageId from(String messageId) {
+    if (messageId == null) {
+      return null;
+    }
+    return new MessageId(messageId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
new file mode 100644
index 0000000..5fe91dd
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ParticipantId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class ParticipantId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Instantiate for a participant with a string name
+   * @param id string participant id
+   */
+  @JsonCreator
+  public ParticipantId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete participant id
+   * @param participantId string participant identifier
+   * @return ParticipantId
+   */
+  public static ParticipantId from(String participantId) {
+    if (participantId == null) {
+      return null;
+    }
+    return new ParticipantId(participantId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
new file mode 100644
index 0000000..dd1bc0d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/PartitionId.java
@@ -0,0 +1,112 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class PartitionId extends Id {
+  @JsonProperty("resourceId")
+  private final ResourceId _resourceId;
+  @JsonProperty("partitionName")
+  private final String _partitionName;
+
+  /**
+   * Instantiate for a resource and suffix
+   * @param resourceId resource that the partition belongs to
+   * @param partitionName name of the partition relative to the resource
+   */
+  @JsonCreator
+  public PartitionId(@JsonProperty("resourceId") ResourceId resourceId,
+      @JsonProperty("partitionName") String partitionName) {
+    _resourceId = resourceId;
+    _partitionName = partitionName;
+  }
+
+  /**
+   * Get the id of the resource containing this partition
+   * @return ResourceId
+   */
+  public ResourceId getResourceId() {
+    return _resourceId;
+  }
+
+  @Override
+  public String stringify() {
+    // check in case the partition name is instantiated incorrectly
+    if (_resourceId.stringify().equals(_partitionName)) {
+      return _partitionName;
+    }
+    return String.format("%s_%s", _resourceId, _partitionName);
+  }
+
+  /**
+   * @param partitionName
+   * @return
+   */
+  public static String stripResourceId(String partitionName) {
+    if (partitionName == null || !partitionName.contains("_")) {
+      return partitionName;
+    }
+    return partitionName.substring(partitionName.lastIndexOf("_") + 1);
+  }
+
+  /**
+   * @param partitionName
+   * @return
+   */
+  public static ResourceId extractResourceId(String partitionName) {
+    if (partitionName == null || !partitionName.contains("_")) {
+      return ResourceId.from(partitionName);
+    }
+    return ResourceId.from(partitionName.substring(0, partitionName.lastIndexOf("_")));
+  }
+
+  /**
+   * Get a concrete partition id
+   * @param partitionId string partition identifier
+   * @return PartitionId
+   */
+  public static PartitionId from(String partitionId) {
+    if (partitionId == null) {
+      return null;
+    }
+    return new PartitionId(extractResourceId(partitionId), stripResourceId(partitionId));
+  }
+
+  /**
+   * Same as {@link PartitionId#from(String)}.
+   * @param partitionId string partition identifier
+   * @return PartitionId
+   */
+  public static PartitionId valueOf(String partitionId) {
+    return from(partitionId);
+  }
+
+  /**
+   * Get a concrete partition id
+   * @param resourceId resource identifier
+   * @param partitionSuffix partition identifier relative to a resource
+   * @return PartitionId
+   */
+  public static PartitionId from(ResourceId resourceId, String partitionSuffix) {
+    return new PartitionId(resourceId, partitionSuffix);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
new file mode 100644
index 0000000..3b8c21d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ProcId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class ProcId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a process id
+   * @param id string representation of a process id
+   */
+  @JsonCreator
+  public ProcId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete process id
+   * @param processId string process identifier (e.g. pid@host)
+   * @return ProcId
+   */
+  public static ProcId from(String processId) {
+    if (processId == null) {
+      return null;
+    }
+    return new ProcId(processId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
new file mode 100644
index 0000000..e70fff1
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/ResourceId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Identifies a resource
+ */
+public class ResourceId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Create a resource id
+   * @param id string representation of a resource id
+   */
+  @JsonCreator
+  public ResourceId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  /**
+   * Get a concrete resource id for a string name
+   * @param resourceId string resource identifier
+   * @return ResourceId
+   */
+  public static ResourceId from(String resourceId) {
+    if (resourceId == null) {
+      return null;
+    }
+    return new ResourceId(resourceId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
new file mode 100644
index 0000000..17fb3a3
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/SessionId.java
@@ -0,0 +1,54 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class SessionId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a session id
+   * @param id string representing a session id
+   */
+  @JsonCreator
+  public SessionId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete session id
+   * @param sessionId string session identifier
+   * @return SessionId
+   */
+  public static SessionId from(String sessionId) {
+    if (sessionId == null) {
+      return null;
+    }
+    return new SessionId(sessionId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
new file mode 100644
index 0000000..d50390c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/SpectatorId.java
@@ -0,0 +1,51 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class SpectatorId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a spectator id
+   * @param id string representing a spectator id
+   */
+  @JsonCreator
+  public SpectatorId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Create a spectator id from a string
+   * @param spectatorId string representing a spectator id
+   * @return SpectatorId
+   */
+  public static SpectatorId from(String spectatorId) {
+    return new SpectatorId(spectatorId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
new file mode 100644
index 0000000..7c84f0f
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelDefId.java
@@ -0,0 +1,66 @@
+package org.apache.helix.api.id;
+
+import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+public class StateModelDefId extends Id {
+  public static final StateModelDefId SchedulerTaskQueue = StateModelDefId
+      .from(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE);
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a state model definition id
+   * @param id string representing a state model definition id
+   */
+  @JsonCreator
+  public StateModelDefId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Check if the underlying state model definition id is equal if case is ignored
+   * @param that the StateModelDefId to compare
+   * @return true if equal ignoring case, false otherwise
+   */
+  public boolean equalsIgnoreCase(StateModelDefId that) {
+    return _id.equalsIgnoreCase(that._id);
+  }
+
+  /**
+   * Get a concrete state model definition id
+   * @param stateModelDefId string state model identifier
+   * @return StateModelDefId
+   */
+  public static StateModelDefId from(String stateModelDefId) {
+    if (stateModelDefId == null) {
+      return null;
+    }
+    return new StateModelDefId(stateModelDefId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
new file mode 100644
index 0000000..795c14c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/id/StateModelFactoryId.java
@@ -0,0 +1,57 @@
+package org.apache.helix.api.id;
+
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Id representing a state model factory
+ */
+public class StateModelFactoryId extends Id {
+  @JsonProperty("id")
+  private final String _id;
+
+  /**
+   * Create a state model factory id
+   * @param id string representing a state model factory
+   */
+  @JsonCreator
+  public StateModelFactoryId(@JsonProperty("id") String id) {
+    _id = id;
+  }
+
+  @Override
+  public String stringify() {
+    return _id;
+  }
+
+  /**
+   * Get a concrete state model factory id
+   * @param stateModelFactoryId the string version of the id
+   * @return StateModelFactoryId
+   */
+  public static StateModelFactoryId from(String stateModelFactoryId) {
+    if (stateModelFactoryId == null) {
+      return null;
+    }
+    return new StateModelFactoryId(stateModelFactoryId);
+  }
+}

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/c070a765/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
index e2c8854..9c24bd6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/MessageGenerationPhase.java
@@ -26,12 +26,12 @@ import java.util.Map;
 import java.util.UUID;
 
 import org.apache.helix.HelixManager;
-import org.apache.helix.api.MessageId;
-import org.apache.helix.api.PartitionId;
-import org.apache.helix.api.ResourceId;
-import org.apache.helix.api.SessionId;
 import org.apache.helix.api.State;
-import org.apache.helix.api.StateModelDefId;
+import org.apache.helix.api.id.MessageId;
+import org.apache.helix.api.id.PartitionId;
+import org.apache.helix.api.id.ResourceId;
+import org.apache.helix.api.id.SessionId;
+import org.apache.helix.api.id.StateModelDefId;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;

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