You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ji...@apache.org on 2019/10/07 21:12:04 UTC

[helix] 01/37: Define the WAGED rebalancer interfaces.

This is an automated email from the ASF dual-hosted git repository.

jiajunwang pushed a commit to branch wagedRebalancer2
in repository https://gitbox.apache.org/repos/asf/helix.git

commit fd20c43acdae355731010dbca247ed8ca2826c81
Author: jiajunwang <er...@gmail.com>
AuthorDate: Thu Jul 18 14:50:20 2019 -0700

    Define the WAGED rebalancer interfaces.
    
    This is the intial check in for the future development of the WAGED rebalancer.
    All the components are placeholders. They will be implemented gradually.
---
 .../controller/rebalancer/GlobalRebalancer.java    | 67 ++++++++++++++++++++
 .../rebalancer/waged/AssignmentMetadataStore.java  | 51 +++++++++++++++
 .../rebalancer/waged/ClusterDataDetector.java      | 72 ++++++++++++++++++++++
 .../rebalancer/waged/ClusterDataProvider.java      | 53 ++++++++++++++++
 .../rebalancer/waged/RebalanceAlgorithm.java       | 45 ++++++++++++++
 .../rebalancer/waged/WagedRebalancer.java          | 58 +++++++++++++++++
 .../constraints/ConstraintsRebalanceAlgorithm.java | 49 +++++++++++++++
 .../waged/constraints/HardConstraint.java          | 50 +++++++++++++++
 .../waged/constraints/SoftConstraint.java          | 49 +++++++++++++++
 .../rebalancer/waged/model/AssignableNode.java     | 28 +++++++++
 .../rebalancer/waged/model/AssignableReplica.java  | 27 ++++++++
 .../rebalancer/waged/model/ClusterContext.java     | 27 ++++++++
 .../rebalancer/waged/model/ClusterModel.java       | 27 ++++++++
 13 files changed, 603 insertions(+)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/GlobalRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/GlobalRebalancer.java
new file mode 100644
index 0000000..a3b9b32
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/GlobalRebalancer.java
@@ -0,0 +1,67 @@
+package org.apache.helix.controller.rebalancer;
+
+/*
+ * 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.HelixManager;
+import org.apache.helix.controller.dataproviders.BaseControllerDataProvider;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Resource;
+
+import java.util.Map;
+
+public interface GlobalRebalancer<T extends BaseControllerDataProvider> {
+  enum RebalanceFailureType {
+    INVALID_CLUSTER_STATUS,
+    INVALID_REBALANCER_STATUS,
+    FAILED_TO_CALCULATE,
+    UNKNOWN_FAILURE
+  }
+
+  class RebalanceFailureReason {
+    private final static String DEFAULT_REASON_MESSAGE = "No detail";
+    private final RebalanceFailureType _type;
+    private final String _reason;
+
+    public RebalanceFailureReason(RebalanceFailureType type) {
+      this(type, DEFAULT_REASON_MESSAGE);
+    }
+
+    public RebalanceFailureReason(RebalanceFailureType type, String reason) {
+      _type = type;
+      _reason = reason;
+    }
+
+    public RebalanceFailureType get_type() {
+      return _type;
+    }
+
+    public String get_reason() {
+      return _reason;
+    }
+  }
+
+  void init(HelixManager manager);
+
+  Map<String, IdealState> computeNewIdealState(final CurrentStateOutput currentStateOutput,
+      T clusterData, Map<String, Resource> resourceMap);
+
+  RebalanceFailureReason getFailureReason();
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java
new file mode 100644
index 0000000..074ff4b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/AssignmentMetadataStore.java
@@ -0,0 +1,51 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+/*
+ * 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.model.IdealState;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A placeholder before we have the real assignment metadata store.
+ */
+public class AssignmentMetadataStore {
+  private Map<String, IdealState> _persistGlobalBaseline = new HashMap<>();
+  private Map<String, IdealState> _persistBestPossibleAssignment = new HashMap<>();
+
+  public Map<String, IdealState> getBaseline() {
+    return _persistGlobalBaseline;
+  }
+
+  public void persistBaseline(Map<String, IdealState> globalBaseline) {
+    // TODO clean up invalid items
+    _persistGlobalBaseline = globalBaseline;
+  }
+
+  public Map<String, IdealState> getBestPossibleAssignment() {
+    return _persistBestPossibleAssignment;
+  }
+
+  public void persistBestPossibleAssignment(Map<String, IdealState> bestPossibleAssignment) {
+    // TODO clean up invalid items
+    _persistBestPossibleAssignment.putAll(bestPossibleAssignment);
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
new file mode 100644
index 0000000..07f16dd
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
@@ -0,0 +1,72 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+/*
+ * 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.dataproviders.BaseControllerDataProvider;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A placeholder before we have the Cluster Data Detector implemented.
+ *
+ * @param <T> The cache class that can be handled by the detector.
+ */
+public class ClusterDataDetector<T extends BaseControllerDataProvider> {
+  /**
+   * All the cluster change type that may trigger a WAGED rebalancer re-calculation.
+   */
+  public enum ChangeType {
+    InstanceConfigChange,
+    ClusterConfigChange,
+    ResourceConfigChange,
+    InstanceStateChange,
+    ResourceIdealStatesChange,
+    OtherChange
+  }
+
+  private Map<ChangeType, Set<String>> _currentChanges =
+      Collections.singletonMap(ChangeType.ClusterConfigChange, Collections.emptySet());
+
+  public void updateClusterStatus(T cache) {
+  }
+
+  /**
+   * Returns all change types detected during the ClusterDetection stage.
+   */
+  public Set<ChangeType> getChangeTypes() {
+    return _currentChanges.keySet();
+  }
+
+  /**
+   * Returns a set of the names of components that changed based on the given change type.
+   */
+  public Set<String> getChangesBasedOnType(ChangeType changeType) {
+    return _currentChanges.get(changeType);
+  }
+
+  /**
+   * Return a map of the change details <type, change details>.
+   */
+  public Map<ChangeType, Set<String>> getAllChanges() {
+    return _currentChanges;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java
new file mode 100644
index 0000000..419be42
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java
@@ -0,0 +1,53 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+/*
+ * 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.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.model.IdealState;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A placeholder before we have the implementation.
+ *
+ * The data provider generates the Cluster Model based on the controller's data cache.
+ */
+public class ClusterDataProvider {
+
+  /**
+   * @param dataProvider           The controller's data cache.
+   * @param activeInstances        The logical active instances that will be used in the calculation. Note
+   *                               This list can be different from the real active node list according to
+   *                               the rebalancer logic.
+   * @param clusterChanges         All the cluster changes that happened after the previous rebalance.
+   * @param baselineAssignment     The persisted Baseline assignment.
+   * @param bestPossibleAssignment The persisted Best Possible assignment that was generated in the
+   *                               previous rebalance.
+   * @return The cluster model as the input for the upcoming rebalance.
+   */
+  protected static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
+      Set<String> activeInstances, Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
+      Map<String, IdealState> baselineAssignment, Map<String, IdealState> bestPossibleAssignment) {
+    // TODO finish the implementation.
+    return new ClusterModel();
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
new file mode 100644
index 0000000..0e6c891
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
@@ -0,0 +1,45 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+/*
+ * 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.waged.constraints.HardConstraint;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.model.IdealState;
+
+import java.util.Map;
+
+/**
+ * A generic rebalance algorithm interface for the WAGED rebalancer.
+ *
+ * @see <a href="Rebalance Algorithm">https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer#rebalance-algorithm-adapter</a>
+ */
+public interface RebalanceAlgorithm {
+
+  /**
+   * Rebalance the Helix resource partitions based on the input cluster model.
+   *
+   * @param clusterModel
+   * @param failureReasons Return the failures <ResourceName, <FailureReason, Count>> that happen during the rebalance calculation.
+   *                       If the map is null, no failure will be returned.
+   * @return A map <ResourceName, FailureReason> of the rebalanced resource assignments that are saved in the IdeaStates.
+   */
+  Map<String, IdealState> rebalance(ClusterModel clusterModel,
+      Map<String, Map<HardConstraint.FailureReason, Integer>> failureReasons);
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java
new file mode 100644
index 0000000..aa3cfee
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/WagedRebalancer.java
@@ -0,0 +1,58 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+/*
+ * 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.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.GlobalRebalancer;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Resource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A placeholder before we have the implementation.
+ * Weight-Aware Globally-Even Distribute Rebalancer.
+ *
+ * @see <a href="Design Document">https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer</a>
+ */
+public class WagedRebalancer implements GlobalRebalancer<ResourceControllerDataProvider> {
+  private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
+
+  @Override
+  public void init(HelixManager manager) { }
+
+  @Override
+  public Map<String, IdealState> computeNewIdealState(CurrentStateOutput currentStateOutput,
+      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
+      throws HelixException {
+    return new HashMap<>();
+  }
+
+  @Override
+  public RebalanceFailureReason getFailureReason() {
+    return new RebalanceFailureReason(RebalanceFailureType.UNKNOWN_FAILURE);
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java
new file mode 100644
index 0000000..292d903
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java
@@ -0,0 +1,49 @@
+package org.apache.helix.controller.rebalancer.waged.constraints;
+
+/*
+ * 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.waged.RebalanceAlgorithm;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.model.IdealState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A placeholder before we have the implementation.
+ * The constraint-based rebalance algorithm that is used in the WAGED rebalancer.
+ */
+public class ConstraintsRebalanceAlgorithm implements RebalanceAlgorithm {
+  private static final Logger LOG = LoggerFactory.getLogger(ConstraintsRebalanceAlgorithm.class);
+
+  private Map<HardConstraint.FailureReason, Integer> _failureReasonCounterMap = new HashMap<>();
+
+  public ConstraintsRebalanceAlgorithm() {
+    // TODO Constraints initialization
+  }
+
+  @Override
+  public Map<String, IdealState> rebalance(ClusterModel clusterModel,
+      Map<String, Map<HardConstraint.FailureReason, Integer>> failureReasons) {
+    return new HashMap<>();
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/HardConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/HardConstraint.java
new file mode 100644
index 0000000..3ee57ea
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/HardConstraint.java
@@ -0,0 +1,50 @@
+package org.apache.helix.controller.rebalancer.waged.constraints;
+
+/*
+ * 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.waged.model.AssignableNode;
+import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
+
+/**
+ * Evaluate a partition allocation proposal and return YES or NO based on the cluster context.
+ * Any proposal fails one or more hard constraints will be rejected.
+ */
+public interface HardConstraint {
+  enum FailureReason {
+    FAULT_ZONES_CONTAIN_SAME_PARTITION,
+    NODES_DEACTIVATED,
+    NODES_NO_TAG,
+    NODES_EXCEED_MAX_PARTITION,
+    NODES_INSUFFICIENT_RESOURCE,
+    NODES_CONTAIN_SAME_PARTITION,
+  }
+
+  /**
+   * @return True if the proposed assignment is valid.
+   */
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica rep,
+      ClusterContext clusterContext);
+
+  /**
+   * @return Detail of the reason that the proposed assignment was rejected.
+   */
+  FailureReason getFailureReason();
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraint.java
new file mode 100644
index 0000000..bce4a5a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraint.java
@@ -0,0 +1,49 @@
+package org.apache.helix.controller.rebalancer.waged.constraints;
+
+/*
+ * 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.waged.model.AssignableNode;
+import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
+
+/**
+ * Evaluate a partition allocation proposal and return a score within the normalized range.
+ * A higher score means the proposal is more preferred.
+ */
+public interface SoftConstraint {
+  float MIN_SCORE = -1000.0f;
+  float MAX_SCORE = 1000.0f;
+
+  /**
+   * The scoring function returns a score between MINIMAL_SCORE and MAXIMUM_SCORE, which is then weighted by the
+   * individual normalized constraint weights.
+   * Each individual constraint will define the meaning of MINIMAL_SCORE to MAXIMUM_SCORE differently.
+   */
+  float assignmentScore(AssignableNode node, AssignableReplica rep, ClusterContext clusterContext);
+
+  /**
+   * Set the importance factor of the soft constraint.
+   * The more important it is, the more contribution it will make to the final evaluation.
+   * @param importance
+   */
+  void setConstraintImportance(float importance);
+
+  float getConstraintImportance();
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableNode.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableNode.java
new file mode 100644
index 0000000..ae037f4
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableNode.java
@@ -0,0 +1,28 @@
+package org.apache.helix.controller.rebalancer.waged.model;
+
+/*
+ * 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.
+ */
+
+/**
+ * A placeholder before we have the implementation.
+ *
+ * This class represents a potential allocation of the replication.
+ * Note that AssignableNode is not thread safe.
+ */
+public class AssignableNode { }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableReplica.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableReplica.java
new file mode 100644
index 0000000..a6a7e4a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/AssignableReplica.java
@@ -0,0 +1,27 @@
+package org.apache.helix.controller.rebalancer.waged.model;
+
+/*
+ * 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.
+ */
+
+/**
+ * A placeholder before we have the implementation.
+ *
+ * This class represents a partition replication that needs to be allocated.
+ */
+public class AssignableReplica { }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterContext.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterContext.java
new file mode 100644
index 0000000..adca7d1
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterContext.java
@@ -0,0 +1,27 @@
+package org.apache.helix.controller.rebalancer.waged.model;
+
+/*
+ * 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.
+ */
+
+/**
+ * A placeholder before we have the implementation.
+ *
+ * This class tracks the global rebalance-related status of a Helix managed cluster.
+ */
+public class ClusterContext { }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModel.java
new file mode 100644
index 0000000..06eebf7
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModel.java
@@ -0,0 +1,27 @@
+package org.apache.helix.controller.rebalancer.waged.model;
+
+/*
+ * 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.
+ */
+
+/**
+ * A placeholder before we have the implementation.
+ *
+ * This class wraps the required input for the rebalance algorithm.
+ */
+public class ClusterModel { }