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

[20/53] [abbrv] git commit: [HELIX-100] Improve the helix config api

[HELIX-100] Improve the helix config api


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

Branch: refs/heads/master
Commit: 917af3eb679a292bd11d7f824f3ef314ae6cdb85
Parents: 243f2ad
Author: Kanak Biscuitwala <ka...@hotmail.com>
Authored: Mon Sep 23 17:00:03 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Wed Nov 6 13:17:34 2013 -0800

----------------------------------------------------------------------
 .../org/apache/helix/api/ClusterAccessor.java   | 133 +++--
 .../java/org/apache/helix/api/ClusterId.java    |  10 +-
 .../java/org/apache/helix/api/ConstraintId.java |  12 +-
 .../java/org/apache/helix/api/ControllerId.java |  11 +-
 .../helix/api/CustomRebalancerConfig.java       | 153 -----
 .../helix/api/FullAutoRebalancerConfig.java     |  94 ----
 .../src/main/java/org/apache/helix/api/Id.java  |   2 +-
 .../java/org/apache/helix/api/MessageId.java    |  12 +-
 .../org/apache/helix/api/NamespacedConfig.java  |   2 +-
 .../org/apache/helix/api/ParticipantId.java     |  11 +-
 .../java/org/apache/helix/api/Partition.java    |  28 +-
 .../java/org/apache/helix/api/PartitionId.java  |  23 +-
 .../main/java/org/apache/helix/api/ProcId.java  |  12 +-
 .../org/apache/helix/api/RebalancerConfig.java  | 556 -------------------
 .../org/apache/helix/api/RebalancerRef.java     |  74 ---
 .../java/org/apache/helix/api/Resource.java     | 112 +---
 .../org/apache/helix/api/ResourceConfig.java    |  73 ++-
 .../java/org/apache/helix/api/ResourceId.java   |  10 +-
 .../helix/api/SemiAutoRebalancerConfig.java     | 156 ------
 .../java/org/apache/helix/api/SessionId.java    |  11 +-
 .../java/org/apache/helix/api/SpectatorId.java  |  19 +-
 .../main/java/org/apache/helix/api/State.java   |  12 +-
 .../org/apache/helix/api/StateModelDefId.java   |  15 +-
 .../apache/helix/api/StateModelFactoryId.java   |  14 +-
 .../helix/api/UserDefinedRebalancerConfig.java  | 122 ----
 .../rebalancer/NewAutoRebalancer.java           | 203 -------
 .../rebalancer/NewCustomRebalancer.java         | 125 -----
 .../controller/rebalancer/NewRebalancer.java    |  42 --
 .../rebalancer/NewSemiAutoRebalancer.java       |  76 ---
 .../rebalancer/NewUserDefinedRebalancer.java    |  35 --
 .../context/BasicRebalancerContext.java         | 239 ++++++++
 .../rebalancer/context/ContextSerializer.java   |  37 ++
 .../rebalancer/context/CustomRebalancer.java    | 123 ++++
 .../context/CustomRebalancerContext.java        | 114 ++++
 .../context/DefaultContextSerializer.java       |  83 +++
 .../rebalancer/context/FullAutoRebalancer.java  | 199 +++++++
 .../context/FullAutoRebalancerContext.java      |  61 ++
 .../context/PartitionedRebalancerContext.java   | 355 ++++++++++++
 .../rebalancer/context/Rebalancer.java          |  38 ++
 .../rebalancer/context/RebalancerConfig.java    | 149 +++++
 .../rebalancer/context/RebalancerContext.java   |  93 ++++
 .../rebalancer/context/RebalancerRef.java       |  94 ++++
 .../context/ReplicatedRebalancerContext.java    |  40 ++
 .../rebalancer/context/SemiAutoRebalancer.java  |  78 +++
 .../context/SemiAutoRebalancerContext.java      | 117 ++++
 .../controller/stages/ClusterDataCache.java     |   1 -
 .../stages/NewBestPossibleStateCalcStage.java   |  64 +--
 .../stages/NewCurrentStateComputationStage.java |   7 +-
 .../stages/NewExternalViewComputeStage.java     |  14 +-
 .../stages/NewMessageGenerationStage.java       |  46 +-
 .../stages/NewMessageSelectionStage.java        |  23 +-
 .../stages/NewMessageThrottleStage.java         |   2 +-
 .../stages/NewResourceComputationStage.java     |  87 +--
 .../stages/NewTaskAssignmentStage.java          |   2 +-
 .../java/org/apache/helix/model/IdealState.java |   7 +-
 .../java/org/apache/helix/model/Message.java    |  17 +
 .../helix/model/ResourceConfiguration.java      |  24 -
 .../helix/tools/ClusterStateVerifier.java       |   2 +-
 .../apache/helix/api/TestNamespacedConfig.java  |  43 --
 .../org/apache/helix/api/TestNewStages.java     |  20 +-
 .../context/TestSerializeRebalancerContext.java | 103 ++++
 .../helix/controller/stages/BaseStageTest.java  |  16 +-
 .../stages/TestResourceComputationStage.java    |  29 +-
 .../TestCustomizedIdealStateRebalancer.java     |  27 +-
 .../apache/helix/examples/NewModelExample.java  |  19 +-
 65 files changed, 2397 insertions(+), 2134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java b/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
index e30825c..fd608e3 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ClusterAccessor.java
@@ -20,16 +20,20 @@ package org.apache.helix.api;
  */
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixConstants.StateModelToken;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
+import org.apache.helix.controller.rebalancer.context.CustomRebalancerContext;
+import org.apache.helix.controller.rebalancer.context.PartitionedRebalancerContext;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+import org.apache.helix.controller.rebalancer.context.SemiAutoRebalancerContext;
 import org.apache.helix.model.ClusterConfiguration;
 import org.apache.helix.model.ClusterConstraints;
 import org.apache.helix.model.ClusterConstraints.ConstraintType;
@@ -40,7 +44,6 @@ import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Message;
-import org.apache.helix.model.PartitionConfiguration;
 import org.apache.helix.model.PauseSignal;
 import org.apache.helix.model.ResourceConfiguration;
 import org.apache.helix.model.StateModelDefinition;
@@ -198,19 +201,27 @@ public class ClusterAccessor {
       } else {
         userConfig = new UserConfig(Scope.resource(resourceId));
       }
-
-      Map<String, PartitionConfiguration> partitionConfigMap =
-          _accessor.getChildValuesMap(_keyBuilder.partitionConfigs(resourceName));
-      if (partitionConfigMap != null) {
-        Map<PartitionId, UserConfig> partitionUserConfigs = new HashMap<PartitionId, UserConfig>();
-        for (String partitionName : partitionConfigMap.keySet()) {
-          partitionUserConfigs.put(PartitionId.from(partitionName),
-              UserConfig.from(partitionConfigMap.get(partitionName)));
+      int bucketSize = 0;
+      boolean batchMessageMode = false;
+      RebalancerContext rebalancerContext;
+      if (idealState != null) {
+        rebalancerContext = PartitionedRebalancerContext.from(idealState);
+        bucketSize = idealState.getBucketSize();
+        batchMessageMode = idealState.getBatchMessageMode();
+      } else {
+        ResourceConfiguration resourceConfiguration = resourceConfigMap.get(resourceName);
+        if (resourceConfiguration != null) {
+          bucketSize = resourceConfiguration.getBucketSize();
+          batchMessageMode = resourceConfiguration.getBatchMessageMode();
+          RebalancerConfig rebalancerConfig = new RebalancerConfig(resourceConfiguration);
+          rebalancerContext = rebalancerConfig.getRebalancerContext(RebalancerContext.class);
+        } else {
+          rebalancerContext = new PartitionedRebalancerContext(RebalanceMode.NONE);
         }
-        resourceMap.put(resourceId,
-            new Resource(resourceId, idealState, null, externalViewMap.get(resourceName),
-                userConfig, partitionUserConfigs));
       }
+      resourceMap.put(resourceId,
+          new Resource(resourceId, idealState, null, externalViewMap.get(resourceName),
+              rebalancerContext, userConfig, bucketSize, batchMessageMode));
     }
 
     Map<ParticipantId, Participant> participantMap = new HashMap<ParticipantId, Participant>();
@@ -280,7 +291,9 @@ public class ClusterAccessor {
    */
   public void addResourceToCluster(ResourceConfig resource) {
     // TODO: this belongs in ResourceAccessor
-    StateModelDefId stateModelDefId = resource.getRebalancerConfig().getStateModelDefId();
+    RebalancerContext context =
+        resource.getRebalancerConfig().getRebalancerContext(RebalancerContext.class);
+    StateModelDefId stateModelDefId = context.getStateModelDefId();
     if (_accessor.getProperty(_keyBuilder.stateModelDef(stateModelDefId.stringify())) == null) {
       throw new HelixException("State model: " + stateModelDefId + " not found in cluster: "
           + _clusterId);
@@ -296,68 +309,50 @@ public class ClusterAccessor {
     if (resource.getUserConfig() != null) {
       ResourceConfiguration configuration = new ResourceConfiguration(resourceId);
       configuration.addNamespacedConfig(resource.getUserConfig());
-      configuration.addRebalancerConfig(resource.getRebalancerConfig());
+      configuration.addNamespacedConfig(resource.getRebalancerConfig().toNamespacedConfig());
+      configuration.setBucketSize(resource.getBucketSize());
+      configuration.setBatchMessageMode(resource.getBatchMessageMode());
       _accessor.setProperty(_keyBuilder.resourceConfig(resourceId.stringify()), configuration);
     }
 
-    // Create an IdealState from a RebalancerConfig
+    // Create an IdealState from a RebalancerConfig (if the resource is partitioned)
     RebalancerConfig rebalancerConfig = resource.getRebalancerConfig();
-    IdealState idealState = new IdealState(resourceId);
-    idealState.setRebalanceMode(rebalancerConfig.getRebalancerMode());
-    idealState.setMaxPartitionsPerInstance(rebalancerConfig.getMaxPartitionsPerParticipant());
-    if (rebalancerConfig.canAssignAnyLiveParticipant()) {
-      idealState.setReplicas(HelixConstants.StateModelToken.ANY_LIVEINSTANCE.toString());
-    } else {
-      idealState.setReplicas(Integer.toString(rebalancerConfig.getReplicaCount()));
-    }
-    idealState.setStateModelDefId(rebalancerConfig.getStateModelDefId());
-    for (PartitionId partitionId : resource.getPartitionSet()) {
-      if (rebalancerConfig.getRebalancerMode() == RebalanceMode.SEMI_AUTO) {
-        SemiAutoRebalancerConfig config = SemiAutoRebalancerConfig.from(rebalancerConfig);
-        List<ParticipantId> preferenceList = config.getPreferenceList(partitionId);
-        if (preferenceList != null) {
-          idealState.setPreferenceList(partitionId, preferenceList);
-        }
-      } else if (rebalancerConfig.getRebalancerMode() == RebalanceMode.CUSTOMIZED) {
-        CustomRebalancerConfig config = CustomRebalancerConfig.from(rebalancerConfig);
-        Map<ParticipantId, State> preferenceMap = config.getPreferenceMap(partitionId);
-        if (preferenceMap != null) {
-          idealState.setParticipantStateMap(partitionId, preferenceMap);
-        }
+    PartitionedRebalancerContext partitionedContext =
+        rebalancerConfig.getRebalancerContext(PartitionedRebalancerContext.class);
+    if (context != null) {
+      IdealState idealState = new IdealState(resourceId);
+      idealState.setRebalanceMode(partitionedContext.getRebalanceMode());
+      idealState.setRebalancerRef(partitionedContext.getRebalancerRef());
+      String replicas = null;
+      if (partitionedContext.anyLiveParticipant()) {
+        replicas = StateModelToken.ANY_LIVEINSTANCE.toString();
       } else {
-        // TODO: need these for as long as we use IdealState as the backing physical model
-        List<ParticipantId> emptyList = Collections.emptyList();
-        Map<ParticipantId, State> emptyMap = Collections.emptyMap();
-        idealState.setPreferenceList(partitionId, emptyList);
-        idealState.setParticipantStateMap(partitionId, emptyMap);
+        replicas = Integer.toString(partitionedContext.getReplicaCount());
       }
-      Partition partition = resource.getPartition(partitionId);
-      if (partition.getUserConfig() != null) {
-        PartitionConfiguration partitionConfig =
-            PartitionConfiguration.from(partition.getUserConfig());
-        _accessor.setProperty(
-            _keyBuilder.partitionConfig(resourceId.stringify(), partitionId.stringify()),
-            partitionConfig);
-      }
-    }
-    idealState.setBucketSize(resource.getBucketSize());
-    idealState.setBatchMessageMode(resource.getBatchMessageMode());
-    String groupTag = rebalancerConfig.getParticipantGroupTag();
-    if (groupTag != null) {
-      idealState.setInstanceGroupTag(groupTag);
-    }
-    if (rebalancerConfig.getRebalancerMode() == RebalanceMode.USER_DEFINED) {
-      UserDefinedRebalancerConfig config = UserDefinedRebalancerConfig.from(rebalancerConfig);
-      RebalancerRef rebalancerRef = config.getRebalancerRef();
-      if (rebalancerRef != null) {
-        idealState.setRebalancerRef(rebalancerRef);
+      idealState.setReplicas(replicas);
+      idealState.setNumPartitions(partitionedContext.getPartitionSet().size());
+      idealState.setInstanceGroupTag(partitionedContext.getParticipantGroupTag());
+      idealState.setMaxPartitionsPerInstance(partitionedContext.getMaxPartitionsPerParticipant());
+      idealState.setStateModelDefId(partitionedContext.getStateModelDefId());
+      idealState.setStateModelFactoryId(partitionedContext.getStateModelFactoryId());
+      idealState.setBucketSize(resource.getBucketSize());
+      idealState.setBatchMessageMode(resource.getBatchMessageMode());
+      if (partitionedContext.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
+        SemiAutoRebalancerContext semiAutoContext =
+            rebalancerConfig.getRebalancerContext(SemiAutoRebalancerContext.class);
+        for (PartitionId partitionId : semiAutoContext.getPartitionSet()) {
+          idealState.setPreferenceList(partitionId, semiAutoContext.getPreferenceList(partitionId));
+        }
+      } else if (partitionedContext.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
+        CustomRebalancerContext customContext =
+            rebalancerConfig.getRebalancerContext(CustomRebalancerContext.class);
+        for (PartitionId partitionId : customContext.getPartitionSet()) {
+          idealState.setParticipantStateMap(partitionId,
+              customContext.getPreferenceMap(partitionId));
+        }
       }
+      _accessor.createProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
     }
-    StateModelFactoryId stateModelFactoryId = rebalancerConfig.getStateModelFactoryId();
-    if (stateModelFactoryId != null) {
-      idealState.setStateModelFactoryId(stateModelFactoryId);
-    }
-    _accessor.createProperty(_keyBuilder.idealState(resourceId.stringify()), idealState);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ClusterId.java b/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
index 3375614..4e8b382 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ClusterId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -23,12 +26,15 @@ package org.apache.helix.api;
  * Identifies a cluster
  */
 public class ClusterId extends Id {
+  @JsonProperty("id")
   final private String _id;
 
   /**
-   * @param id
+   * Create a cluster id
+   * @param id string representation of the id
    */
-  private ClusterId(String id) {
+  @JsonCreator
+  public ClusterId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java b/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
index 85c9430..7da6714 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ConstraintId.java
@@ -1,6 +1,8 @@
 package org.apache.helix.api;
 
 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
@@ -25,19 +27,21 @@ import org.apache.helix.model.Transition;
  * Identifies a constraint item on the cluster
  */
 public class ConstraintId extends Id {
-  private final String _constraintId;
+  @JsonProperty("id")
+  private final String _id;
 
   /**
    * Create a constraint id
    * @param constraintId string representing the constraint id
    */
-  private ConstraintId(String constraintId) {
-    _constraintId = constraintId;
+  @JsonCreator
+  public ConstraintId(@JsonProperty("id") String id) {
+    _id = id;
   }
 
   @Override
   public String stringify() {
-    return _constraintId;
+    return _id;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ControllerId.java b/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
index 8a8c31f..e55f37a 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ControllerId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -23,9 +26,15 @@ package org.apache.helix.api;
  * Identifies Helix nodes that take on the CONTROLLER role
  */
 public class ControllerId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
-  private ControllerId(String id) {
+  /**
+   * Create a controller id
+   * @param id string representation of a controller id
+   */
+  @JsonCreator
+  public ControllerId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/CustomRebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/CustomRebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/api/CustomRebalancerConfig.java
deleted file mode 100644
index a5f2bbe..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/CustomRebalancerConfig.java
+++ /dev/null
@@ -1,153 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Configuration properties for the CUSTOMIZED rebalancer
- */
-public final class CustomRebalancerConfig extends RebalancerConfig {
-  /**
-   * Instantiate a new config for CUSTOMIZED
-   * @param resourceId the resource to rebalance
-   * @param stateModelDefId the state model that the resource follows
-   * @param partitionMap map of partition id to partition
-   */
-  public CustomRebalancerConfig(ResourceId resourceId, StateModelDefId stateModelDefId,
-      Map<PartitionId, Partition> partitionMap) {
-    super(resourceId, RebalanceMode.CUSTOMIZED, stateModelDefId, partitionMap);
-  }
-
-  /**
-   * Instantiate from a base RebalancerConfig
-   * @param config populated rebalancer config
-   */
-  private CustomRebalancerConfig(RebalancerConfig config) {
-    super(config);
-  }
-
-  /**
-   * Get the preference map for a partition
-   * @param partitionId the partition to look up
-   * @return preference map of participant to state for each replica
-   */
-  public Map<ParticipantId, State> getPreferenceMap(PartitionId partitionId) {
-    Map<String, String> rawPreferenceMap = getMapField(partitionId.stringify());
-    if (rawPreferenceMap != null) {
-      return IdealState.participantStateMapFromStringMap(rawPreferenceMap);
-    }
-    return Collections.emptyMap();
-  }
-
-  /**
-   * Set the preference map for a partition
-   * @param partitionId the partition to set
-   * @param preferenceMap map of participant to state for each replica
-   */
-  public void setPreferenceMap(PartitionId partitionId, Map<ParticipantId, State> preferenceMap) {
-    setMapField(partitionId.toString(), IdealState.stringMapFromParticipantStateMap(preferenceMap));
-  }
-
-  /**
-   * Get all the preference maps for a partition
-   * @return map of partition id to map of participant id to state for each replica
-   */
-  public Map<PartitionId, Map<ParticipantId, State>> getPreferenceMaps() {
-    Map<String, Map<String, String>> rawPreferenceMaps = getMapFields();
-    return IdealState.participantStateMapsFromStringMaps(rawPreferenceMaps);
-  }
-
-  /**
-   * Set all the preference maps for a partition
-   * @param preferenceMaps map of partition id to map of participant id to state for each replica
-   */
-  public void setPreferenceMaps(Map<PartitionId, Map<ParticipantId, State>> preferenceMaps) {
-    setMapFields(IdealState.stringMapsFromParticipantStateMaps(preferenceMaps));
-  }
-
-  /**
-   * Get a CustomRebalancerConfig from a RebalancerConfig
-   * @param config populated RebalancerConfig
-   * @return CustomRebalancerConfig
-   */
-  public static CustomRebalancerConfig from(RebalancerConfig config) {
-    return new CustomRebalancerConfig(config);
-  }
-
-  /**
-   * Assembler for a CUSTOMIZED configuration
-   */
-  public static class Builder extends RebalancerConfig.Builder<Builder> {
-    private final Map<PartitionId, Map<ParticipantId, State>> _preferenceMaps;
-
-    /**
-     * Build for a specific resource
-     * @param resourceId the resource to rebalance
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-      _preferenceMaps = new HashMap<PartitionId, Map<ParticipantId, State>>();
-    }
-
-		/**
-		 * Construct builder using an existing custom rebalancer config
-		 * @param config
-		 */
-		public Builder(CustomRebalancerConfig config) {
-			super(config);
-			_preferenceMaps = new HashMap<PartitionId, Map<ParticipantId, State>>();
-			_preferenceMaps.putAll(config.getPreferenceMaps());
-		}
-
-    /**
-     * Add a preference map of a partition
-     * @param partitionId the partition to set
-     * @param preferenceMap map of participant id to state
-     * @return Builder
-     */
-    public Builder preferenceMap(PartitionId partitionId, Map<ParticipantId, State> preferenceMap) {
-      _preferenceMaps.put(partitionId, preferenceMap);
-      return this;
-    }
-
-    @Override
-    public CustomRebalancerConfig build() {
-      if (_partitionMap.isEmpty()) {
-        addPartitions(1);
-      }
-      CustomRebalancerConfig config =
-          new CustomRebalancerConfig(_resourceId, _stateModelDefId, _partitionMap);
-      update(config);
-      config.setPreferenceMaps(_preferenceMaps);
-      return config;
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/FullAutoRebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/FullAutoRebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/api/FullAutoRebalancerConfig.java
deleted file mode 100644
index 599d20d..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/FullAutoRebalancerConfig.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Map;
-
-import org.apache.helix.model.IdealState.RebalanceMode;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Configuration properties for the FULL_AUTO rebalancer
- */
-public final class FullAutoRebalancerConfig extends RebalancerConfig {
-  /**
-   * Instantiate a new config for FULL_AUTO
-   * @param resourceId the resource to rebalance
-   * @param stateModelDefId the state model that the resource follows
-   * @param partitionMap map of partition id to partition
-   */
-  public FullAutoRebalancerConfig(ResourceId resourceId, StateModelDefId stateModelDefId,
-      Map<PartitionId, Partition> partitionMap) {
-    super(resourceId, RebalanceMode.FULL_AUTO, stateModelDefId, partitionMap);
-  }
-
-  /**
-   * Instantiate from a base RebalancerConfig
-   * @param config populated rebalancer config
-   */
-  private FullAutoRebalancerConfig(RebalancerConfig config) {
-    super(config);
-  }
-
-  /**
-   * Get a FullAutoRebalancerConfig from a RebalancerConfig
-   * @param config populated RebalancerConfig
-   * @return FullAutoRebalancerConfig
-   */
-  public static FullAutoRebalancerConfig from(RebalancerConfig config) {
-    return new FullAutoRebalancerConfig(config);
-  }
-
-  /**
-   * Assembler for a FULL_AUTO configuration
-   */
-  public static class Builder extends RebalancerConfig.Builder<Builder> {
-    /**
-     * Build for a specific resource
-     * @param resourceId the resource to rebalance
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-    }
-
-    /**
-     * Construct a builder using an existing full-auto rebalancer config
-     * @param config
-     */
-    public Builder(FullAutoRebalancerConfig config) {
-      super(config);
-    }
-
-    @Override
-    public FullAutoRebalancerConfig build() {
-      if (_partitionMap.isEmpty()) {
-        addPartitions(1);
-      }
-      FullAutoRebalancerConfig config =
-          new FullAutoRebalancerConfig(_resourceId, _stateModelDefId, _partitionMap);
-      update(config);
-      return config;
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/Id.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Id.java b/helix-core/src/main/java/org/apache/helix/api/Id.java
index 312bb26..ce5d2e4 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Id.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Id.java
@@ -20,7 +20,7 @@ package org.apache.helix.api;
  */
 
 /**
- *
+ * Generic identifier for Helix constructs
  */
 public abstract class Id implements Comparable<Id> {
   public abstract String stringify();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/MessageId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/MessageId.java b/helix-core/src/main/java/org/apache/helix/api/MessageId.java
index c84aae3..c5d4002 100644
--- a/helix-core/src/main/java/org/apache/helix/api/MessageId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/MessageId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,10 +23,15 @@ package org.apache.helix.api;
  */
 
 public class MessageId extends Id {
-
+  @JsonProperty("id")
   private final String _id;
 
-  private MessageId(String id) {
+  /**
+   * Create a message id
+   * @param id string representation of a message id
+   */
+  @JsonCreator
+  public MessageId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java b/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
index 5c7fbc3..675b144 100644
--- a/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/api/NamespacedConfig.java
@@ -33,7 +33,7 @@ import com.google.common.collect.Maps;
 /**
  * Generic configuration of Helix components prefixed with a namespace
  */
-public abstract class NamespacedConfig extends ZNRecord {
+public class NamespacedConfig extends ZNRecord {
   private static final char PREFIX_CHAR = '!';
   private final String _prefix;
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java b/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
index 7390d5d..59aa93b 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ParticipantId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,9 +23,15 @@ package org.apache.helix.api;
  */
 
 public class ParticipantId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
-  private ParticipantId(String id) {
+  /**
+   * Instantiate for a participant with a string name
+   * @param id string participant id
+   */
+  @JsonCreator
+  public ParticipantId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/Partition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Partition.java b/helix-core/src/main/java/org/apache/helix/api/Partition.java
index e76c8d1..3853f61 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Partition.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Partition.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -23,27 +26,16 @@ package org.apache.helix.api;
  * A partition of a resource
  */
 public class Partition {
-
+  @JsonProperty("id")
   private final PartitionId _id;
-  private final UserConfig _userConfig;
-
-  /**
-   * Construct a partition with user-specified configuration
-   * @param id
-   * @param userConfig user-defined properties of this partition
-   */
-  public Partition(PartitionId id, UserConfig userConfig) {
-    _id = id;
-    _userConfig = userConfig;
-  }
 
   /**
    * Construct a partition
    * @param id
    */
-  public Partition(PartitionId id) {
+  @JsonCreator
+  public Partition(@JsonProperty("id") PartitionId id) {
     _id = id;
-    _userConfig = null;
   }
 
   /**
@@ -54,14 +46,6 @@ public class Partition {
     return _id;
   }
 
-  /**
-   * Get the user config of this partition
-   * @return UserConfig properties, or null if none
-   */
-  public UserConfig getUserConfig() {
-    return _userConfig;
-  }
-
   @Override
   public String toString() {
     return _id.toString();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/PartitionId.java b/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
index 757406d..8f852c2 100644
--- a/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/PartitionId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,10 +23,19 @@ package org.apache.helix.api;
  */
 
 public class PartitionId extends Id {
+  @JsonProperty("resourceId")
   private final ResourceId _resourceId;
+  @JsonProperty("partitionName")
   private final String _partitionName;
 
-  private PartitionId(ResourceId resourceId, 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;
   }
@@ -80,6 +92,15 @@ public class PartitionId extends Id {
   }
 
   /**
+   * 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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ProcId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ProcId.java b/helix-core/src/main/java/org/apache/helix/api/ProcId.java
index b97512a..4668f57 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ProcId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ProcId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,10 +23,15 @@ package org.apache.helix.api;
  */
 
 public class ProcId extends Id {
-
+  @JsonProperty("id")
   private final String _id;
 
-  private ProcId(String id) {
+  /**
+   * Create a process id
+   * @param id string representation of a process id
+   */
+  @JsonCreator
+  public ProcId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/RebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/RebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/api/RebalancerConfig.java
deleted file mode 100644
index d377af8..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/RebalancerConfig.java
+++ /dev/null
@@ -1,556 +0,0 @@
-package org.apache.helix.api;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.helix.model.IdealState.RebalanceMode;
-import org.apache.helix.model.ResourceConfiguration;
-
-import com.google.common.base.Functions;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-/**
- * Captures the configuration properties necessary for rebalancing
- */
-public class RebalancerConfig extends NamespacedConfig {
-
-  /**
-   * Fields used by the base RebalancerConfig
-   */
-  public enum Fields {
-    ANY_LIVE_PARTICIPANT,
-    MAX_PARTITIONS_PER_PARTICIPANT,
-    PARTICIPANT_GROUP_TAG,
-    REBALANCE_MODE,
-    REPLICA_COUNT,
-    STATE_MODEL_DEFINITION,
-    STATE_MODEL_FACTORY
-  }
-
-  private final ResourceId _resourceId;
-  private final Set<String> _fieldsSet;
-  private final Map<PartitionId, Partition> _partitionMap;
-
-  /**
-   * Instantiate a RebalancerConfig.
-   * @param resourceId the resource to rebalance
-   * @param rebalancerMode the mode to rebalance with
-   * @param stateModelDefId the state model that the resource uses
-   * @param stateModelFacotryId the state model factory that the resource uses
-   * @param partitionMap partitions of the resource
-   */
-  public RebalancerConfig(ResourceId resourceId, RebalanceMode rebalancerMode,
-      StateModelDefId stateModelDefId, StateModelFactoryId stateModelFactoryId,
-      Map<PartitionId, Partition> partitionMap) {
-    super(Scope.resource(resourceId), RebalancerConfig.class.getSimpleName());
-    _resourceId = resourceId;
-    _fieldsSet =
-        ImmutableSet.copyOf(Lists.transform(Arrays.asList(Fields.values()),
-            Functions.toStringFunction()));
-    setEnumField(Fields.REBALANCE_MODE.toString(), rebalancerMode);
-    setSimpleField(Fields.STATE_MODEL_DEFINITION.toString(), stateModelDefId.stringify());
-    setSimpleField(Fields.STATE_MODEL_FACTORY.name(), stateModelFactoryId.stringify());
-    _partitionMap = ImmutableMap.copyOf(partitionMap);
-  }
-
-  /**
-   * Instantiate a RebalancerConfig.
-   * @param resourceId the resource to rebalance
-   * @param rebalancerMode the mode to rebalance with
-   * @param stateModelDefId the state model that the resource uses
-   * @param partitionMap partitions of the resource
-   */
-  public RebalancerConfig(ResourceId resourceId, RebalanceMode rebalancerMode,
-      StateModelDefId stateModelDefId, Map<PartitionId, Partition> partitionMap) {
-    super(Scope.resource(resourceId), RebalancerConfig.class.getSimpleName());
-    _resourceId = resourceId;
-    _fieldsSet =
-        ImmutableSet.copyOf(Lists.transform(Arrays.asList(Fields.values()),
-            Functions.toStringFunction()));
-    setEnumField(Fields.REBALANCE_MODE.toString(), rebalancerMode);
-    setSimpleField(Fields.STATE_MODEL_DEFINITION.toString(), stateModelDefId.stringify());
-    _partitionMap = ImmutableMap.copyOf(partitionMap);
-  }
-
-  /**
-   * Extract rebalancer-specific configuration from a physical resource config
-   * @param resourceConfiguration resource config
-   * @param partitionMap map of partition id to partition object
-   */
-  protected RebalancerConfig(ResourceConfiguration resourceConfiguration,
-      Map<PartitionId, Partition> partitionMap) {
-    super(resourceConfiguration, RebalancerConfig.class.getSimpleName());
-    _resourceId = resourceConfiguration.getResourceId();
-    _fieldsSet =
-        ImmutableSet.copyOf(Lists.transform(Arrays.asList(Fields.values()),
-            Functions.toStringFunction()));
-    _partitionMap = ImmutableMap.copyOf(partitionMap);
-  }
-
-  /**
-   * Copy a RebalancerConfig from an existing one
-   * @param config populated RebalancerConfig
-   */
-  protected RebalancerConfig(RebalancerConfig config) {
-    super(Scope.resource(config.getResourceId()), RebalancerConfig.class.getSimpleName());
-    _resourceId = config.getResourceId();
-    _partitionMap = config.getPartitionMap();
-    _fieldsSet =
-        ImmutableSet.copyOf(Lists.transform(Arrays.asList(Fields.values()),
-            Functions.toStringFunction()));
-    super.setSimpleFields(config.getRawSimpleFields());
-    super.setListFields(config.getRawListFields());
-    super.setMapFields(config.getRawMapFields());
-    if (config.getRawPayload() != null && config.getRawPayload().length > 0) {
-      setRawPayload(config.getRawPayload());
-      setPayloadSerializer(config.getPayloadSerializer());
-    }
-  }
-
-  /**
-   * Get the resource id
-   * @return ResourceId
-   */
-  public ResourceId getResourceId() {
-    return _resourceId;
-  }
-
-  /**
-   * Get the partitions of the resource
-   * @return map of partition id to partition or empty map if none
-   */
-  public Map<PartitionId, Partition> getPartitionMap() {
-    return _partitionMap;
-  }
-
-  /**
-   * Get a partition that the resource contains
-   * @param partitionId the partition id to look up
-   * @return Partition or null if none is present with the given id
-   */
-  public Partition getPartition(PartitionId partitionId) {
-    return _partitionMap.get(partitionId);
-  }
-
-  /**
-   * Get the set of partition ids that the resource contains
-   * @return partition id set, or empty if none
-   */
-  public Set<PartitionId> getPartitionSet() {
-    Set<PartitionId> partitionSet = new HashSet<PartitionId>();
-    partitionSet.addAll(_partitionMap.keySet());
-    return ImmutableSet.copyOf(partitionSet);
-  }
-
-  /**
-   * Get the rebalancer mode
-   * @return rebalancer mode
-   */
-  public RebalanceMode getRebalancerMode() {
-    return getEnumField(Fields.REBALANCE_MODE.toString(), RebalanceMode.class, RebalanceMode.NONE);
-  }
-
-  /**
-   * Get state model definition name of the resource
-   * @return state model definition
-   */
-  public StateModelDefId getStateModelDefId() {
-    return StateModelDefId.from(getStringField(Fields.STATE_MODEL_DEFINITION.toString(), null));
-  }
-
-  /**
-   * Get the number of replicas each partition should have. This function will return 0 if some
-   * policy overrides the replica count, for instance if any live participant can accept a replica.
-   * @return replica count
-   */
-  public int getReplicaCount() {
-    return getIntField(Fields.REPLICA_COUNT.toString(), 0);
-  }
-
-  /**
-   * Set the number of replicas each partition should have.
-   * @param replicaCount replica count
-   */
-  public void setReplicaCount(int replicaCount) {
-    setIntField(Fields.REPLICA_COUNT.toString(), replicaCount);
-  }
-
-  /**
-   * Get the number of partitions of this resource that a given participant can accept
-   * @return maximum number of partitions
-   */
-  public int getMaxPartitionsPerParticipant() {
-    return getIntField(Fields.MAX_PARTITIONS_PER_PARTICIPANT.toString(), Integer.MAX_VALUE);
-  }
-
-  /**
-   * Set the number of partitions of this resource that a given participant can accept
-   * @param maxPartitionsPerParticipant maximum number of partitions
-   */
-  public void setMaxPartitionsPerParticipant(int maxPartitionsPerParticipant) {
-    setIntField(Fields.MAX_PARTITIONS_PER_PARTICIPANT.toString(), maxPartitionsPerParticipant);
-  }
-
-  /**
-   * Get the tag, if any, which must be present on assignable instances
-   * @return group tag, or null if it is not present
-   */
-  public String getParticipantGroupTag() {
-    return getStringField(Fields.PARTICIPANT_GROUP_TAG.toString(), null);
-  }
-
-  /**
-   * Set the tag, if any, which must be present on assignable instances
-   * @param participantGroupTag group tag
-   */
-  public void setParticipantGroupTag(String participantGroupTag) {
-    setSimpleField(Fields.PARTICIPANT_GROUP_TAG.toString(), participantGroupTag);
-  }
-
-  /**
-   * Get state model factory id
-   * @return state model factory id
-   */
-  public StateModelFactoryId getStateModelFactoryId() {
-    return StateModelFactoryId.from(getStringField(Fields.STATE_MODEL_FACTORY.toString(), null));
-  }
-
-  /**
-   * Set state model factory id
-   * @param factoryId state model factory id
-   */
-  public void setStateModelFactoryId(StateModelFactoryId factoryId) {
-    if (factoryId != null) {
-      setSimpleField(Fields.STATE_MODEL_FACTORY.toString(), factoryId.stringify());
-    }
-  }
-
-  /**
-   * Check if replicas can be assigned to any live participant
-   * @return true if they can, false if they cannot
-   */
-  public boolean canAssignAnyLiveParticipant() {
-    return getBooleanField(Fields.ANY_LIVE_PARTICIPANT.toString(), false);
-  }
-
-  /**
-   * Specify if replicas can be assigned to any live participant
-   * @param canAssignAny true if they can, false if they cannot
-   */
-  public void setCanAssignAnyLiveParticipant(boolean canAssignAny) {
-    setBooleanField(Fields.ANY_LIVE_PARTICIPANT.toString(), canAssignAny);
-  }
-
-  /*
-   * Override: removes from view fields set by RebalancerConfig
-   */
-  @Override
-  public Map<String, String> getSimpleFields() {
-    return Maps.filterKeys(super.getSimpleFields(), filterBaseConfigFields());
-  }
-
-  /*
-   * Override: makes sure that updated simple fields include those from this class
-   */
-  @Override
-  public void setSimpleFields(Map<String, String> simpleFields) {
-    Map<String, String> copySimpleFields = new HashMap<String, String>();
-    copySimpleFields.putAll(simpleFields);
-    for (String field : _fieldsSet) {
-      String value = getStringField(field, null);
-      if (value != null) {
-        copySimpleFields.put(field, value);
-      }
-    }
-    super.setSimpleFields(copySimpleFields);
-  }
-
-  /**
-   * Get a predicate that can checks if a key is used by this config
-   * @return Guava Predicate
-   */
-  private Predicate<String> filterBaseConfigFields() {
-    return new Predicate<String>() {
-      @Override
-      public boolean apply(String key) {
-        return !_fieldsSet.contains(key);
-      }
-    };
-  }
-
-  /**
-   * Get simple fields without filtering out base fields
-   * @return simple fields
-   */
-  private Map<String, String> getRawSimpleFields() {
-    return super.getSimpleFields();
-  }
-
-  /**
-   * Get list fields without filtering out base fields
-   * @return list fields
-   */
-  private Map<String, List<String>> getRawListFields() {
-    return super.getListFields();
-  }
-
-  /**
-   * Get map fields without filtering out base fields
-   * @return map fields
-   */
-  private Map<String, Map<String, String>> getRawMapFields() {
-    return super.getMapFields();
-  }
-
-  /**
-   * Get a RebalancerConfig from a physical resource configuration
-   * @param config resource configuration
-   * @return RebalancerConfig
-   */
-  public static RebalancerConfig from(ResourceConfiguration config,
-      Map<PartitionId, UserConfig> partitionConfigs) {
-    Map<PartitionId, Partition> partitionMap = new HashMap<PartitionId, Partition>();
-    for (PartitionId partitionId : config.getPartitionIds()) {
-      if (partitionConfigs.containsKey(partitionId)) {
-        partitionMap
-            .put(partitionId, new Partition(partitionId, partitionConfigs.get(partitionId)));
-      } else {
-        partitionMap.put(partitionId, new Partition(partitionId));
-      }
-    }
-    return new RebalancerConfig(config, partitionMap);
-  }
-
-  /**
-   * Assembles a RebalancerConfig
-   */
-  public static abstract class Builder<T extends Builder<T>> {
-    protected final ResourceId _resourceId;
-    protected final Map<PartitionId, Partition> _partitionMap;
-    protected StateModelDefId _stateModelDefId;
-    private StateModelFactoryId _stateModelFactoryId;
-    private boolean _anyLiveParticipant;
-    private int _replicaCount;
-    private int _maxPartitionsPerParticipant;
-    private String _participantGroupTag;
-
-    /**
-     * Configure the rebalancer for a resource
-     * @param resourceId the resource to rebalance
-     */
-    public Builder(ResourceId resourceId) {
-      _resourceId = resourceId;
-      _anyLiveParticipant = false;
-      _replicaCount = 0;
-      _maxPartitionsPerParticipant = Integer.MAX_VALUE;
-      _partitionMap = new TreeMap<PartitionId, Partition>();
-    }
-
-    /**
-     * Construct a builder from an existing rebalancer config
-     * @param config
-     */
-    public Builder(RebalancerConfig config) {
-      _resourceId = config.getResourceId();
-      _partitionMap = new TreeMap<PartitionId, Partition>();
-      _partitionMap.putAll(config.getPartitionMap());
-      _stateModelDefId = config.getStateModelDefId();
-      _stateModelFactoryId = config.getStateModelFactoryId();
-      _anyLiveParticipant = config.canAssignAnyLiveParticipant();
-      _replicaCount = config.getReplicaCount();
-      _maxPartitionsPerParticipant = config.getMaxPartitionsPerParticipant();
-      _participantGroupTag = config.getParticipantGroupTag();
-    }
-
-    /**
-     * Set the state model definition
-     * @param stateModelDefId state model identifier
-     * @return Builder
-     */
-    public T stateModelDef(StateModelDefId stateModelDefId) {
-      _stateModelDefId = stateModelDefId;
-      return self();
-    }
-
-    /**
-     * Set the number of replicas
-     * @param replicaCount number of replicas
-     * @return Builder
-     */
-    public T replicaCount(int replicaCount) {
-      _replicaCount = replicaCount;
-      return self();
-    }
-
-    /**
-     * Set the maximum number of partitions to assign to any participant
-     * @param maxPartitions
-     * @return Builder
-     */
-    public T maxPartitionsPerParticipant(int maxPartitions) {
-      _maxPartitionsPerParticipant = maxPartitions;
-      return self();
-    }
-
-    /**
-     * Set state model factory
-     * @param stateModelFactoryId
-     * @return Builder
-     */
-    public T stateModelFactoryId(StateModelFactoryId stateModelFactoryId) {
-      _stateModelFactoryId = stateModelFactoryId;
-      return self();
-    }
-
-    /**
-     * Set whether any live participant should be used in rebalancing
-     * @param useAnyParticipant true if any live participant can be used, false otherwise
-     * @return Builder
-     */
-    public T anyLiveParticipant(boolean useAnyParticipant) {
-      _anyLiveParticipant = useAnyParticipant;
-      return self();
-    }
-
-    /**
-     * Set participant group tag
-     * @param tag
-     * @return Builder
-     */
-    public T participantGroupTag(String tag) {
-      _participantGroupTag = tag;
-      return self();
-    }
-
-    /**
-     * Add a partition that the resource serves
-     * @param partition fully-qualified partition
-     * @return Builder
-     */
-    public T addPartition(Partition partition) {
-      _partitionMap.put(partition.getId(), partition);
-      return self();
-    }
-
-    /**
-     * Add a collection of partitions
-     * @param partitions
-     * @return Builder
-     */
-    public T addPartitions(Collection<Partition> partitions) {
-      for (Partition partition : partitions) {
-        addPartition(partition);
-      }
-      return self();
-    }
-
-    /**
-     * Add a specified number of partitions with a default naming scheme, namely
-     * resourceId_partitionNumber where partitionNumber starts at 0
-     * These partitions are added without any user configuration properties
-     * @param partitionCount number of partitions to add
-     * @return Builder
-     */
-    public T addPartitions(int partitionCount) {
-      for (int i = 0; i < partitionCount; i++) {
-        addPartition(new Partition(PartitionId.from(_resourceId, Integer.toString(i)), null));
-      }
-      return self();
-    }
-
-    /**
-     * Update a RebalancerConfig with built fields
-     * @param rebalancerConfig the config to update
-     */
-    protected void update(RebalancerConfig rebalancerConfig) {
-      rebalancerConfig.setReplicaCount(_replicaCount);
-      rebalancerConfig.setCanAssignAnyLiveParticipant(_anyLiveParticipant);
-      rebalancerConfig.setMaxPartitionsPerParticipant(_maxPartitionsPerParticipant);
-      if (_stateModelFactoryId != null) {
-        rebalancerConfig.setStateModelFactoryId(_stateModelFactoryId);
-      }
-      if (_participantGroupTag != null) {
-        rebalancerConfig.setParticipantGroupTag(_participantGroupTag);
-      }
-    }
-
-    /**
-     * Get a reference to the actual builder class
-     * @return Builder
-     */
-    protected abstract T self();
-
-    /**
-     * Get a fully-initialized RebalancerConfig instance
-     * @return RebalancerConfig based on what was built
-     */
-    public abstract RebalancerConfig build();
-  }
-
-  /**
-   * Simple none-mode builder for rebalancer config
-   */
-  public static class SimpleBuilder extends Builder<SimpleBuilder> {
-    /**
-     * Construct with a resource-id
-     * @param resourceId
-     */
-    public SimpleBuilder(ResourceId resourceId) {
-      super(resourceId);
-    }
-
-    /**
-     * Construct with an existing rebalancer config
-     * @param config
-     */
-    public SimpleBuilder(RebalancerConfig config) {
-      super(config);
-    }
-
-    /**
-     * Build a rebalancer config
-     * @return
-     */
-    @Override
-    public RebalancerConfig build() {
-      RebalancerConfig config =
-          new RebalancerConfig(_resourceId, RebalanceMode.NONE, _stateModelDefId, _partitionMap);
-      update(config);
-      return config;
-    }
-
-    @Override
-    protected SimpleBuilder self() {
-      return this;
-    }
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/Resource.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/Resource.java b/helix-core/src/main/java/org/apache/helix/api/Resource.java
index eade077..b8a6daf 100644
--- a/helix-core/src/main/java/org/apache/helix/api/Resource.java
+++ b/helix-core/src/main/java/org/apache/helix/api/Resource.java
@@ -24,10 +24,11 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.helix.HelixConstants.StateModelToken;
+import org.apache.helix.api.ResourceConfig.ResourceType;
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.ResourceAssignment;
 
@@ -42,12 +43,16 @@ public class Resource {
    * Construct a resource
    * @param id resource id
    * @param idealState ideal state of the resource
-   * @param currentStateMap map of participant-id to current state
-   * @param liveParticipantCount number of live participants in the system
+   * @param externalView external view of the resource
+   * @param resourceAssignment current resource assignment of the cluster
+   * @param rebalancerContext contextual parameters that the rebalancer should be aware of
+   * @param userConfig any resource user-defined configuration
+   * @param bucketSize the bucket size to use for physically saved state
+   * @param batchMessageMode true if batch messaging allowed, false otherwise
    */
   public Resource(ResourceId id, IdealState idealState, ResourceAssignment resourceAssignment,
-      ExternalView externalView, UserConfig userConfig,
-      Map<PartitionId, UserConfig> partitionUserConfigs) {
+      ExternalView externalView, RebalancerContext rebalancerContext, UserConfig userConfig,
+      int bucketSize, boolean batchMessageMode) {
     Map<PartitionId, Partition> partitionMap = new HashMap<PartitionId, Partition>();
     new HashMap<PartitionId, Map<String, String>>();
     Set<PartitionId> partitionSet = idealState.getPartitionSet();
@@ -57,75 +62,16 @@ public class Resource {
         partitionSet.add(PartitionId.from(id, Integer.toString(i)));
       }
     }
-
     for (PartitionId partitionId : partitionSet) {
-      UserConfig partitionUserConfig = partitionUserConfigs.get(partitionId);
-      if (partitionUserConfig == null) {
-        partitionUserConfig = new UserConfig(Scope.partition(partitionId));
-      }
-      partitionMap.put(partitionId, new Partition(partitionId, partitionUserConfig));
-
-    }
-
-    String replicas = idealState.getReplicas();
-    boolean anyLiveParticipant = false;
-    int replicaCount = 0;
-    if (replicas.equals(StateModelToken.ANY_LIVEINSTANCE.toString())) {
-      anyLiveParticipant = true;
-    } else {
-      replicaCount = Integer.parseInt(replicas);
-    }
-
-    // Build a RebalancerConfig specific to the mode
-    RebalancerConfig rebalancerConfig = null;
-    if (idealState.getRebalanceMode() == RebalanceMode.FULL_AUTO) {
-      rebalancerConfig =
-          new FullAutoRebalancerConfig.Builder(id).addPartitions(partitionMap.values())
-              .anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
-              .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
-              .stateModelDef(idealState.getStateModelDefId())
-              .stateModelFactoryId(idealState.getStateModelFactoryId())
-              .participantGroupTag(idealState.getInstanceGroupTag()).build();
-    } else if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
-      SemiAutoRebalancerConfig semiAutoConfig =
-          new SemiAutoRebalancerConfig.Builder(id).addPartitions(partitionMap.values())
-              .anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
-              .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
-              .stateModelDef(idealState.getStateModelDefId())
-              .stateModelFactoryId(idealState.getStateModelFactoryId())
-              .participantGroupTag(idealState.getInstanceGroupTag()).build();
-      for (PartitionId partitionId : partitionMap.keySet()) {
-        semiAutoConfig.setPreferenceList(partitionId, idealState.getPreferenceList(partitionId));
-      }
-      rebalancerConfig = semiAutoConfig;
-    } else if (idealState.getRebalanceMode() == RebalanceMode.CUSTOMIZED) {
-      CustomRebalancerConfig customConfig =
-          new CustomRebalancerConfig.Builder(id).addPartitions(partitionMap.values())
-              .anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
-              .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
-              .stateModelDef(idealState.getStateModelDefId())
-              .stateModelFactoryId(idealState.getStateModelFactoryId())
-              .participantGroupTag(idealState.getInstanceGroupTag()).build();
-      for (PartitionId partitionId : partitionMap.keySet()) {
-        customConfig.setPreferenceMap(partitionId, idealState.getParticipantStateMap(partitionId));
-      }
-      rebalancerConfig = customConfig;
-    } else if (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED) {
-      rebalancerConfig =
-          new UserDefinedRebalancerConfig.Builder(id).addPartitions(partitionMap.values())
-              .anyLiveParticipant(anyLiveParticipant).replicaCount(replicaCount)
-              .maxPartitionsPerParticipant(idealState.getMaxPartitionsPerInstance())
-              .stateModelDef(idealState.getStateModelDefId())
-              .stateModelFactoryId(idealState.getStateModelFactoryId())
-              .rebalancerRef(idealState.getRebalancerRef())
-              .participantGroupTag(idealState.getInstanceGroupTag()).build();
+      partitionMap.put(partitionId, new Partition(partitionId));
     }
 
     SchedulerTaskConfig schedulerTaskConfig = schedulerTaskConfig(idealState);
+    RebalancerConfig rebalancerConfig = new RebalancerConfig(rebalancerContext);
 
     _config =
-        new ResourceConfig(id, schedulerTaskConfig, rebalancerConfig, userConfig,
-            idealState.getBucketSize(), idealState.getBatchMessageMode());
+        new ResourceConfig(id, ResourceType.DATA, schedulerTaskConfig, rebalancerConfig,
+            userConfig, bucketSize, batchMessageMode);
     _externalView = externalView;
   }
 
@@ -135,7 +81,9 @@ public class Resource {
    * @return scheduler-task config or null if state-model-def is not SchedulerTaskQueue
    */
   SchedulerTaskConfig schedulerTaskConfig(IdealState idealState) {
-
+    if (idealState == null) {
+      return null;
+    }
     // TODO refactor get timeout
     Map<String, Integer> transitionTimeoutMap = new HashMap<String, Integer>();
     for (String simpleKey : idealState.getRecord().getSimpleFields().keySet()) {
@@ -170,28 +118,28 @@ public class Resource {
   }
 
   /**
-   * Get the partitions of the resource
-   * @return map of partition id to partition or empty map if none
+   * Get the subunits of the resource
+   * @return map of subunit id to partition or empty map if none
    */
-  public Map<PartitionId, Partition> getPartitionMap() {
-    return _config.getPartitionMap();
+  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap() {
+    return _config.getSubUnitMap();
   }
 
   /**
-   * Get a partition that the resource contains
-   * @param partitionId the partition id to look up
+   * 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 getPartition(PartitionId partitionId) {
-    return _config.getPartition(partitionId);
+  public Partition getSubUnit(PartitionId subUnitId) {
+    return _config.getSubUnit(subUnitId);
   }
 
   /**
-   * Get the set of partition ids that the resource contains
-   * @return partition id set, or empty if none
+   * Get the set of subunit ids that the resource contains
+   * @return subunit id set, or empty if none
    */
-  public Set<PartitionId> getPartitionSet() {
-    return _config.getPartitionSet();
+  public Set<? extends PartitionId> getSubUnitSet() {
+    return _config.getSubUnitSet();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
index dd93d79..a21301b 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ResourceConfig.java
@@ -3,6 +3,9 @@ package org.apache.helix.api;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.controller.rebalancer.context.RebalancerConfig;
+import org.apache.helix.controller.rebalancer.context.RebalancerContext;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -26,12 +29,23 @@ import java.util.Set;
  * 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
@@ -43,10 +57,11 @@ public class ResourceConfig {
    * @param bucketSize bucket size for this resource
    * @param batchMessageMode whether or not batch messaging is allowed
    */
-  public ResourceConfig(ResourceId id, SchedulerTaskConfig schedulerTaskConfig,
-      RebalancerConfig rebalancerConfig, UserConfig userConfig, int bucketSize,
-      boolean batchMessageMode) {
+  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;
@@ -55,28 +70,28 @@ public class ResourceConfig {
   }
 
   /**
-   * Get the partitions of the resource
-   * @return map of partition id to partition or empty map if none
+   * Get the subunits of the resource
+   * @return map of subunit id to subunit or empty map if none
    */
-  public Map<PartitionId, Partition> getPartitionMap() {
-    return _rebalancerConfig.getPartitionMap();
+  public Map<? extends PartitionId, ? extends Partition> getSubUnitMap() {
+    return _rebalancerConfig.getRebalancerContext(RebalancerContext.class).getSubUnitMap();
   }
 
   /**
-   * Get a partition that the resource contains
-   * @param partitionId the partition id to look up
+   * 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 getPartition(PartitionId partitionId) {
-    return _rebalancerConfig.getPartition(partitionId);
+  public Partition getSubUnit(PartitionId subUnitId) {
+    return getSubUnitMap().get(subUnitId);
   }
 
   /**
-   * Get the set of partition ids that the resource contains
-   * @return partition id set, or empty if none
+   * Get the set of subunit ids that the resource contains
+   * @return subunit id set, or empty if none
    */
-  public Set<PartitionId> getPartitionSet() {
-    return _rebalancerConfig.getPartitionSet();
+  public Set<? extends PartitionId> getSubUnitSet() {
+    return getSubUnitMap().keySet();
   }
 
   /**
@@ -96,6 +111,14 @@ public class ResourceConfig {
   }
 
   /**
+   * Get the resource type
+   * @return ResourceType
+   */
+  public ResourceType getType() {
+    return _resourceType;
+  }
+
+  /**
    * Get the properties configuring scheduler tasks
    * @return SchedulerTaskConfig properties
    */
@@ -129,7 +152,7 @@ public class ResourceConfig {
 
   @Override
   public String toString() {
-    return _rebalancerConfig.getPartitionMap().toString();
+    return getSubUnitMap().toString();
   }
 
   /**
@@ -137,6 +160,7 @@ public class ResourceConfig {
    */
   public static class Builder {
     private final ResourceId _id;
+    private ResourceType _type;
     private RebalancerConfig _rebalancerConfig;
     private SchedulerTaskConfig _schedulerTaskConfig;
     private UserConfig _userConfig;
@@ -149,18 +173,29 @@ public class ResourceConfig {
      */
     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 rebalancerConfig properties of interest for rebalancing
      * @return Builder
      */
-    public Builder rebalancerConfig(RebalancerConfig rebalancerConfig) {
-      _rebalancerConfig = rebalancerConfig;
+    public Builder rebalancerContext(RebalancerContext rebalancerContext) {
+      _rebalancerConfig = new RebalancerConfig(rebalancerContext);
       return this;
     }
 
@@ -208,7 +243,7 @@ public class ResourceConfig {
      * @return instantiated Resource
      */
     public ResourceConfig build() {
-      return new ResourceConfig(_id, _schedulerTaskConfig, _rebalancerConfig, _userConfig,
+      return new ResourceConfig(_id, _type, _schedulerTaskConfig, _rebalancerConfig, _userConfig,
           _bucketSize, _batchMessageMode);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/ResourceId.java b/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
index 3df2ef0..dfffe5e 100644
--- a/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/ResourceId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -23,6 +26,7 @@ package org.apache.helix.api;
  * Identifies a resource
  */
 public class ResourceId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
   @Override
@@ -31,9 +35,11 @@ public class ResourceId extends Id {
   }
 
   /**
-   * @param id
+   * Create a resource id
+   * @param id string representation of a resource id
    */
-  private ResourceId(String id) {
+  @JsonCreator
+  public ResourceId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/SemiAutoRebalancerConfig.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SemiAutoRebalancerConfig.java b/helix-core/src/main/java/org/apache/helix/api/SemiAutoRebalancerConfig.java
deleted file mode 100644
index 409769e..0000000
--- a/helix-core/src/main/java/org/apache/helix/api/SemiAutoRebalancerConfig.java
+++ /dev/null
@@ -1,156 +0,0 @@
-package org.apache.helix.api;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.helix.model.IdealState;
-import org.apache.helix.model.IdealState.RebalanceMode;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Configuration properties for the SEMI_AUTO rebalancer
- */
-public final class SemiAutoRebalancerConfig extends RebalancerConfig {
-  /**
-   * Instantiate a new config for SEMI_AUTO
-   * @param resourceId the resource to rebalance
-   * @param stateModelDefId the state model that the resource follows
-   * @param partitionMap map of partition id to partition
-   */
-  public SemiAutoRebalancerConfig(ResourceId resourceId, StateModelDefId stateModelDefId,
-      Map<PartitionId, Partition> partitionMap) {
-    super(resourceId, RebalanceMode.SEMI_AUTO, stateModelDefId, partitionMap);
-  }
-
-  /**
-   * Instantiate from a base RebalancerConfig
-   * @param config populated rebalancer config
-   */
-  private SemiAutoRebalancerConfig(RebalancerConfig config) {
-    super(config);
-  }
-
-  /**
-   * Get the preference list for a partition
-   * @param partitionId the partition to look up
-   * @return preference list ordered from most preferred to least preferred
-   */
-  public List<ParticipantId> getPreferenceList(PartitionId partitionId) {
-    List<String> rawPreferenceList = getListField(partitionId.stringify());
-    if (rawPreferenceList != null) {
-      return IdealState.preferenceListFromStringList(rawPreferenceList);
-    }
-    return Collections.emptyList();
-  }
-
-  /**
-   * Set the preference list for a partition
-   * @param partitionId the partition to set
-   * @param preferenceList preference list ordered from most preferred to least preferred
-   */
-  public void setPreferenceList(PartitionId partitionId, List<ParticipantId> preferenceList) {
-    setListField(partitionId.toString(), IdealState.stringListFromPreferenceList(preferenceList));
-  }
-
-  /**
-   * Get all the preference lists for a partition
-   * @return map of partition id to list of participants ordered from most preferred to least
-   *         preferred
-   */
-  public Map<PartitionId, List<ParticipantId>> getPreferenceLists() {
-    Map<String, List<String>> rawPreferenceLists = getListFields();
-    return IdealState.preferenceListsFromStringLists(rawPreferenceLists);
-  }
-
-  /**
-   * Set all the preference lists for a partition
-   * @param preferenceLists map of partition id to list of participants ordered from most preferred
-   *          to least preferred
-   */
-  public void setPreferenceLists(Map<PartitionId, List<ParticipantId>> preferenceLists) {
-    setListFields(IdealState.stringListsFromPreferenceLists(preferenceLists));
-  }
-
-  /**
-   * Get a SemiAutoRebalancerConfig from a RebalancerConfig
-   * @param config populated RebalancerConfig
-   * @return SemiAutoRebalancerConfig
-   */
-  public static SemiAutoRebalancerConfig from(RebalancerConfig config) {
-    return new SemiAutoRebalancerConfig(config);
-  }
-
-  /**
-   * Assembler for a SEMI_AUTO configuration
-   */
-  public static class Builder extends RebalancerConfig.Builder<Builder> {
-    private final Map<PartitionId, List<ParticipantId>> _preferenceLists;
-
-    /**
-     * Build for a specific resource
-     * @param resourceId the resource to rebalance
-     */
-    public Builder(ResourceId resourceId) {
-      super(resourceId);
-      _preferenceLists = new HashMap<PartitionId, List<ParticipantId>>();
-    }
-
-    /**
-     * Construct a builder from an existing semi-auto rebalancer config
-     * @param config
-     */
-    public Builder(SemiAutoRebalancerConfig config) {
-    	super(config);
-        _preferenceLists = new HashMap<PartitionId, List<ParticipantId>>();
-        _preferenceLists.putAll(config.getPreferenceLists());
-    }
-    
-    /**
-     * Add a preference list of a partition
-     * @param partitionId the partition to set
-     * @param preferenceList list of participant ids, most preferred first
-     * @return Builder
-     */
-    public Builder preferenceList(PartitionId partitionId, List<ParticipantId> preferenceList) {
-      _preferenceLists.put(partitionId, preferenceList);
-      return this;
-    }
-
-    @Override
-    public SemiAutoRebalancerConfig build() {
-      if (_partitionMap.isEmpty()) {
-        addPartitions(1);
-      }
-      SemiAutoRebalancerConfig config =
-          new SemiAutoRebalancerConfig(_resourceId, _stateModelDefId, _partitionMap);
-      update(config);
-      config.setPreferenceLists(_preferenceLists);
-      return config;
-    }
-
-    @Override
-    protected Builder self() {
-      return this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/SessionId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SessionId.java b/helix-core/src/main/java/org/apache/helix/api/SessionId.java
index 777c506..df36193 100644
--- a/helix-core/src/main/java/org/apache/helix/api/SessionId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/SessionId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,9 +23,15 @@ package org.apache.helix.api;
  */
 
 public class SessionId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
-  private SessionId(String id) {
+  /**
+   * Create a session id
+   * @param id string representing a session id
+   */
+  @JsonCreator
+  public SessionId(@JsonProperty("id") String id) {
     _id = id;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java b/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
index ebfa845..851eb0d 100644
--- a/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/SpectatorId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -20,9 +23,15 @@ package org.apache.helix.api;
  */
 
 public class SpectatorId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
-  private SpectatorId(String id) {
+  /**
+   * Create a spectator id
+   * @param id string representing a spectator id
+   */
+  @JsonCreator
+  public SpectatorId(@JsonProperty("id") String id) {
     _id = id;
   }
 
@@ -31,4 +40,12 @@ public class SpectatorId extends Id {
     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/917af3eb/helix-core/src/main/java/org/apache/helix/api/State.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/State.java b/helix-core/src/main/java/org/apache/helix/api/State.java
index 5d0af41..3315987 100644
--- a/helix-core/src/main/java/org/apache/helix/api/State.java
+++ b/helix-core/src/main/java/org/apache/helix/api/State.java
@@ -1,6 +1,8 @@
 package org.apache.helix.api;
 
 import org.apache.helix.HelixDefinedState;
+import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonProperty;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -22,12 +24,18 @@ import org.apache.helix.HelixDefinedState;
  */
 
 /**
- *
+ * A state in a state model managed by Helix
  */
 public class State {
+  @JsonProperty("state")
   private final String _state;
 
-  public State(String state) {
+  /**
+   * Create a state
+   * @param state string representing a state
+   */
+  @JsonCreator
+  public State(@JsonProperty("state") String state) {
     _state = state.toUpperCase();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java b/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
index e4e05ba..50ea020 100644
--- a/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/StateModelDefId.java
@@ -1,6 +1,8 @@
 package org.apache.helix.api;
 
 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
@@ -24,9 +26,15 @@ import org.apache.helix.manager.zk.DefaultSchedulerMessageHandlerFactory;
 public class StateModelDefId extends Id {
   public static final StateModelDefId SchedulerTaskQueue = StateModelDefId
       .from(DefaultSchedulerMessageHandlerFactory.SCHEDULER_TASK_QUEUE);
+  @JsonProperty("id")
   private final String _id;
 
-  private StateModelDefId(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;
   }
 
@@ -35,6 +43,11 @@ public class StateModelDefId extends Id {
     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);
   }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/917af3eb/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java b/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
index fcd9f0d..422b4e9 100644
--- a/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
+++ b/helix-core/src/main/java/org/apache/helix/api/StateModelFactoryId.java
@@ -1,5 +1,8 @@
 package org.apache.helix.api;
 
+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
@@ -19,10 +22,19 @@ package org.apache.helix.api;
  * under the License.
  */
 
+/**
+ * Id representing a state model factory
+ */
 public class StateModelFactoryId extends Id {
+  @JsonProperty("id")
   private final String _id;
 
-  private StateModelFactoryId(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;
   }