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:17:13 UTC

[helix] branch wagedRebalancer updated (7c6f1be -> 4353f67)

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

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


    omit 7c6f1be  Add latency metric components for WAGED rebalancer (#490)
    omit be1f041  Load soft constraint weight from resources/properties file (#492)
    omit 3ac3b22  Adjust the topology processing logic for instance to ensure backward compatibility.
    omit aae2e93  Enable maintenance mode for the WAGED rebalancer.
    omit 1b4457d  Add delayed rebalance and user-defined preference list features to the WAGED rebalancer. (#456)
    omit 88b1f80  Separate AssignableNode properties by Immutable and Mutable (#485)
    omit 935360e  Integrate the WAGED rebalancer with all the related components. (#466)
    omit 718ea9c  Add the remaining implementation of ConstraintBasedAlgorithmFactory (#478)
    omit e13ab51  PartitionMovementSoftConstraint Implementation (#474)
    omit 951d3bf  Adjust the expected replica count according to fault zone count. (#476)
    omit 7be94fe  Improve ResourceTopStateAntiAffinityConstraint (#475)
    omit e99e358  Add soft constraint: ResourcePartitionAntiAffinityConstraint (#464)
    omit 81fd8ce  Implement MaxCapacityUsageInstanceConstraint soft constraint (#463)
    omit 8d2f752  Add soft constraint: ResourcetopStateAntiAffinityConstraint (#465)
    omit cbd295a  Implement one of the soft constraints (#450)
    omit 7cb8083  Fix TestWagedRebalancer and add constructor in AssignmentMetadataStore
    omit 884dc1d  Implement AssignmentMetadataStore (#453)
    omit c40f15b  HardConstraints Implementation and unit tests (#433)
    omit e654f6b  Implement the WAGED rebalancer with the limited functionality. (#443)
    omit 0fa347d  Validate the instance capacity/partition weight configuration while constructing the assignable instances (#451)
    omit fae0eb1  Implement the basic constraint based algorithm (#381)
    omit 31a1875  Add BucketDataAccessor for large writes
    omit fd18212  Record the replica objects in the AssignableNode in addition to the partition name (#440)
    omit 7938b73  Add special treatment for ClusterConfig
    omit 28e6fdd  Modify the expected change type from CONFIG to CLUSTER_CONFIG in the WAGED rebalancer. (#438)
    omit e6edcd9  Resubmit the change: Refine the WAGED rebalancer related interfaces for integration (#431)
    omit 6db8387  Revert "Refine the WAGED rebalancer related interfaces for integration (#431)" (#437)
    omit 08a2015  Refine the WAGED rebalancer related interfaces for integration (#431)
    omit 672c9cb  Redefine the hard/soft constraints (#422)
    omit 55e21b8  Add ChangeDetector interface and ResourceChangeDetector implementation (#388)
    omit 4c96562  Add cluster level default instance config. (#413)
    omit b552731  Implement Cluster Model Provider. (#392)
    omit 05e360d  Convert all the internal assignment state objects to be ResourceAssignment. (#399)
    omit 19a6ff4  Change the rebalancer assignment record to be ResourceAssignment instead of IdealState. (#398)
    omit d2a801a  Implement the WAGED rebalancer cluster model (#362)
    omit 65d26aa  Adding the configuration items of the WAGED rebalancer. (#348)
    omit 5d1d770  Define the WAGED rebalancer interfaces.
     add 22e4ec7  Fix the execution delay for the jobs
     add 9434cf1  Use the system property value as the sliding window length. #382
     add cc73afd  Use the system property value as the sliding window length. #382
     add d30fefb  Use the system property value as the sliding window length. #382
     add 51abb53  Add a unit test case.. #382
     add 6fb58a3  Fix a typo. #382
     add b6aee9f  Fix a typo. #382
     add ab5ec69  Make the reservoir sliding window length used in Helix monintor metrics configurable. #382
     add 0f16687  Make the reservoir sliding window length used in Helix monintor metrics configurable. #382
     add 955e566  Make the reservoir sliding window length used in Helix monintor metrics configurable. #382
     add ec8ba98  Make the reservoir sliding window length used in Helix monintor metrics configurable. #382
     add d0422b7  TASK: Fix incorrect counting of numAttempts for tasks (#432)
     add 33aa50c  Add integration test for workflow ForceDelete
     add cb8db7b  remove all unused imports
     add c08f9d3  TASK: Fix forceDelete for jobs in JobQueue
     add cfd270f  Fix the order of workflow context update
     add 58a8ff8  Fix the issue where JobContext is not updated properly (#435)
     add 61f7380  Make State Transition Throttling respect MIN_ACTIVE_REPLICA
     add 8e5b0ec  Add field for MIN_ACTIVE_REPLICA_NOT_SET
     add 5c8c4af  Fix helix-front build failure by downgrading types/lodash version. (#470)
     add 71becb3  Fix missed callbacks in CurrentStates based RoutingTableProvider. (#458)
     add 6197c71  Fix CustomRebalancer's assignment computation (#477)
     add bf85e79  [helix-rest] Delete unused default namespace (api "/namespaces/default") (#449)
     add afcfe80  Filter instances of weight = 0 for any partition assignment (#369)
     add e4f3da4  Change the way Helix triggers rebalance (#472)
     add 690e630  Add Intellij code style XML file for Helix code style. (#481)
     add 3d1729b  Make the Java Doc for API more clear
     add bb198fc  #493 Set jersey servlet application name with namespace name.
     add cdbdd4f  Add unit test for setting application name.
     new fd20c43  Define the WAGED rebalancer interfaces.
     new 39f7adf  Adding the configuration items of the WAGED rebalancer. (#348)
     new 730054b  Implement the WAGED rebalancer cluster model (#362)
     new 60dc54a  Change the rebalancer assignment record to be ResourceAssignment instead of IdealState. (#398)
     new 912ed34  Convert all the internal assignment state objects to be ResourceAssignment. (#399)
     new dc828b8  Implement Cluster Model Provider. (#392)
     new 0d67764  Add cluster level default instance config. (#413)
     new 84ac35b  Add ChangeDetector interface and ResourceChangeDetector implementation (#388)
     new ae282a8  Redefine the hard/soft constraints (#422)
     new c3d8501  Refine the WAGED rebalancer related interfaces for integration (#431)
     new c65476d  Revert "Refine the WAGED rebalancer related interfaces for integration (#431)" (#437)
     new 1ed9bbf  Resubmit the change: Refine the WAGED rebalancer related interfaces for integration (#431)
     new 2780fd9  Modify the expected change type from CONFIG to CLUSTER_CONFIG in the WAGED rebalancer. (#438)
     new eb7a010  Add special treatment for ClusterConfig
     new 9ee65a0  Record the replica objects in the AssignableNode in addition to the partition name (#440)
     new 34f8260  Add BucketDataAccessor for large writes
     new 9f73d8d  Implement the basic constraint based algorithm (#381)
     new a4b61ee  Validate the instance capacity/partition weight configuration while constructing the assignable instances (#451)
     new 702c547  Implement the WAGED rebalancer with the limited functionality. (#443)
     new 43d3476  HardConstraints Implementation and unit tests (#433)
     new 3d2da32  Implement AssignmentMetadataStore (#453)
     new c719273  Fix TestWagedRebalancer and add constructor in AssignmentMetadataStore
     new 06cb692  Implement one of the soft constraints (#450)
     new 153d385  Add soft constraint: ResourcetopStateAntiAffinityConstraint (#465)
     new 52e1ba5  Implement MaxCapacityUsageInstanceConstraint soft constraint (#463)
     new e3cb1a6  Add soft constraint: ResourcePartitionAntiAffinityConstraint (#464)
     new eb95a19  Improve ResourceTopStateAntiAffinityConstraint (#475)
     new 679547f  Adjust the expected replica count according to fault zone count. (#476)
     new 50e8cec  PartitionMovementSoftConstraint Implementation (#474)
     new 86f1a00  Add the remaining implementation of ConstraintBasedAlgorithmFactory (#478)
     new 5c6f1e9  Integrate the WAGED rebalancer with all the related components. (#466)
     new 0b87426  Separate AssignableNode properties by Immutable and Mutable (#485)
     new 88ff705  Add delayed rebalance and user-defined preference list features to the WAGED rebalancer. (#456)
     new 1712a88  Enable maintenance mode for the WAGED rebalancer.
     new 6f812d7  Adjust the topology processing logic for instance to ensure backward compatibility.
     new 75b60d1  Load soft constraint weight from resources/properties file (#492)
     new 4353f67  Add latency metric components for WAGED rebalancer (#490)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (7c6f1be)
            \
             N -- N -- N   refs/heads/wagedRebalancer (4353f67)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 37 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../helix/webapp/resources/ConfigResource.java     |   1 -
 .../webapp/resources/SchedulerTasksResource.java   |   1 -
 .../org/apache/helix/webapp/AdminTestHelper.java   |   4 -
 .../helix/webapp/TestClusterManagementWebapp.java  |   1 -
 .../helix/webapp/resources/TestJsonParameters.java |   2 -
 .../helix/webapp/resources/TestResourceUtil.java   |   2 -
 .../apache/helix/agent/ProcessMonitorThread.java   |   1 -
 .../main/java/org/apache/helix/HelixProperty.java  |   1 -
 .../helix/common/caches/BasicClusterDataCache.java | 116 ++++--
 .../helix/common/caches/InstanceMessagesCache.java |  12 +-
 .../helix/controller/GenericHelixController.java   |  43 +-
 .../ResourceControllerDataProvider.java            |   1 -
 .../WorkflowControllerDataProvider.java            |   1 -
 .../controller/pipeline/AbstractBaseStage.java     |   1 -
 .../controller/rebalancer/CustomRebalancer.java    |  17 +-
 .../AbstractEvenDistributionRebalanceStrategy.java |  26 +-
 .../CardDealingAdjustmentAlgorithmV2.java          |  25 +-
 .../rebalancer/util/RebalanceScheduler.java        |   9 +-
 .../stages/BestPossibleStateCalcStage.java         |   2 +-
 .../stages/IntermediateStateCalcStage.java         |   7 +-
 .../apache/helix/examples/BootstrapProcess.java    |   1 -
 .../helix/manager/zk/ControllerManagerHelper.java  |   1 -
 .../zk/DefaultControllerMessageHandlerFactory.java |   1 -
 ...faultParticipantErrorMessageHandlerFactory.java |   1 -
 .../org/apache/helix/manager/zk/ZKHelixAdmin.java  |   1 -
 .../manager/zk/ZNRecordStreamingSerializer.java    |   3 -
 .../zk/serializer/JacksonPayloadSerializer.java    |   1 -
 .../org/apache/helix/messaging/AsyncCallback.java  |   1 -
 .../HelixStateTransitionCancellationHandler.java   |   1 -
 .../main/java/org/apache/helix/model/TaskSMD.java  |   1 -
 .../monitoring/mbeans/ClusterEventMonitor.java     |   2 +-
 .../monitoring/mbeans/ClusterMBeanObserver.java    |   1 -
 .../monitoring/mbeans/HelixCallbackMonitor.java    |   2 +-
 .../apache/helix/monitoring/mbeans/JobMonitor.java |  20 +-
 .../monitoring/mbeans/MessageLatencyMonitor.java   |   2 +-
 .../helix/monitoring/mbeans/ResourceMonitor.java   |   6 +-
 .../mbeans/RoutingTableProviderMonitor.java        |   7 +-
 .../mbeans/StateTransitionStatMonitor.java         |   7 +-
 .../mbeans/ThreadPoolExecutorMonitor.java          |   1 -
 .../monitoring/mbeans/ZkClientPathMonitor.java     |  10 +-
 .../helix/participant/StateMachineEngine.java      |   1 -
 .../apache/helix/spectator/RoutingDataCache.java   |   1 -
 .../helix/spectator/RoutingTableProvider.java      |  17 +-
 .../apache/helix/task/AbstractTaskDispatcher.java  |  37 +-
 .../helix/task/AssignableInstanceManager.java      |   1 -
 .../task/FixedTargetTaskAssignmentCalculator.java  |   1 -
 .../main/java/org/apache/helix/task/JobDag.java    |   1 -
 .../java/org/apache/helix/task/JobDispatcher.java  |  32 +-
 .../java/org/apache/helix/task/TaskDriver.java     |  84 ++--
 .../main/java/org/apache/helix/task/TaskUtil.java  |   4 +-
 .../ThreadCountBasedTaskAssignmentCalculator.java  |   1 -
 .../org/apache/helix/task/WorkflowDispatcher.java  |  75 ++--
 .../org/apache/helix/task/WorkflowRebalancer.java  |   1 -
 .../java/org/apache/helix/tools/ClusterSetup.java  |   1 -
 .../apache/helix/tools/ClusterStateVerifier.java   |   1 -
 .../ClusterVerifiers/ZkHelixClusterVerifier.java   |   4 -
 .../tools/commandtools/CurrentStateCleanUp.java    |   4 -
 .../tools/commandtools/ExampleParticipant.java     |   3 -
 .../tools/commandtools/IntegrationTestUtil.java    |   1 -
 .../main/java/org/apache/helix/util/HelixUtil.java |   4 +-
 .../java/org/apache/helix/util/RebalanceUtil.java  |  23 ++
 .../helix/common/caches/TestPropertyCache.java     |   2 -
 .../TestWorkflowControllerDataProvider.java        |   1 -
 .../helix/controller/stages/BaseStageTest.java     |  17 +-
 .../controller/stages/TestRecoveryLoadBalance.java |  34 +-
 .../helix/controller/strategy/TestTopology.java    |   1 +
 .../TestCardDealingAdjustmentAlgorithmV2.java      | 455 +++++++++++++++++++++
 .../TestAddNodeAfterControllerStart.java           |   2 -
 .../helix/integration/TestDisablePartition.java    |   1 -
 .../helix/integration/TestEnableCompression.java   |   1 -
 .../helix/integration/TestErrorReplicaPersist.java |   3 -
 .../integration/TestPreferenceListAsQueue.java     |   1 -
 .../integration/TestSyncSessionToController.java   |   1 -
 .../integration/common/ZkStandAloneCMTestBase.java |   2 -
 .../integration/manager/TestZkHelixAdmin.java      |   1 -
 .../rebalancer/TestCustomRebalancer.java           |  81 ++++
 .../rebalancer/TestFullAutoNodeTagging.java        |   1 -
 .../TestRoutingTableProviderFromCurrentStates.java |   4 -
 .../task/TestDeleteJobFromJobQueue.java            |  83 ++++
 .../helix/integration/task/TestExecutionDelay.java | 132 ++++++
 .../integration/task/TestForceDeleteWorkflow.java  | 371 +++++++++++++++++
 .../task/TestStopWorkflowWithExecutionDelay.java   |  93 +++++
 .../integration/task/TestTaskConditionalRetry.java |   4 +-
 .../integration/task/TestTaskNumAttempts.java      |  79 ++++
 .../task/TestTaskRebalancerRetryLimit.java         |   5 +-
 .../helix/integration/task/TestTaskRetryDelay.java |   4 +-
 .../helix/integration/task/TestTaskStopQueue.java  | 113 +++++
 .../apache/helix/manager/zk/TestRawZkClient.java   |   1 -
 .../helix/manager/zk/TestZKLiveInstanceData.java   |   1 -
 .../helix/manager/zk/TestZNRecordSerializer.java   |   1 -
 .../helix/mock/controller/MockController.java      |   1 -
 .../org/apache/helix/model/TestConstraint.java     |   2 -
 .../mbeans/TestDropResourceMetricsReset.java       |   1 -
 .../monitoring/mbeans/TestZkClientMonitor.java     |  27 ++
 .../participant/TestDistControllerElection.java    |   2 -
 .../assigner/TestThreadCountBasedTaskAssigner.java |   2 -
 .../apache/helix/util/TestRebalanceScheduler.java  |   2 -
 .../TestRecoveryLoadBalance.MasterSlave.json       | 112 ++++-
 helix-front/package-lock.json                      |   6 +-
 helix-front/package.json                           |   2 +-
 .../apache/helix/rest/server/HelixRestServer.java  |  10 +-
 .../rest/server/resources/helix/JobAccessor.java   |   2 +-
 .../server/resources/helix/WorkflowAccessor.java   |   1 -
 .../helix/rest/server/TestClusterAccessor.java     |   3 -
 .../rest/server/TestDefaultMonitoringMbeans.java   |  32 ++
 .../helix/rest/server/TestHelixRestServer.java     |   2 +-
 .../helix/rest/server/TestNamespacedAPIAccess.java |  50 +--
 helix-style-intellij.xml                           | 387 ++++++++++++++++++
 .../apache/helix/recipes/rabbitmq/Consumer.java    |   1 -
 .../apache/helix/filestore/ChangeLogGenerator.java |   2 -
 .../apache/helix/filestore/ChangeLogProcessor.java |   6 -
 .../apache/helix/filestore/ChangeLogReader.java    |   2 -
 .../java/org/apache/helix/filestore/FileStore.java |   5 -
 .../main/java/org/apache/helix/filestore/Test.java |   3 -
 .../helix/servicediscovery/ServiceDiscovery.java   |   1 -
 .../apache/helix/taskexecution/TaskStateModel.java |   2 -
 116 files changed, 2451 insertions(+), 350 deletions(-)
 create mode 100644 helix-core/src/test/java/org/apache/helix/controller/strategy/crushMapping/TestCardDealingAdjustmentAlgorithmV2.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestCustomRebalancer.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestDeleteJobFromJobQueue.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestExecutionDelay.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestForceDeleteWorkflow.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestStopWorkflowWithExecutionDelay.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestTaskNumAttempts.java
 create mode 100644 helix-core/src/test/java/org/apache/helix/integration/task/TestTaskStopQueue.java
 create mode 100644 helix-style-intellij.xml


[helix] 30/37: Add the remaining implementation of ConstraintBasedAlgorithmFactory (#478)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 86f1a000c2742ecca2a0dfec36f2f04525175dfc
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Tue Sep 17 15:41:08 2019 -0700

    Add the remaining implementation of ConstraintBasedAlgorithmFactory (#478)
    
    Implementation of ConstraintBasedAlgorithmFactory and the soft constraint weight model.
    Remove SoftConstraintWeightModel class.
    Get the rebalance preference and adjust the corresponding weight.
    Pass the preference keys instead of cluster config.
---
 .../rebalancer/waged/WagedRebalancer.java          | 19 ++++---
 .../constraints/ConstraintBasedAlgorithm.java      | 21 +++++---
 .../ConstraintBasedAlgorithmFactory.java           | 38 ++++++++++----
 .../constraints/SoftConstraintWeightModel.java     | 58 ----------------------
 .../stages/BestPossibleStateCalcStage.java         |  5 +-
 .../constraints/TestConstraintBasedAlgorithm.java  |  8 ++-
 6 files changed, 58 insertions(+), 91 deletions(-)

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
index 22cac7e..551239d 100644
--- 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
@@ -20,7 +20,6 @@ package org.apache.helix.controller.rebalancer.waged;
  */
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -29,7 +28,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
@@ -42,6 +40,7 @@ import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModelProvider;
 import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
 import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
@@ -49,6 +48,9 @@ import org.apache.helix.model.ResourceAssignment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
+
 /**
  * Weight-Aware Globally-Even Distribute Rebalancer.
  * @see <a
@@ -62,10 +64,8 @@ public class WagedRebalancer {
   // When any of the following change happens, the rebalancer needs to do a global rebalance which
   // contains 1. baseline recalculate, 2. partial rebalance that is based on the new baseline.
   private static final Set<HelixConstants.ChangeType> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES =
-      Collections
-          .unmodifiableSet(new HashSet<>(Arrays.asList(HelixConstants.ChangeType.RESOURCE_CONFIG,
-              HelixConstants.ChangeType.CLUSTER_CONFIG,
-              HelixConstants.ChangeType.INSTANCE_CONFIG)));
+      ImmutableSet.of(HelixConstants.ChangeType.RESOURCE_CONFIG,
+          HelixConstants.ChangeType.CLUSTER_CONFIG, HelixConstants.ChangeType.INSTANCE_CONFIG);
   // The cluster change detector is a stateful object.
   // Make it static to avoid unnecessary reinitialization.
   private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
@@ -79,13 +79,12 @@ public class WagedRebalancer {
   private final RebalanceAlgorithm _rebalanceAlgorithm;
   // ------------------------------------------------------------------------------------//
 
-  public WagedRebalancer(HelixManager helixManager) {
+  public WagedRebalancer(HelixManager helixManager,
+      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
     this(
         // TODO init the metadata store according to their requirement when integrate,
         // or change to final static method if possible.
-        new AssignmentMetadataStore(helixManager),
-        // TODO parse the cluster setting
-        ConstraintBasedAlgorithmFactory.getInstance(),
+        new AssignmentMetadataStore(helixManager), ConstraintBasedAlgorithmFactory.getInstance(preferences),
         // Use DelayedAutoRebalancer as the mapping calculator for the final assignment output.
         // Mapping calculator will translate the best possible assignment into the applicable state
         // mapping based on the current states.
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
index 479fb78..89a3f29 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
@@ -53,14 +53,12 @@ import com.google.common.collect.Maps;
 class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
   private static final Logger LOG = LoggerFactory.getLogger(ConstraintBasedAlgorithm.class);
   private final List<HardConstraint> _hardConstraints;
-  private final List<SoftConstraint> _softConstraints;
-  private final SoftConstraintWeightModel _softConstraintsWeightModel;
+  private final Map<SoftConstraint, Float> _softConstraints;
 
   ConstraintBasedAlgorithm(List<HardConstraint> hardConstraints,
-      List<SoftConstraint> softConstraints, SoftConstraintWeightModel softConstraintWeightModel) {
+      Map<SoftConstraint, Float> softConstraints) {
     _hardConstraints = hardConstraints;
     _softConstraints = softConstraints;
-    _softConstraintsWeightModel = softConstraintWeightModel;
   }
 
   @Override
@@ -115,13 +113,22 @@ class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
     }
 
     Function<AssignableNode, Float> calculatePoints =
-        (candidateNode) -> _softConstraintsWeightModel.getSumOfScores(_softConstraints.stream()
-            .collect(Collectors.toMap(Function.identity(), softConstraint -> softConstraint
-                .getAssignmentNormalizedScore(candidateNode, replica, clusterContext))));
+        (candidateNode) -> getAssignmentNormalizedScore(candidateNode, replica, clusterContext);
 
     return candidateNodes.stream().max(Comparator.comparing(calculatePoints));
   }
 
+  private float getAssignmentNormalizedScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    float sum = 0;
+    for (Map.Entry<SoftConstraint, Float> softConstraintEntry : _softConstraints.entrySet()) {
+      SoftConstraint softConstraint = softConstraintEntry.getKey();
+      float weight = softConstraintEntry.getValue();
+      sum += weight * softConstraint.getAssignmentNormalizedScore(node, replica, clusterContext);
+    }
+    return sum;
+  }
+
   private List<String> convertFailureReasons(List<HardConstraint> hardConstraints) {
     return hardConstraints.stream().map(HardConstraint::getDescription)
         .collect(Collectors.toList());
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
index 895fa61..8568444 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
@@ -19,23 +19,41 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
  * under the License.
  */
 
-import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
 import org.apache.helix.model.ClusterConfig;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * The factory class to create an instance of {@link ConstraintBasedAlgorithm}
+ */
 public class ConstraintBasedAlgorithmFactory {
 
-  // TODO: the parameter comes from cluster config, will tune how these 2 integers will change the
-  // soft constraint weight model
-  public static RebalanceAlgorithm getInstance() {
-    // TODO initialize constraints, depending on constraints implementations PRs
-    List<HardConstraint> hardConstraints = new ArrayList<>();
-    List<SoftConstraint> softConstraints = new ArrayList<>();
-    SoftConstraintWeightModel softConstraintWeightModel = new SoftConstraintWeightModel();
+  public static RebalanceAlgorithm getInstance(
+      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
+    List<HardConstraint> hardConstraints =
+        ImmutableList.of(new FaultZoneAwareConstraint(), new NodeCapacityConstraint(),
+            new ReplicaActivateConstraint(), new NodeMaxPartitionLimitConstraint(),
+            new ValidGroupTagConstraint(), new SamePartitionOnInstanceConstraint());
+
+    int evennessPreference =
+        preferences.getOrDefault(ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS, 1);
+    int movementPreference =
+        preferences.getOrDefault(ClusterConfig.GlobalRebalancePreferenceKey.LESS_MOVEMENT, 1);
+    float evennessRatio = (float) evennessPreference / (evennessPreference + movementPreference);
+    float movementRatio = (float) movementPreference / (evennessPreference + movementPreference);
+
+    Map<SoftConstraint, Float> softConstraints = ImmutableMap.<SoftConstraint, Float> builder()
+        .put(new PartitionMovementConstraint(), movementRatio)
+        .put(new InstancePartitionsCountConstraint(), 0.3f * evennessRatio)
+        .put(new ResourcePartitionAntiAffinityConstraint(), 0.1f * evennessRatio)
+        .put(new ResourceTopStateAntiAffinityConstraint(), 0.1f * evennessRatio)
+        .put(new MaxCapacityUsageInstanceConstraint(), 0.5f * evennessRatio).build();
 
-    return new ConstraintBasedAlgorithm(hardConstraints, softConstraints,
-        softConstraintWeightModel);
+    return new ConstraintBasedAlgorithm(hardConstraints, softConstraints);
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
deleted file mode 100644
index 953005c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
+++ /dev/null
@@ -1,58 +0,0 @@
-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 java.util.Map;
-
-import com.google.common.collect.ImmutableMap;
-
-/**
- * The class retrieves the offline model that defines the relative importance of soft constraints.
- */
-class SoftConstraintWeightModel {
-  private static Map<Class, Float> MODEL;
-
-  // TODO either define the weights in property files or zookeeper node or static human input
-  SoftConstraintWeightModel() {
-
-  }
-
-  static {
-    // TODO update the weight
-    MODEL = ImmutableMap.<Class, Float> builder().put(InstancePartitionsCountConstraint.class, 1.0f)
-        .build();
-  }
-
-  /**
-   * Get the sum of normalized scores, given calculated scores map of soft constraints
-   * @param originScoresMap The origin scores by soft constraints
-   * @return The sum of soft constraints scores
-   */
-  float getSumOfScores(Map<SoftConstraint, Float> originScoresMap) {
-    float sum = 0;
-    for (Map.Entry<SoftConstraint, Float> softConstraintScoreEntry : originScoresMap.entrySet()) {
-      SoftConstraint softConstraint = softConstraintScoreEntry.getKey();
-      float weight = MODEL.get(softConstraint.getClass());
-      sum += softConstraintScoreEntry.getValue() * weight;
-    }
-
-    return sum;
-  }
-}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index ba4da88..806ef85 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -40,6 +40,7 @@ import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
+import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.MaintenanceSignal;
@@ -121,7 +122,9 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     // configured to use the WAGED rebalancer.
     // For the other resources, the legacy rebalancers will be triggered in the next step.
     Map<String, IdealState> newIdealStates = new HashMap<>();
-    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences = cache.getClusterConfig()
+        .getGlobalRebalancePreference();
+    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager, preferences);
     try {
       newIdealStates
           .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, currentStateOutput));
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
index 0e61eb3..b2deaef 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
@@ -34,6 +34,7 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 
 public class TestConstraintBasedAlgorithm {
   private ConstraintBasedAlgorithm _algorithm;
@@ -42,12 +43,11 @@ public class TestConstraintBasedAlgorithm {
   public void beforeMethod() {
     HardConstraint mockHardConstraint = mock(HardConstraint.class);
     SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
-    SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
     when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(false);
     when(mockSoftConstraint.getAssignmentNormalizedScore(any(), any(), any())).thenReturn(1.0f);
 
     _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
-        ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
+        ImmutableMap.of(mockSoftConstraint, 1f));
   }
 
   @Test(expectedExceptions = HelixRebalanceException.class)
@@ -60,12 +60,10 @@ public class TestConstraintBasedAlgorithm {
   public void testCalculateWithValidAssignment() throws IOException, HelixRebalanceException {
     HardConstraint mockHardConstraint = mock(HardConstraint.class);
     SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
-    SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
     when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(true);
     when(mockSoftConstraint.getAssignmentNormalizedScore(any(), any(), any())).thenReturn(1.0f);
-    when(mockSoftConstraintWeightModel.getSumOfScores(any())).thenReturn(1.0f);
     _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
-        ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
+        ImmutableMap.of(mockSoftConstraint, 1f));
     ClusterModel clusterModel = new ClusterModelTestHelper().getDefaultClusterModel();
     OptimalAssignment optimalAssignment = _algorithm.calculate(clusterModel);
 


[helix] 12/37: Resubmit the change: Refine the WAGED rebalancer related interfaces for integration (#431)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1ed9bbffb0dffa99984d408d9d7968699840bbd6
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Wed Aug 28 22:28:10 2019 -0700

    Resubmit the change: Refine the WAGED rebalancer related interfaces for integration (#431)
    
    * Refine the WAGED rebalancer related interfaces and initial integrate with the BestPossibleStateCalStage.
    
    - Modify the BestPossibleStateCalStage logic to plugin the WAGED rebalancer.
    - Refine ClusterModel to integrate with the ClusterDataDetector implementation.
    - Enabling getting the changed details for Cluster Config in the change detector. Which is required by the WAGED rebalancer.
    
    * Bring back the interface class and algorithm placeholder class that was removed prematurely.
---
 .../org/apache/helix/HelixRebalanceException.java  | 43 ++++++++++
 .../changedetector/ResourceChangeDetector.java     | 20 +++--
 .../controller/rebalancer/GlobalRebalancer.java    | 67 ---------------
 .../rebalancer/waged/ClusterDataDetector.java      | 73 ----------------
 .../rebalancer/waged/ClusterDataProvider.java      | 54 ------------
 .../rebalancer/waged/WagedRebalancer.java          | 66 +++++++++++----
 .../constraints/ConstraintsRebalanceAlgorithm.java | 48 +++++++++++
 .../waged/constraints/RebalanceAlgorithm.java      | 44 ++++++++++
 .../waged/model/ClusterModelProvider.java          | 25 +++---
 .../stages/BestPossibleStateCalcStage.java         | 98 ++++++++++++++++------
 .../waged/model/TestClusterModelProvider.java      |  6 +-
 11 files changed, 289 insertions(+), 255 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
new file mode 100644
index 0000000..c01b173
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
@@ -0,0 +1,43 @@
+package org.apache.helix;
+
+/*
+ * 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.
+ */
+
+/**
+ * Exception thrown by Helix due to rebalance failures.
+ */
+public class HelixRebalanceException extends Exception {
+  enum RebalanceFailureType {
+    INVALID_CLUSTER_STATUS,
+    INVALID_REBALANCER_STATUS,
+    FAILED_TO_CALCULATE,
+    UNKNOWN_FAILURE
+  }
+
+  private final RebalanceFailureType _type;
+
+  public HelixRebalanceException(String message, RebalanceFailureType type, Throwable cause) {
+    super(String.format("%s. Failure Type: %s", message, type.name()), cause);
+    _type = type;
+  }
+
+  public RebalanceFailureType getFailureType() {
+    return _type;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
index d65e609..611f4b2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
@@ -20,15 +20,17 @@ package org.apache.helix.controller.changedetector;
  */
 
 import com.google.common.collect.Sets;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
-import org.apache.helix.HelixConstants;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixProperty;
-import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 
 /**
  * ResourceChangeDetector implements ChangeDetector. It caches resource-related metadata from
@@ -37,6 +39,7 @@ import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
  * WARNING: the methods of this class are not thread-safe.
  */
 public class ResourceChangeDetector implements ChangeDetector {
+  private static final Logger LOG = LoggerFactory.getLogger(ResourceChangeDetector.class.getName());
 
   private ResourceChangeSnapshot _oldSnapshot; // snapshot for previous pipeline run
   private ResourceChangeSnapshot _newSnapshot; // snapshot for this pipeline run
@@ -108,10 +111,13 @@ public class ResourceChangeDetector implements ChangeDetector {
       return snapshot.getResourceConfigMap();
     case LIVE_INSTANCE:
       return snapshot.getLiveInstances();
+    case CONFIG:
+      return Collections.emptyMap();
     default:
-      throw new HelixException(String.format(
-          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: %s",
-          changeType));
+      LOG.warn(
+          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: {}",
+          changeType);
+      return Collections.emptyMap();
     }
   }
 
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
deleted file mode 100644
index a3b9b32..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/GlobalRebalancer.java
+++ /dev/null
@@ -1,67 +0,0 @@
-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/ClusterDataDetector.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
deleted file mode 100644
index 0423edf..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
+++ /dev/null
@@ -1,73 +0,0 @@
-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 {
-    BaselineAssignmentChange,
-    InstanceConfigChange,
-    ClusterConfigChange,
-    ResourceConfigChange,
-    ResourceIdealStatesChange,
-    InstanceStateChange,
-    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
deleted file mode 100644
index 387666c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java
+++ /dev/null
@@ -1,54 +0,0 @@
-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.ResourceAssignment;
-
-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, ResourceAssignment> baselineAssignment,
-      Map<String, ResourceAssignment> bestPossibleAssignment) {
-    // TODO finish the implementation.
-    return null;
-  }
-}
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
index aa3cfee..43b2564 100644
--- 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
@@ -19,10 +19,14 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.changedetector.ResourceChangeDetector;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.GlobalRebalancer;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintsRebalanceAlgorithm;
+import org.apache.helix.controller.rebalancer.waged.constraints.RebalanceAlgorithm;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Resource;
@@ -36,23 +40,57 @@ 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>
+ * @see <a href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
+ * Design Document
+ * </a>
  */
-public class WagedRebalancer implements GlobalRebalancer<ResourceControllerDataProvider> {
+public class WagedRebalancer {
   private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
 
-  @Override
-  public void init(HelixManager manager) { }
+  // --------- The following fields are placeholders and need replacement. -----------//
+  // TODO Shall we make the metadata store a static threadlocal object as well to avoid reinitialization?
+  private final AssignmentMetadataStore _assignmentMetadataStore;
+  private final RebalanceAlgorithm _rebalanceAlgorithm;
+  // ------------------------------------------------------------------------------------//
 
-  @Override
-  public Map<String, IdealState> computeNewIdealState(CurrentStateOutput currentStateOutput,
-      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
-      throws HelixException {
-    return new HashMap<>();
+  // The cluster change detector is a stateful object. Make it static to avoid unnecessary
+  // reinitialization.
+  private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
+      new ThreadLocal<>();
+  private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
+
+  private ResourceChangeDetector getChangeDetector() {
+    if (CHANGE_DETECTOR_THREAD_LOCAL.get() == null) {
+      CHANGE_DETECTOR_THREAD_LOCAL.set(new ResourceChangeDetector());
+    }
+    return CHANGE_DETECTOR_THREAD_LOCAL.get();
+  }
+
+  public WagedRebalancer(HelixManager helixManager) {
+    // TODO init the metadata store according to their requirement when integrate, or change to final static method if possible.
+    _assignmentMetadataStore = new AssignmentMetadataStore();
+    // TODO init the algorithm according to the requirement when integrate.
+    _rebalanceAlgorithm = new ConstraintsRebalanceAlgorithm();
+
+    // Use the mapping calculator in DelayedAutoRebalancer for calculating the final assignment
+    // output.
+    // This calculator will translate the best possible assignment into an applicable state mapping
+    // based on the current states.
+    // TODO abstract and separate the mapping calculator logic from the DelayedAutoRebalancer
+    _mappingCalculator = new DelayedAutoRebalancer();
   }
 
-  @Override
-  public RebalanceFailureReason getFailureReason() {
-    return new RebalanceFailureReason(RebalanceFailureType.UNKNOWN_FAILURE);
+  /**
+   * Compute the new IdealStates for all the resources input. The IdealStates include both the new
+   * partition assignment (in the listFiles) and the new replica state mapping (in the mapFields).
+   * @param clusterData        The Cluster status data provider.
+   * @param resourceMap        A map containing all the rebalancing resources.
+   * @param currentStateOutput The present Current State of the cluster.
+   * @return A map containing the computed new IdealStates.
+   */
+  public Map<String, IdealState> computeNewIdealStates(ResourceControllerDataProvider clusterData,
+      Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
+      throws HelixRebalanceException {
+    return new HashMap<>();
   }
 }
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..8adaa73
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java
@@ -0,0 +1,48 @@
+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.ClusterModel;
+import org.apache.helix.model.ResourceAssignment;
+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, Integer> _failureReasonCounterMap = new HashMap<>();
+
+  public ConstraintsRebalanceAlgorithm() {
+    // TODO Constraints initialization
+  }
+
+  @Override
+  public Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
+      Map<String, Map<HardConstraint, Integer>> failureReasons) {
+    return new HashMap<>();
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java
new file mode 100644
index 0000000..b652836
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java
@@ -0,0 +1,44 @@
+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.ClusterModel;
+import org.apache.helix.model.ResourceAssignment;
+
+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 of <ResourceName, ResourceAssignment>.
+   */
+  Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
+      Map<String, Map<HardConstraint, Integer>> failureReasons);
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 9de023b..c4f7d02 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -19,9 +19,9 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -59,7 +59,7 @@ public class ClusterModelProvider {
    */
   public static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
       Map<String, Resource> resourceMap, Set<String> activeInstances,
-      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
       Map<String, ResourceAssignment> baselineAssignment,
       Map<String, ResourceAssignment> bestPossibleAssignment) {
     // Generate replica objects for all the resource partitions.
@@ -108,14 +108,13 @@ public class ClusterModelProvider {
    */
   private static Set<AssignableReplica> findToBeAssignedReplicas(
       Map<String, Set<AssignableReplica>> replicaMap,
-      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
       Map<String, ResourceAssignment> bestPossibleAssignment,
       Map<String, Set<AssignableReplica>> allocatedReplicas) {
     Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
-    if (clusterChanges.containsKey(ClusterDataDetector.ChangeType.ClusterConfigChange)
-        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.InstanceConfigChange)
-        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.BaselineAssignmentChange)) {
-      // If the cluster topology or baseline assignment has been modified, need to reassign all replicas
+    if (clusterChanges.containsKey(HelixConstants.ChangeType.CONFIG)
+        || clusterChanges.containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
+      // If the cluster topology has been modified, need to reassign all replicas
       toBeAssignedReplicas
           .addAll(replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()));
     } else {
@@ -124,11 +123,13 @@ public class ClusterModelProvider {
         Set<AssignableReplica> replicas = replicaMap.get(resourceName);
         // 1. if the resource config/idealstate is changed, need to reassign.
         // 2. if the resource does appear in the best possible assignment, need to reassign.
-        if (clusterChanges.getOrDefault(ClusterDataDetector.ChangeType.ResourceConfigChange,
-            Collections.emptySet()).contains(resourceName) || clusterChanges
-            .getOrDefault(ClusterDataDetector.ChangeType.ResourceIdealStatesChange,
-                Collections.emptySet()).contains(resourceName) || !bestPossibleAssignment
-            .containsKey(resourceName)) {
+        if (clusterChanges
+            .getOrDefault(HelixConstants.ChangeType.RESOURCE_CONFIG, Collections.emptySet())
+            .contains(resourceName)
+            || clusterChanges
+            .getOrDefault(HelixConstants.ChangeType.IDEAL_STATE, Collections.emptySet())
+            .contains(resourceName)
+            || !bestPossibleAssignment.containsKey(resourceName)) {
           toBeAssignedReplicas.addAll(replicas);
           continue; // go to check next resource
         } else {
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 49a72e0..ba4da88 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
@@ -37,6 +39,7 @@ import org.apache.helix.controller.rebalancer.MaintenanceRebalancer;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.MaintenanceSignal;
@@ -56,18 +59,19 @@ import org.slf4j.LoggerFactory;
  * IdealState,StateModel,LiveInstance
  */
 public class BestPossibleStateCalcStage extends AbstractBaseStage {
-  private static final Logger logger = LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
+  private static final Logger logger =
+      LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
-    CurrentStateOutput currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.name());
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
     final Map<String, Resource> resourceMap =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     final ClusterStatusMonitor clusterStatusMonitor =
         event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
     if (currentStateOutput == null || resourceMap == null || cache == null) {
       throw new StageException(
@@ -90,8 +94,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
                     resourceMap, stateModelDefMap);
           }
         } catch (Exception e) {
-          LogUtil
-              .logError(logger, _eventId, "Could not update cluster status metrics!", e);
+          LogUtil.logError(logger, _eventId, "Could not update cluster status metrics!", e);
         }
         return null;
       }
@@ -100,7 +103,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
   private BestPossibleStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
       CurrentStateOutput currentStateOutput) {
-    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
     BestPossibleStateOutput output = new BestPossibleStateOutput();
 
     HelixManager helixManager = event.getAttribute(AttributeName.helixmanager.name());
@@ -112,19 +116,50 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     boolean isValid = validateOfflineInstancesLimit(cache,
         (HelixManager) event.getAttribute(AttributeName.helixmanager.name()));
 
+    // 1. Rebalance with the WAGED rebalancer
+    // The rebalancer only calculates the new ideal assignment for all the resources that are
+    // configured to use the WAGED rebalancer.
+    // For the other resources, the legacy rebalancers will be triggered in the next step.
+    Map<String, IdealState> newIdealStates = new HashMap<>();
+    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
+    try {
+      newIdealStates
+          .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, currentStateOutput));
+    } catch (HelixRebalanceException ex) {
+      // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
+      // Since it calculates for all the eligible resources globally, a partial result is invalid.
+      // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
+      LogUtil.logError(logger, _eventId, String
+          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
+              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
+    }
+
     final List<String> failureResources = new ArrayList<>();
     Iterator<Resource> itr = resourceMap.values().iterator();
     while (itr.hasNext()) {
       Resource resource = itr.next();
       boolean result = false;
-      try {
-        result =
-            computeResourceBestPossibleState(event, cache, currentStateOutput, resource, output);
-      } catch (HelixException ex) {
-        LogUtil.logError(logger, _eventId,
-            "Exception when calculating best possible states for " + resource.getResourceName(),
-            ex);
-
+      IdealState is = newIdealStates.get(resource.getResourceName());
+      if (is != null) {
+        // 2. Check if the WAGED rebalancer has calculated for this resource or not.
+        result = checkBestPossibleStateCalculation(is);
+        if (result) {
+          // The WAGED rebalancer calculates a valid result, record in the output
+          updateBestPossibleStateOutput(output, resource, is);
+        }
+      } else {
+        // 3. The WAGED rebalancer skips calculating the resource assignment, fallback to use a
+        // legacy resource rebalancer if applicable.
+        // If this calculation fails, the resource will be reported in the failureResources list.
+        try {
+          result =
+              computeSingleResourceBestPossibleState(event, cache, currentStateOutput, resource,
+                  output);
+        } catch (HelixException ex) {
+          LogUtil.logError(logger, _eventId,
+              "Exception when calculating best possible states for " + resource.getResourceName(),
+              ex);
+        }
       }
       if (!result) {
         failureResources.add(resource.getResourceName());
@@ -185,8 +220,9 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
         if (manager != null) {
           if (manager.getHelixDataAccessor()
               .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
-            manager.getClusterManagmentTool().autoEnableMaintenanceMode(manager.getClusterName(),
-                true, errMsg, MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
+            manager.getClusterManagmentTool()
+                .autoEnableMaintenanceMode(manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
             LogUtil.logWarn(logger, _eventId, errMsg);
           }
         } else {
@@ -199,8 +235,19 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     return true;
   }
 
-  private boolean computeResourceBestPossibleState(ClusterEvent event, ResourceControllerDataProvider cache,
-      CurrentStateOutput currentStateOutput, Resource resource, BestPossibleStateOutput output) {
+  private void updateBestPossibleStateOutput(BestPossibleStateOutput output, Resource resource,
+      IdealState computedIdealState) {
+    output.setPreferenceLists(resource.getResourceName(), computedIdealState.getPreferenceLists());
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> newStateMap =
+          computedIdealState.getInstanceStateMap(partition.getPartitionName());
+      output.setState(resource.getResourceName(), partition, newStateMap);
+    }
+  }
+
+  private boolean computeSingleResourceBestPossibleState(ClusterEvent event,
+      ResourceControllerDataProvider cache, CurrentStateOutput currentStateOutput,
+      Resource resource, BestPossibleStateOutput output) {
     // for each ideal state
     // read the state model def
     // for each resource
@@ -229,12 +276,13 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
     Rebalancer<ResourceControllerDataProvider> rebalancer =
         getRebalancer(idealState, resourceName, cache.isMaintenanceModeEnabled());
-    MappingCalculator<ResourceControllerDataProvider> mappingCalculator = getMappingCalculator(rebalancer, resourceName);
+    MappingCalculator<ResourceControllerDataProvider> mappingCalculator =
+        getMappingCalculator(rebalancer, resourceName);
 
     if (rebalancer == null || mappingCalculator == null) {
-      LogUtil.logError(logger, _eventId,
-          "Error computing assignment for resource " + resourceName + ". no rebalancer found. rebalancer: " + rebalancer
-              + " mappingCalculator: " + mappingCalculator);
+      LogUtil.logError(logger, _eventId, "Error computing assignment for resource " + resourceName
+          + ". no rebalancer found. rebalancer: " + rebalancer + " mappingCalculator: "
+          + mappingCalculator);
     }
 
     if (rebalancer != null && mappingCalculator != null) {
@@ -299,8 +347,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     }
   }
 
-  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState, String resourceName,
-      boolean isMaintenanceModeEnabled) {
+  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState,
+      String resourceName, boolean isMaintenanceModeEnabled) {
     Rebalancer<ResourceControllerDataProvider> customizedRebalancer = null;
     String rebalancerClassName = idealState.getRebalancerClassName();
     if (rebalancerClassName != null) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index f92a66c..1221b6f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -19,8 +19,8 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixConstants;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
@@ -177,7 +177,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     // 5. test with best possible assignment but cluster topology is changed
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ClusterConfigChange,
+        _instances, Collections.singletonMap(HelixConstants.ChangeType.CONFIG,
             Collections.emptySet()), Collections.emptyMap(), bestPossibleAssignment);
     // There should be no existing assignment since the topology change invalidates all existing assignment
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
@@ -194,7 +194,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     String changedResourceName = _resourceNames.get(0);
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ResourceConfigChange,
+        _instances, Collections.singletonMap(HelixConstants.ChangeType.RESOURCE_CONFIG,
             Collections.singleton(changedResourceName)), Collections.emptyMap(),
         bestPossibleAssignment);
     // There should be no existing assignment for all the resource except for resource2.


[helix] 35/37: Adjust the topology processing logic for instance to ensure backward compatibility.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6f812d70b2024fc1c47f9de1242c25be7d4888a8
Author: jiajunwang <jj...@linkedin.com>
AuthorDate: Tue Oct 1 13:49:33 2019 -0700

    Adjust the topology processing logic for instance to ensure backward compatibility.
---
 .../rebalancer/waged/model/AssignableNode.java     | 52 +++++++++-------------
 .../rebalancer/waged/model/TestAssignableNode.java | 15 ++++---
 2 files changed, 28 insertions(+), 39 deletions(-)

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
index 2a68e15..3bfd225 100644
--- 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
@@ -28,16 +28,14 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import org.apache.helix.HelixException;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.InstanceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
 /**
  * This class represents a possible allocation of the replication.
  * Note that any usage updates to the AssignableNode are not thread safe.
@@ -138,8 +136,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
     }
 
     Map<String, AssignableReplica> partitionMap = _currentAssignedReplicaMap.get(resourceName);
-    if (!partitionMap.containsKey(partitionName)
-        || !partitionMap.get(partitionName).equals(replica)) {
+    if (!partitionMap.containsKey(partitionName) || !partitionMap.get(partitionName)
+        .equals(replica)) {
       LOG.warn("Replica {} is not assigned to node {}. Ignore the release call.",
           replica.toString(), getInstanceName());
       return;
@@ -269,10 +267,9 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * For example, when
    * the domain is "zone=2, instance=testInstance" and the fault zone type is "zone", this function
    * returns "2".
-   * If cannot find the fault zone id, this function leaves the fault zone id as the instance name.
-   * TODO merge this logic with Topology.java tree building logic.
-   * For now, the WAGED rebalancer has a more strict topology def requirement.
-   * Any missing field will cause an invalid topology config exception.
+   * If cannot find the fault zone type, this function leaves the fault zone id as the instance name.
+   * Note the WAGED rebalancer does not require full topology tree to be created. So this logic is
+   * simpler than the CRUSH based rebalancer.
    */
   private String computeFaultZone(ClusterConfig clusterConfig, InstanceConfig instanceConfig) {
     if (!clusterConfig.isTopologyAwareEnabled()) {
@@ -290,36 +287,27 @@ public class AssignableNode implements Comparable<AssignableNode> {
       return zoneId == null ? instanceConfig.getInstanceName() : zoneId;
     } else {
       // Get the fault zone information from the complete topology definition.
-      String[] topologyDef = topologyStr.trim().split("/");
-      if (topologyDef.length == 0
-          || Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
+      String[] topologyKeys = topologyStr.trim().split("/");
+      if (topologyKeys.length == 0 || Arrays.stream(topologyKeys)
+          .noneMatch(type -> type.equals(faultZoneType))) {
         throw new HelixException(
             "The configured topology definition is empty or does not contain the fault zone type.");
       }
 
       Map<String, String> domainAsMap = instanceConfig.getDomainAsMap();
-      if (domainAsMap == null) {
-        throw new HelixException(
-            String.format("The domain configuration of node %s is not configured", _instanceName));
-      } else {
-        StringBuilder faultZoneStringBuilder = new StringBuilder();
-        for (String key : topologyDef) {
-          if (!key.isEmpty()) {
-            if (domainAsMap.containsKey(key)) {
-              faultZoneStringBuilder.append(domainAsMap.get(key));
-              faultZoneStringBuilder.append('/');
-            } else {
-              throw new HelixException(String.format(
-                  "The domain configuration of node %s is not complete. Type %s is not found.",
-                  _instanceName, key));
-            }
-            if (key.equals(faultZoneType)) {
-              break;
-            }
+      StringBuilder faultZoneStringBuilder = new StringBuilder();
+      for (String key : topologyKeys) {
+        if (!key.isEmpty()) {
+          // if a key does not exist in the instance domain config, apply the default domain value.
+          faultZoneStringBuilder.append(domainAsMap.getOrDefault(key, "Default_" + key));
+          if (key.equals(faultZoneType)) {
+            break;
+          } else {
+            faultZoneStringBuilder.append('/');
           }
         }
-        return faultZoneStringBuilder.toString();
       }
+      return faultZoneStringBuilder.toString();
     }
   }
 
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index b48587f..e8b010e 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -189,24 +189,25 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     assignableNode.assign(duplicateReplica);
   }
 
-  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "The domain configuration of node testInstanceId is not complete. Type DOES_NOT_EXIST is not found.")
+  @Test
   public void testParseFaultZoneNotFound() throws IOException {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
 
     ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
-    testClusterConfig.setFaultZoneType("DOES_NOT_EXIST");
+    testClusterConfig.setFaultZoneType("zone");
     testClusterConfig.setTopologyAwareEnabled(true);
-    testClusterConfig.setTopology("/DOES_NOT_EXIST/");
+    testClusterConfig.setTopology("/zone/");
     when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
 
     InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
-    testInstanceConfig.setDomain("zone=2, instance=testInstance");
+    testInstanceConfig.setDomain("instance=testInstance");
     Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
     instanceConfigMap.put(_testInstanceId, testInstanceConfig);
     when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
 
-    new AssignableNode(testCache.getClusterConfig(),
+    AssignableNode node = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
+    Assert.assertEquals(node.getFaultZone(), "Default_zone");
   }
 
   @Test
@@ -228,7 +229,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
 
-    Assert.assertEquals(assignableNode.getFaultZone(), "2/");
+    Assert.assertEquals(assignableNode.getFaultZone(), "2");
 
     testClusterConfig = new ClusterConfig("testClusterConfigId");
     testClusterConfig.setFaultZoneType("instance");
@@ -245,7 +246,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
 
-    Assert.assertEquals(assignableNode.getFaultZone(), "2/testInstance/");
+    Assert.assertEquals(assignableNode.getFaultZone(), "2/testInstance");
   }
 
   @Test


[helix] 14/37: Add special treatment for ClusterConfig

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit eb7a0107629c09d6276a35e122c9d3b7049d0f51
Author: Hunter Lee <na...@gmail.com>
AuthorDate: Thu Aug 29 12:40:05 2019 -0700

    Add special treatment for ClusterConfig
    
    This diff allows callers of getChangeType to iterate over the result of getChangeType() by changing determinePropertyMapByType so that it just returns an empty map for ClusterConfig.
---
 .../helix/controller/changedetector/ResourceChangeDetector.java    | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
index 611f4b2..a154af2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
@@ -111,11 +111,14 @@ public class ResourceChangeDetector implements ChangeDetector {
       return snapshot.getResourceConfigMap();
     case LIVE_INSTANCE:
       return snapshot.getLiveInstances();
-    case CONFIG:
+    case CLUSTER_CONFIG:
+      // In the case of ClusterConfig, we return an empty map
+      // This is to allow the caller to iterate on the change types without throwing an exception or
+      // leaving a warn log for ClusterConfig changes
       return Collections.emptyMap();
     default:
       LOG.warn(
-          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: {}",
+          "ResourceChangeDetector cannot determine propertyMap for the given ChangeType: {}. Returning an empty map.",
           changeType);
       return Collections.emptyMap();
     }


[helix] 22/37: Fix TestWagedRebalancer and add constructor in AssignmentMetadataStore

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c719273daf4236e2a803bb0d99724e5a843f5512
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Tue Sep 10 09:42:23 2019 -0700

    Fix TestWagedRebalancer and add constructor in AssignmentMetadataStore
    
    TestWagedRebalancer was failing because it was not using a proper HelixManager to instantiate a mock version of AssignmentMetadataStore. This diff refactors the constructors in AssignmentMetadataStore and fixes the failing test.
---
 .../rebalancer/waged/AssignmentMetadataStore.java  | 18 +++--
 .../waged/MockAssignmentMetadataStore.java         | 10 +--
 .../rebalancer/waged/TestWagedRebalancer.java      | 93 +++++++++++++---------
 3 files changed, 71 insertions(+), 50 deletions(-)

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
index bf9f292..fd655d1 100644
--- 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
@@ -21,6 +21,8 @@ package org.apache.helix.controller.rebalancer.waged;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.BucketDataAccessor;
 import org.apache.helix.HelixException;
@@ -31,9 +33,6 @@ import org.apache.helix.manager.zk.ZNRecordJacksonSerializer;
 import org.apache.helix.manager.zk.ZkBucketDataAccessor;
 import org.apache.helix.model.ResourceAssignment;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * A placeholder before we have the real assignment metadata store.
  */
@@ -51,12 +50,15 @@ public class AssignmentMetadataStore {
   private Map<String, ResourceAssignment> _globalBaseline;
   private Map<String, ResourceAssignment> _bestPossibleAssignment;
 
+  AssignmentMetadataStore(BucketDataAccessor bucketDataAccessor, String clusterName) {
+    _dataAccessor = bucketDataAccessor;
+    _baselinePath = String.format(BASELINE_TEMPLATE, clusterName, ASSIGNMENT_METADATA_KEY);
+    _bestPossiblePath = String.format(BEST_POSSIBLE_TEMPLATE, clusterName, ASSIGNMENT_METADATA_KEY);
+  }
+
   AssignmentMetadataStore(HelixManager helixManager) {
-    _dataAccessor = new ZkBucketDataAccessor(helixManager.getMetadataStoreConnectionString());
-    _baselinePath =
-        String.format(BASELINE_TEMPLATE, helixManager.getClusterName(), ASSIGNMENT_METADATA_KEY);
-    _bestPossiblePath = String.format(BEST_POSSIBLE_TEMPLATE, helixManager.getClusterName(),
-        ASSIGNMENT_METADATA_KEY);
+    this(new ZkBucketDataAccessor(helixManager.getMetadataStoreConnectionString()),
+        helixManager.getClusterName());
   }
 
   public Map<String, ResourceAssignment> getBaseline() {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
index 8b80f2d..3371c8b 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
@@ -19,11 +19,10 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
-import org.apache.helix.HelixManager;
-import org.apache.helix.model.ResourceAssignment;
-
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.helix.BucketDataAccessor;
+import org.apache.helix.model.ResourceAssignment;
 
 /**
  * A mock up metadata store for unit test.
@@ -33,9 +32,8 @@ public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
   private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
   private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
 
-  public MockAssignmentMetadataStore() {
-    // In-memory mock component, so pass null for HelixManager since it's not needed
-    super(null);
+  MockAssignmentMetadataStore(BucketDataAccessor bucketDataAccessor, String clusterName) {
+    super(bucketDataAccessor, clusterName);
   }
 
   public Map<String, ResourceAssignment> getBaseline() {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
index 6759a10..d6fd99b 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
@@ -1,5 +1,24 @@
 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 java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
@@ -7,7 +26,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-
+import org.apache.helix.BucketDataAccessor;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
@@ -28,13 +47,13 @@ import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.when;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
 
 public class TestWagedRebalancer extends AbstractTestClusterModel {
   private Set<String> _instances;
   private MockRebalanceAlgorithm _algorithm;
+  private MockAssignmentMetadataStore _metadataStore;
 
   @BeforeClass
   public void initialize() {
@@ -42,6 +61,11 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     _instances = new HashSet<>();
     _instances.add(_testInstanceId);
     _algorithm = new MockRebalanceAlgorithm();
+
+    // Initialize a mock assignment metadata store
+    BucketDataAccessor mockAccessor = Mockito.mock(BucketDataAccessor.class);
+    String clusterName = ""; // an empty string for testing purposes
+    _metadataStore = new MockAssignmentMetadataStore(mockAccessor, clusterName);
   }
 
   @Override
@@ -88,9 +112,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   @Test
   public void testRebalance() throws IOException, HelixRebalanceException {
-    // Init mock metadatastore for the unit test
-    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -111,9 +134,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   @Test(dependsOnMethods = "testRebalance")
   public void testPartialRebalance() throws IOException, HelixRebalanceException {
-    // Init mock metadatastore for the unit test
-    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -127,7 +149,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
     // Test with partial resources listed in the resourceMap input.
     // Remove the first resource from the input. Note it still exists in the cluster data cache.
-    metadataStore.clearMetadataStore();
+    _metadataStore.clearMetadataStore();
     resourceMap.remove(_resourceNames.get(0));
     Map<String, IdealState> newIdealStates =
         rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
@@ -137,9 +159,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   @Test(dependsOnMethods = "testRebalance")
   public void testRebalanceWithCurrentState() throws IOException, HelixRebalanceException {
-    // Init mock metadatastore for the unit test
-    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -160,9 +181,9 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         String resourceName = csEntry.getKey();
         CurrentState cs = csEntry.getValue();
         for (Map.Entry<String, String> partitionStateEntry : cs.getPartitionStateMap().entrySet()) {
-          currentStateOutput
-              .setCurrentState(resourceName, new Partition(partitionStateEntry.getKey()),
-                  instanceName, partitionStateEntry.getValue());
+          currentStateOutput.setCurrentState(resourceName,
+              new Partition(partitionStateEntry.getKey()), instanceName,
+              partitionStateEntry.getValue());
         }
       }
     }
@@ -197,7 +218,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   @Test(dependsOnMethods = "testRebalance")
   public void testNonCompatibleConfiguration() throws IOException, HelixRebalanceException {
-    WagedRebalancer rebalancer = new WagedRebalancer(new MockAssignmentMetadataStore(), _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     String nonCompatibleResourceName = _resourceNames.get(0);
@@ -222,7 +244,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   // TODO test with invalid capacity configuration which will fail the cluster model constructing.
   @Test(dependsOnMethods = "testRebalance")
   public void testInvalidClusterStatus() throws IOException {
-    WagedRebalancer rebalancer = new WagedRebalancer(new MockAssignmentMetadataStore(), _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     String invalidResource = _resourceNames.get(0);
@@ -270,8 +293,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     when(badAlgorithm.calculate(any())).thenThrow(new HelixRebalanceException("Algorithm fails.",
         HelixRebalanceException.Type.FAILED_TO_CALCULATE));
 
-    WagedRebalancer rebalancer =
-        new WagedRebalancer(new MockAssignmentMetadataStore(), badAlgorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, badAlgorithm);
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     Map<String, Resource> resourceMap = clusterData.getIdealStates().keySet().stream().collect(
@@ -294,9 +317,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     // Note that this test relies on the MockRebalanceAlgorithm implementation. The mock algorithm
     // won't propagate any existing assignment from the cluster model.
 
-    // Init mock metadatastore for the unit test
-    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+    _metadataStore.clearMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
 
     // 1. rebalance with baseline calculation done
     // Generate the input for the rebalancer.
@@ -317,10 +339,10 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     // Since there is no special condition, the calculated IdealStates should be exactly the same
     // as the mock algorithm result.
     validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
-    Map<String, ResourceAssignment> baseline = metadataStore.getBaseline();
+    Map<String, ResourceAssignment> baseline = _metadataStore.getBaseline();
     Assert.assertEquals(baseline, algorithmResult);
     Map<String, ResourceAssignment> bestPossibleAssignment =
-        metadataStore.getBestPossibleAssignment();
+        _metadataStore.getBestPossibleAssignment();
     Assert.assertEquals(bestPossibleAssignment, algorithmResult);
 
     // 2. rebalance with one ideal state changed only
@@ -344,14 +366,14 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         Collections.singletonMap(changedResourceName, new Resource(changedResourceName)),
         newIdealStates, partialAlgorithmResult);
     // Baseline should be empty, because there is no cluster topology change.
-    baseline = metadataStore.getBaseline();
+    baseline = _metadataStore.getBaseline();
     Assert.assertEquals(baseline, Collections.emptyMap());
     // Best possible assignment contains the new assignment of only one resource.
-    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    bestPossibleAssignment = _metadataStore.getBestPossibleAssignment();
     Assert.assertEquals(bestPossibleAssignment, partialAlgorithmResult);
 
     // * Before the next test, recover the best possible assignment record.
-    metadataStore.persistBestPossibleAssignment(algorithmResult);
+    _metadataStore.persistBestPossibleAssignment(algorithmResult);
 
     // 3. rebalance with current state change only
     // Create a new cluster data cache to simulate cluster change
@@ -373,9 +395,9 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     // Verify that only the changed resource has been included in the calculation.
     validateRebalanceResult(Collections.emptyMap(), newIdealStates, algorithmResult);
     // Both assignment state should be empty.
-    baseline = metadataStore.getBaseline();
+    baseline = _metadataStore.getBaseline();
     Assert.assertEquals(baseline, Collections.emptyMap());
-    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    bestPossibleAssignment = _metadataStore.getBestPossibleAssignment();
     Assert.assertEquals(bestPossibleAssignment, Collections.emptyMap());
 
     // 4. rebalance with no change but best possible state record missing.
@@ -389,10 +411,10 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     // Verify that both resource has been included in the calculation.
     validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
     // Both assignment state should be empty since no cluster topology change.
-    baseline = metadataStore.getBaseline();
+    baseline = _metadataStore.getBaseline();
     Assert.assertEquals(baseline, Collections.emptyMap());
     // The best possible assignment should be present.
-    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    bestPossibleAssignment = _metadataStore.getBestPossibleAssignment();
     Assert.assertEquals(bestPossibleAssignment, algorithmResult);
   }
 
@@ -403,9 +425,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
       Assert.assertTrue(newIdealStates.containsKey(resourceName));
       IdealState is = newIdealStates.get(resourceName);
       ResourceAssignment assignment = expectedResult.get(resourceName);
-      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(
-          assignment.getMappedPartitions().stream().map(partition -> partition.getPartitionName())
-              .collect(Collectors.toSet())));
+      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(assignment.getMappedPartitions()
+          .stream().map(partition -> partition.getPartitionName()).collect(Collectors.toSet())));
       for (String partitionName : is.getPartitionSet()) {
         Assert.assertEquals(is.getInstanceStateMap(partitionName),
             assignment.getReplicaMap(new Partition(partitionName)));


[helix] 09/37: Redefine the hard/soft constraints (#422)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ae282a88b90274c414bff95c55eeb1232c7b484e
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Mon Aug 26 12:27:42 2019 -0700

    Redefine the hard/soft constraints (#422)
    
    * Refactor the interfaces of hard/soft constraints and a central place to keep the softConstraint weights
---
 .../rebalancer/waged/RebalanceAlgorithm.java       | 45 ------------------
 .../constraints/ConstraintsRebalanceAlgorithm.java | 49 --------------------
 .../waged/constraints/HardConstraint.java          | 23 ++++-----
 .../constraints/LeastPartitionCountConstraint.java | 53 +++++++++++++++++++++
 .../waged/constraints/SoftConstraint.java          | 54 +++++++++++++++++-----
 .../constraints/SoftConstraintWeightModel.java     | 54 ++++++++++++++++++++++
 6 files changed, 159 insertions(+), 119 deletions(-)

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
deleted file mode 100644
index d4221a1..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
+++ /dev/null
@@ -1,45 +0,0 @@
-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.ResourceAssignment;
-
-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 of <ResourceName, ResourceAssignment>.
-   */
-  Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
-      Map<String, Map<HardConstraint.FailureReason, Integer>> failureReasons);
-}
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
deleted file mode 100644
index 286fd07..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java
+++ /dev/null
@@ -1,49 +0,0 @@
-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.ResourceAssignment;
-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, ResourceAssignment> 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
index 3ee57ea..f544d4b 100644
--- 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
@@ -27,24 +27,21 @@ 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,
-  }
+abstract class HardConstraint {
 
   /**
-   * @return True if the proposed assignment is valid.
+   * Check if the replica could be assigned to the node
+   * @return True if the proposed assignment is valid; False otherwise
    */
-  boolean isAssignmentValid(AssignableNode node, AssignableReplica rep,
+  abstract boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext);
 
   /**
-   * @return Detail of the reason that the proposed assignment was rejected.
+   * Return class name by default as description if it's explanatory enough, child class could override
+   * the method and add more detailed descriptions
+   * @return The detailed description of hard constraint
    */
-  FailureReason getFailureReason();
+  String getDescription() {
+    return getClass().getName();
+  }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java
new file mode 100644
index 0000000..a8d36db
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java
@@ -0,0 +1,53 @@
+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 the proposed assignment according to the instance's partition count.
+ */
+class LeastPartitionCountConstraint extends SoftConstraint {
+  static LeastPartitionCountConstraint INSTANCE = new LeastPartitionCountConstraint();
+
+  private LeastPartitionCountConstraint() {
+  }
+
+  /**
+   * Returns a score depending on the number of assignments on this node. The score is scaled evenly
+   * between the minScore and maxScore.
+   * When the node is idle, return with the maxScore.
+   * When the node usage reaches the estimated max partition, return with (minScore + maxScore ) /
+   * 2.
+   * When the node usage reaches 2 * estimated_max or more, return with the minScore.
+   * If the estimated max partition count is not set, it defaults to Integer.MAX_VALUE in
+   * clusterContext.
+   */
+  @Override
+  float getAssignmentOriginScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+      throw new UnsupportedOperationException("The POC implementation has a bug, will fix it as TODO");
+//    float doubleMaxPartitionCount = 2.0f * clusterContext.getEstimatedMaxPartitionCount();
+//    int curPartitionCount = node.getCurrentAssignmentCount();
+//    return Math.max((doubleMaxPartitionCount - curPartitionCount) / doubleMaxPartitionCount, 0);
+  }
+}
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
index bce4a5a..db145fe 100644
--- 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
@@ -27,23 +27,53 @@ 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;
+abstract class SoftConstraint {
+  private float _maxScore = 1000f;
+  private float _minScore = -1000f;
+
+  interface ScalerFunction {
+    /**
+     * Scale the origin score to a normalized range (0, 1).
+     * The purpose is to compare scores between different soft constraints.
+     * @param originScore The origin score
+     * @return The normalized value between (0, 1)
+     */
+    float scale(float originScore);
+  }
 
   /**
-   * 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.
+   * Default constructor, uses default min/max scores
+   */
+  SoftConstraint() {
+  }
+
+  /**
+   * Child class customize the min/max score on its own
+   * @param maxScore The max score
+   * @param minScore The min score
    */
-  float assignmentScore(AssignableNode node, AssignableReplica rep, ClusterContext clusterContext);
+  SoftConstraint(float maxScore, float minScore) {
+    _maxScore = maxScore;
+    _minScore = minScore;
+  }
 
   /**
-   * 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
+   * 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.
+   * @return float value representing the score
    */
-  void setConstraintImportance(float importance);
+  abstract float getAssignmentOriginScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext);
 
-  float getConstraintImportance();
+  /**
+   * The default scaler function that squashes any score within (min_score, max_score) to (0, 1);
+   * Child class could override the method and customize the method on its own
+   * @return The MinMaxScaler instance by default
+   */
+  ScalerFunction getScalerFunction() {
+    return (score) -> (score - _minScore) / (_maxScore - _minScore);
+  }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
new file mode 100644
index 0000000..10201ce
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
@@ -0,0 +1,54 @@
+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 java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * The class retrieves the offline model that defines the relative importance of soft constraints.
+ */
+class SoftConstraintWeightModel {
+  private static Map<? extends SoftConstraint, Float> MODEL;
+
+  static {
+    // TODO either define the weights in property files or zookeeper node or static human input
+    MODEL = ImmutableMap.<SoftConstraint, Float> builder()
+        .put(LeastPartitionCountConstraint.INSTANCE, 1.0f).build();
+  }
+
+  /**
+   * Get the sum of normalized scores, given calculated scores map of soft constraints
+   * @param originScoresMap The origin scores by soft constraints
+   * @return The sum of soft constraints scores
+   */
+  float getSumOfScores(Map<SoftConstraint, Float> originScoresMap) {
+    float sum = 0;
+    for (Map.Entry<SoftConstraint, Float> softConstraintScoreEntry : originScoresMap.entrySet()) {
+      SoftConstraint softConstraint = softConstraintScoreEntry.getKey();
+      float score = softConstraint.getScalerFunction().scale(softConstraintScoreEntry.getValue());
+      float weight = MODEL.get(softConstraint);
+      sum += score * weight;
+    }
+
+    return sum;
+  }
+}


[helix] 11/37: Revert "Refine the WAGED rebalancer related interfaces for integration (#431)" (#437)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c65476d7808cd5be1e7b27ac5254b390c21de5f8
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Wed Aug 28 22:47:58 2019 -0700

    Revert "Refine the WAGED rebalancer related interfaces for integration (#431)" (#437)
    
    This reverts commit 08a2015c617ddd3c93525afc572081a7836f9476.
---
 .../org/apache/helix/HelixRebalanceException.java  | 43 ----------
 .../changedetector/ResourceChangeDetector.java     | 20 ++---
 .../controller/rebalancer/GlobalRebalancer.java    | 67 +++++++++++++++
 .../rebalancer/waged/ClusterDataDetector.java      | 73 ++++++++++++++++
 .../rebalancer/waged/ClusterDataProvider.java      | 54 ++++++++++++
 .../rebalancer/waged/WagedRebalancer.java          | 65 ++++----------
 .../waged/model/ClusterModelProvider.java          | 25 +++---
 .../stages/BestPossibleStateCalcStage.java         | 98 ++++++----------------
 .../waged/model/TestClusterModelProvider.java      |  6 +-
 9 files changed, 255 insertions(+), 196 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
deleted file mode 100644
index c01b173..0000000
--- a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
+++ /dev/null
@@ -1,43 +0,0 @@
-package org.apache.helix;
-
-/*
- * 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.
- */
-
-/**
- * Exception thrown by Helix due to rebalance failures.
- */
-public class HelixRebalanceException extends Exception {
-  enum RebalanceFailureType {
-    INVALID_CLUSTER_STATUS,
-    INVALID_REBALANCER_STATUS,
-    FAILED_TO_CALCULATE,
-    UNKNOWN_FAILURE
-  }
-
-  private final RebalanceFailureType _type;
-
-  public HelixRebalanceException(String message, RebalanceFailureType type, Throwable cause) {
-    super(String.format("%s. Failure Type: %s", message, type.name()), cause);
-    _type = type;
-  }
-
-  public RebalanceFailureType getFailureType() {
-    return _type;
-  }
-}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
index 611f4b2..d65e609 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
@@ -20,17 +20,15 @@ package org.apache.helix.controller.changedetector;
  */
 
 import com.google.common.collect.Sets;
-import org.apache.helix.HelixConstants;
-import org.apache.helix.HelixProperty;
-import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 
 /**
  * ResourceChangeDetector implements ChangeDetector. It caches resource-related metadata from
@@ -39,7 +37,6 @@ import java.util.Map;
  * WARNING: the methods of this class are not thread-safe.
  */
 public class ResourceChangeDetector implements ChangeDetector {
-  private static final Logger LOG = LoggerFactory.getLogger(ResourceChangeDetector.class.getName());
 
   private ResourceChangeSnapshot _oldSnapshot; // snapshot for previous pipeline run
   private ResourceChangeSnapshot _newSnapshot; // snapshot for this pipeline run
@@ -111,13 +108,10 @@ public class ResourceChangeDetector implements ChangeDetector {
       return snapshot.getResourceConfigMap();
     case LIVE_INSTANCE:
       return snapshot.getLiveInstances();
-    case CONFIG:
-      return Collections.emptyMap();
     default:
-      LOG.warn(
-          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: {}",
-          changeType);
-      return Collections.emptyMap();
+      throw new HelixException(String.format(
+          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: %s",
+          changeType));
     }
   }
 
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/ClusterDataDetector.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
new file mode 100644
index 0000000..0423edf
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
@@ -0,0 +1,73 @@
+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 {
+    BaselineAssignmentChange,
+    InstanceConfigChange,
+    ClusterConfigChange,
+    ResourceConfigChange,
+    ResourceIdealStatesChange,
+    InstanceStateChange,
+    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..387666c
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java
@@ -0,0 +1,54 @@
+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.ResourceAssignment;
+
+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, ResourceAssignment> baselineAssignment,
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
+    // TODO finish the implementation.
+    return null;
+  }
+}
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
index fd740e6..aa3cfee 100644
--- 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
@@ -19,13 +19,10 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
-import org.apache.helix.HelixRebalanceException;
-import org.apache.helix.controller.changedetector.ResourceChangeDetector;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
-import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
-import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintsRebalanceAlgorithm;
+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;
@@ -39,57 +36,23 @@ import java.util.Map;
  * A placeholder before we have the implementation.
  * Weight-Aware Globally-Even Distribute Rebalancer.
  *
- * @see <a href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
- * Design Document
- * </a>
+ * @see <a href="Design Document">https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer</a>
  */
-public class WagedRebalancer {
+public class WagedRebalancer implements GlobalRebalancer<ResourceControllerDataProvider> {
   private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
 
-  // --------- The following fields are placeholders and need replacement. -----------//
-  // TODO Shall we make the metadata store a static threadlocal object as well to avoid reinitialization?
-  private final AssignmentMetadataStore _assignmentMetadataStore;
-  private final RebalanceAlgorithm _rebalanceAlgorithm;
-  // ------------------------------------------------------------------------------------//
+  @Override
+  public void init(HelixManager manager) { }
 
-  // The cluster change detector is a stateful object. Make it static to avoid unnecessary
-  // reinitialization.
-  private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
-      new ThreadLocal<>();
-  private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
-
-  private ResourceChangeDetector getChangeDetector() {
-    if (CHANGE_DETECTOR_THREAD_LOCAL.get() == null) {
-      CHANGE_DETECTOR_THREAD_LOCAL.set(new ResourceChangeDetector());
-    }
-    return CHANGE_DETECTOR_THREAD_LOCAL.get();
-  }
-
-  public WagedRebalancer(HelixManager helixManager) {
-    // TODO init the metadata store according to their requirement when integrate, or change to final static method if possible.
-    _assignmentMetadataStore = new AssignmentMetadataStore();
-    // TODO init the algorithm according to the requirement when integrate.
-    _rebalanceAlgorithm = new ConstraintsRebalanceAlgorithm();
-
-    // Use the mapping calculator in DelayedAutoRebalancer for calculating the final assignment
-    // output.
-    // This calculator will translate the best possible assignment into an applicable state mapping
-    // based on the current states.
-    // TODO abstract and separate the mapping calculator logic from the DelayedAutoRebalancer
-    _mappingCalculator = new DelayedAutoRebalancer();
+  @Override
+  public Map<String, IdealState> computeNewIdealState(CurrentStateOutput currentStateOutput,
+      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
+      throws HelixException {
+    return new HashMap<>();
   }
 
-  /**
-   * Compute the new IdealStates for all the resources input. The IdealStates include both the new
-   * partition assignment (in the listFiles) and the new replica state mapping (in the mapFields).
-   * @param clusterData        The Cluster status data provider.
-   * @param resourceMap        A map containing all the rebalancing resources.
-   * @param currentStateOutput The present Current State of the cluster.
-   * @return A map containing the computed new IdealStates.
-   */
-  public Map<String, IdealState> computeNewIdealStates(ResourceControllerDataProvider clusterData,
-      Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
-      throws HelixRebalanceException {
-    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/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index c4f7d02..9de023b 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -19,9 +19,9 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -59,7 +59,7 @@ public class ClusterModelProvider {
    */
   public static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
       Map<String, Resource> resourceMap, Set<String> activeInstances,
-      Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
+      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
       Map<String, ResourceAssignment> baselineAssignment,
       Map<String, ResourceAssignment> bestPossibleAssignment) {
     // Generate replica objects for all the resource partitions.
@@ -108,13 +108,14 @@ public class ClusterModelProvider {
    */
   private static Set<AssignableReplica> findToBeAssignedReplicas(
       Map<String, Set<AssignableReplica>> replicaMap,
-      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
+      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
       Map<String, ResourceAssignment> bestPossibleAssignment,
       Map<String, Set<AssignableReplica>> allocatedReplicas) {
     Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
-    if (clusterChanges.containsKey(HelixConstants.ChangeType.CONFIG)
-        || clusterChanges.containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
-      // If the cluster topology has been modified, need to reassign all replicas
+    if (clusterChanges.containsKey(ClusterDataDetector.ChangeType.ClusterConfigChange)
+        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.InstanceConfigChange)
+        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.BaselineAssignmentChange)) {
+      // If the cluster topology or baseline assignment has been modified, need to reassign all replicas
       toBeAssignedReplicas
           .addAll(replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()));
     } else {
@@ -123,13 +124,11 @@ public class ClusterModelProvider {
         Set<AssignableReplica> replicas = replicaMap.get(resourceName);
         // 1. if the resource config/idealstate is changed, need to reassign.
         // 2. if the resource does appear in the best possible assignment, need to reassign.
-        if (clusterChanges
-            .getOrDefault(HelixConstants.ChangeType.RESOURCE_CONFIG, Collections.emptySet())
-            .contains(resourceName)
-            || clusterChanges
-            .getOrDefault(HelixConstants.ChangeType.IDEAL_STATE, Collections.emptySet())
-            .contains(resourceName)
-            || !bestPossibleAssignment.containsKey(resourceName)) {
+        if (clusterChanges.getOrDefault(ClusterDataDetector.ChangeType.ResourceConfigChange,
+            Collections.emptySet()).contains(resourceName) || clusterChanges
+            .getOrDefault(ClusterDataDetector.ChangeType.ResourceIdealStatesChange,
+                Collections.emptySet()).contains(resourceName) || !bestPossibleAssignment
+            .containsKey(resourceName)) {
           toBeAssignedReplicas.addAll(replicas);
           continue; // go to check next resource
         } else {
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index ba4da88..49a72e0 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -20,7 +20,6 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -28,7 +27,6 @@ import java.util.concurrent.Callable;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
-import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
@@ -39,7 +37,6 @@ import org.apache.helix.controller.rebalancer.MaintenanceRebalancer;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
-import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.MaintenanceSignal;
@@ -59,19 +56,18 @@ import org.slf4j.LoggerFactory;
  * IdealState,StateModel,LiveInstance
  */
 public class BestPossibleStateCalcStage extends AbstractBaseStage {
-  private static final Logger logger =
-      LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
+  private static final Logger logger = LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
-    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+    CurrentStateOutput currentStateOutput =
+        event.getAttribute(AttributeName.CURRENT_STATE.name());
     final Map<String, Resource> resourceMap =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     final ClusterStatusMonitor clusterStatusMonitor =
         event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    ResourceControllerDataProvider cache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
 
     if (currentStateOutput == null || resourceMap == null || cache == null) {
       throw new StageException(
@@ -94,7 +90,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
                     resourceMap, stateModelDefMap);
           }
         } catch (Exception e) {
-          LogUtil.logError(logger, _eventId, "Could not update cluster status metrics!", e);
+          LogUtil
+              .logError(logger, _eventId, "Could not update cluster status metrics!", e);
         }
         return null;
       }
@@ -103,8 +100,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
   private BestPossibleStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
       CurrentStateOutput currentStateOutput) {
-    ResourceControllerDataProvider cache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
     BestPossibleStateOutput output = new BestPossibleStateOutput();
 
     HelixManager helixManager = event.getAttribute(AttributeName.helixmanager.name());
@@ -116,50 +112,19 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     boolean isValid = validateOfflineInstancesLimit(cache,
         (HelixManager) event.getAttribute(AttributeName.helixmanager.name()));
 
-    // 1. Rebalance with the WAGED rebalancer
-    // The rebalancer only calculates the new ideal assignment for all the resources that are
-    // configured to use the WAGED rebalancer.
-    // For the other resources, the legacy rebalancers will be triggered in the next step.
-    Map<String, IdealState> newIdealStates = new HashMap<>();
-    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
-    try {
-      newIdealStates
-          .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, currentStateOutput));
-    } catch (HelixRebalanceException ex) {
-      // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
-      // Since it calculates for all the eligible resources globally, a partial result is invalid.
-      // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
-      LogUtil.logError(logger, _eventId, String
-          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
-              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
-    }
-
     final List<String> failureResources = new ArrayList<>();
     Iterator<Resource> itr = resourceMap.values().iterator();
     while (itr.hasNext()) {
       Resource resource = itr.next();
       boolean result = false;
-      IdealState is = newIdealStates.get(resource.getResourceName());
-      if (is != null) {
-        // 2. Check if the WAGED rebalancer has calculated for this resource or not.
-        result = checkBestPossibleStateCalculation(is);
-        if (result) {
-          // The WAGED rebalancer calculates a valid result, record in the output
-          updateBestPossibleStateOutput(output, resource, is);
-        }
-      } else {
-        // 3. The WAGED rebalancer skips calculating the resource assignment, fallback to use a
-        // legacy resource rebalancer if applicable.
-        // If this calculation fails, the resource will be reported in the failureResources list.
-        try {
-          result =
-              computeSingleResourceBestPossibleState(event, cache, currentStateOutput, resource,
-                  output);
-        } catch (HelixException ex) {
-          LogUtil.logError(logger, _eventId,
-              "Exception when calculating best possible states for " + resource.getResourceName(),
-              ex);
-        }
+      try {
+        result =
+            computeResourceBestPossibleState(event, cache, currentStateOutput, resource, output);
+      } catch (HelixException ex) {
+        LogUtil.logError(logger, _eventId,
+            "Exception when calculating best possible states for " + resource.getResourceName(),
+            ex);
+
       }
       if (!result) {
         failureResources.add(resource.getResourceName());
@@ -220,9 +185,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
         if (manager != null) {
           if (manager.getHelixDataAccessor()
               .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
-            manager.getClusterManagmentTool()
-                .autoEnableMaintenanceMode(manager.getClusterName(), true, errMsg,
-                    MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
+            manager.getClusterManagmentTool().autoEnableMaintenanceMode(manager.getClusterName(),
+                true, errMsg, MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
             LogUtil.logWarn(logger, _eventId, errMsg);
           }
         } else {
@@ -235,19 +199,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     return true;
   }
 
-  private void updateBestPossibleStateOutput(BestPossibleStateOutput output, Resource resource,
-      IdealState computedIdealState) {
-    output.setPreferenceLists(resource.getResourceName(), computedIdealState.getPreferenceLists());
-    for (Partition partition : resource.getPartitions()) {
-      Map<String, String> newStateMap =
-          computedIdealState.getInstanceStateMap(partition.getPartitionName());
-      output.setState(resource.getResourceName(), partition, newStateMap);
-    }
-  }
-
-  private boolean computeSingleResourceBestPossibleState(ClusterEvent event,
-      ResourceControllerDataProvider cache, CurrentStateOutput currentStateOutput,
-      Resource resource, BestPossibleStateOutput output) {
+  private boolean computeResourceBestPossibleState(ClusterEvent event, ResourceControllerDataProvider cache,
+      CurrentStateOutput currentStateOutput, Resource resource, BestPossibleStateOutput output) {
     // for each ideal state
     // read the state model def
     // for each resource
@@ -276,13 +229,12 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
     Rebalancer<ResourceControllerDataProvider> rebalancer =
         getRebalancer(idealState, resourceName, cache.isMaintenanceModeEnabled());
-    MappingCalculator<ResourceControllerDataProvider> mappingCalculator =
-        getMappingCalculator(rebalancer, resourceName);
+    MappingCalculator<ResourceControllerDataProvider> mappingCalculator = getMappingCalculator(rebalancer, resourceName);
 
     if (rebalancer == null || mappingCalculator == null) {
-      LogUtil.logError(logger, _eventId, "Error computing assignment for resource " + resourceName
-          + ". no rebalancer found. rebalancer: " + rebalancer + " mappingCalculator: "
-          + mappingCalculator);
+      LogUtil.logError(logger, _eventId,
+          "Error computing assignment for resource " + resourceName + ". no rebalancer found. rebalancer: " + rebalancer
+              + " mappingCalculator: " + mappingCalculator);
     }
 
     if (rebalancer != null && mappingCalculator != null) {
@@ -347,8 +299,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     }
   }
 
-  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState,
-      String resourceName, boolean isMaintenanceModeEnabled) {
+  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState, String resourceName,
+      boolean isMaintenanceModeEnabled) {
     Rebalancer<ResourceControllerDataProvider> customizedRebalancer = null;
     String rebalancerClassName = idealState.getRebalancerClassName();
     if (rebalancerClassName != null) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index 1221b6f..f92a66c 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -19,8 +19,8 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixConstants;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
@@ -177,7 +177,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     // 5. test with best possible assignment but cluster topology is changed
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(HelixConstants.ChangeType.CONFIG,
+        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ClusterConfigChange,
             Collections.emptySet()), Collections.emptyMap(), bestPossibleAssignment);
     // There should be no existing assignment since the topology change invalidates all existing assignment
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
@@ -194,7 +194,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     String changedResourceName = _resourceNames.get(0);
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(HelixConstants.ChangeType.RESOURCE_CONFIG,
+        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ResourceConfigChange,
             Collections.singleton(changedResourceName)), Collections.emptyMap(),
         bestPossibleAssignment);
     // There should be no existing assignment for all the resource except for resource2.


[helix] 07/37: Add cluster level default instance config. (#413)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0d677647b75f2617bf61d7dc7627b855e1c11171
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Mon Aug 19 10:49:12 2019 -0700

    Add cluster level default instance config. (#413)
    
    This config will be applied to the instance when there is no (or empty) capacity configuration in the Instance Config.
    Also add unit tests.
---
 .../rebalancer/waged/model/AssignableNode.java     |  8 +++-
 .../java/org/apache/helix/model/ClusterConfig.java | 51 ++++++++++++++++++++
 .../org/apache/helix/model/InstanceConfig.java     | 13 ++++--
 .../rebalancer/waged/model/TestAssignableNode.java | 13 ++++++
 .../org/apache/helix/model/TestClusterConfig.java  | 54 ++++++++++++++++++++++
 5 files changed, 134 insertions(+), 5 deletions(-)

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
index 5fc04d7..e2fd676 100644
--- 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
@@ -88,11 +88,15 @@ public class AssignableNode {
       Collection<AssignableReplica> existingAssignment) {
     reset();
 
-    _currentCapacity.putAll(instanceConfig.getInstanceCapacityMap());
+    Map<String, Integer> instanceCapacity = instanceConfig.getInstanceCapacityMap();
+    if (instanceCapacity.isEmpty()) {
+      instanceCapacity = clusterConfig.getDefaultInstanceCapacityMap();
+    }
+    _currentCapacity.putAll(instanceCapacity);
     _faultZone = computeFaultZone(clusterConfig, instanceConfig);
     _instanceTags = new HashSet<>(instanceConfig.getTags());
     _disabledPartitionsMap = instanceConfig.getDisabledPartitionsMap();
-    _maxCapacity = instanceConfig.getInstanceCapacityMap();
+    _maxCapacity = instanceCapacity;
     _maxPartition = clusterConfig.getMaxPartitionsPerInstance();
 
     assignNewBatch(existingAssignment);
diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
index ee942c7..a8c1da9 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
@@ -29,10 +29,12 @@ import org.apache.helix.api.config.StateTransitionThrottleConfig;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 /**
  * Cluster configurations
@@ -86,6 +88,8 @@ public class ClusterConfig extends HelixProperty {
 
     // The required instance capacity keys for resource partition assignment calculation.
     INSTANCE_CAPACITY_KEYS,
+    // The default instance capacity if no capacity is configured in the Instance Config node.
+    DEFAULT_INSTANCE_CAPACITY_MAP,
     // The preference of the rebalance result.
     // EVENNESS - Evenness of the resource utilization, partition, and top state distribution.
     // LESS_MOVEMENT - the tendency of keeping the current assignment instead of moving the partition for optimal assignment.
@@ -700,6 +704,53 @@ public class ClusterConfig extends HelixProperty {
   }
 
   /**
+   * Get the default instance capacity information from the map fields.
+   * @return data map if it exists, or empty map
+   */
+  public Map<String, Integer> getDefaultInstanceCapacityMap() {
+    Map<String, String> capacityData =
+        _record.getMapField(ClusterConfigProperty.DEFAULT_INSTANCE_CAPACITY_MAP.name());
+
+    if (capacityData != null) {
+      return capacityData.entrySet().stream().collect(
+          Collectors.toMap(entry -> entry.getKey(), entry -> Integer.parseInt(entry.getValue())));
+    }
+    return Collections.emptyMap();
+  }
+
+  /**
+   * Set the default instance capacity information with an Integer mapping.
+   * @param capacityDataMap - map of instance capacity data
+   * @throws IllegalArgumentException - when any of the data value is a negative number or when the map is empty
+   *
+   * This information is required by the global rebalancer.
+   * @see <a href="Rebalance Algorithm">
+   *   https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer#rebalance-algorithm-adapter
+   *   </a>
+   * If the instance capacity is not configured in neither Instance Config nor Cluster Config, the
+   * cluster topology is considered invalid. So the rebalancer may stop working.
+   */
+  public void setDefaultInstanceCapacityMap(Map<String, Integer> capacityDataMap)
+      throws IllegalArgumentException {
+    if (capacityDataMap == null || capacityDataMap.size() == 0) {
+      throw new IllegalArgumentException("Default Instance Capacity Data is empty");
+    }
+
+    Map<String, String> capacityData = new HashMap<>();
+
+    capacityDataMap.entrySet().stream().forEach(entry -> {
+      if (entry.getValue() < 0) {
+        throw new IllegalArgumentException(String
+            .format("Default Instance Capacity Data contains a negative value: %s = %d",
+                entry.getKey(), entry.getValue()));
+      }
+      capacityData.put(entry.getKey(), Integer.toString(entry.getValue()));
+    });
+
+    _record.setMapField(ClusterConfigProperty.DEFAULT_INSTANCE_CAPACITY_MAP.name(), capacityData);
+  }
+
+  /**
    * Set the global rebalancer's assignment preference.
    * @param preference A map of the GlobalRebalancePreferenceKey and the corresponding weight.
    *                   The ratio of the configured weights will determine the rebalancer's behavior.
diff --git a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
index 88fd1dd..ac1814d 100644
--- a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
@@ -507,8 +507,7 @@ public class InstanceConfig extends HelixProperty {
   }
 
   /**
-   * Get the instance capacity information from the map fields
-   *
+   * Get the instance capacity information from the map fields.
    * @return data map if it exists, or empty map
    */
   public Map<String, Integer> getInstanceCapacityMap() {
@@ -523,9 +522,17 @@ public class InstanceConfig extends HelixProperty {
   }
 
   /**
-   * Set the instance capacity information with an Integer mapping
+   * Set the instance capacity information with an Integer mapping.
    * @param capacityDataMap - map of instance capacity data
    * @throws IllegalArgumentException - when any of the data value is a negative number or when the map is empty
+   *
+   * This information is required by the global rebalancer.
+   * @see <a href="Rebalance Algorithm">
+   *   https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer#rebalance-algorithm-adapter
+   *   </a>
+   * If the instance capacity is not configured in neither Instance Config nor Cluster Config, the
+   * cluster topology is considered invalid. So the rebalancer may stop working.
+   * Note that when a rebalancer requires this capacity information, it will ignore INSTANCE_WEIGHT.
    */
   public void setInstanceCapacityMap(Map<String, Integer> capacityDataMap)
       throws IllegalArgumentException {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index d7fcce9..f55d0fc 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -200,4 +200,17 @@ public class TestAssignableNode extends AbstractTestClusterModel {
 
     Assert.assertEquals(assignableNode.getFaultZone(), "2/testInstance/");
   }
+
+  @Test
+  public void testDefaultInstanceCapacity() {
+    ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
+    testClusterConfig.setDefaultInstanceCapacityMap(_capacityDataMap);
+
+    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
+
+    AssignableNode assignableNode =
+        new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId,
+            Collections.emptyList());
+    Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
+  }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java b/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java
index 209b196..5cf9bff 100644
--- a/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java
+++ b/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java
@@ -20,6 +20,8 @@ package org.apache.helix.model;
  */
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.ZNRecord;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -127,4 +129,56 @@ public class TestClusterConfig {
     ClusterConfig testConfig = new ClusterConfig("testId");
     testConfig.setGlobalRebalancePreference(preference);
   }
+
+  @Test
+  public void testGetInstanceCapacityMap() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1, "item2", 2, "item3", 3);
+
+    Map<String, String> capacityDataMapString =
+        ImmutableMap.of("item1", "1", "item2", "2", "item3", "3");
+
+    ZNRecord rec = new ZNRecord("testId");
+    rec.setMapField(ClusterConfig.ClusterConfigProperty.DEFAULT_INSTANCE_CAPACITY_MAP.name(),
+        capacityDataMapString);
+    ClusterConfig testConfig = new ClusterConfig(rec);
+
+    Assert.assertTrue(testConfig.getDefaultInstanceCapacityMap().equals(capacityDataMap));
+  }
+
+  @Test
+  public void testGetInstanceCapacityMapEmpty() {
+    ClusterConfig testConfig = new ClusterConfig("testId");
+
+    Assert.assertTrue(testConfig.getDefaultInstanceCapacityMap().equals(Collections.emptyMap()));
+  }
+
+  @Test
+  public void testSetInstanceCapacityMap() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1, "item2", 2, "item3", 3);
+
+    Map<String, String> capacityDataMapString =
+        ImmutableMap.of("item1", "1", "item2", "2", "item3", "3");
+
+    ClusterConfig testConfig = new ClusterConfig("testConfig");
+    testConfig.setDefaultInstanceCapacityMap(capacityDataMap);
+
+    Assert.assertEquals(testConfig.getRecord().getMapField(ClusterConfig.ClusterConfigProperty.
+        DEFAULT_INSTANCE_CAPACITY_MAP.name()), capacityDataMapString);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Default Instance Capacity Data is empty")
+  public void testSetInstanceCapacityMapEmpty() {
+    Map<String, Integer> capacityDataMap = new HashMap<>();
+
+    ClusterConfig testConfig = new ClusterConfig("testConfig");
+    testConfig.setDefaultInstanceCapacityMap(capacityDataMap);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Default Instance Capacity Data contains a negative value: item3 = -3")
+  public void testSetInstanceCapacityMapInvalid() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1, "item2", 2, "item3", -3);
+
+    ClusterConfig testConfig = new ClusterConfig("testConfig");
+    testConfig.setDefaultInstanceCapacityMap(capacityDataMap);
+  }
 }


[helix] 08/37: Add ChangeDetector interface and ResourceChangeDetector implementation (#388)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 84ac35ba3d281bdcdc8ba35dfec3156c73e0624b
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Thu Aug 15 14:33:02 2019 -0700

    Add ChangeDetector interface and ResourceChangeDetector implementation (#388)
    
    Add ChangeDetector interface and ResourceChangeDetector implementation
    
    In order to efficiently react to changes happening to the cluster in the new WAGED rebalancer, a new component called ChangeDetector was added.
    
    Changelist:
    1. Add ChangeDetector interface
    2. Implement ResourceChangeDetector
    3. Add ResourceChangeCache, a wrapper for critical cluster metadata
    4. Add an integration test, TestResourceChangeDetector
---
 .../controller/changedetector/ChangeDetector.java  |  57 ++++
 .../changedetector/ResourceChangeDetector.java     | 158 +++++++++
 .../changedetector/ResourceChangeSnapshot.java     | 105 ++++++
 .../ResourceControllerDataProvider.java            |  33 +-
 .../changedetector/TestResourceChangeDetector.java | 357 +++++++++++++++++++++
 5 files changed, 705 insertions(+), 5 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ChangeDetector.java
new file mode 100644
index 0000000..fbe4afc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ChangeDetector.java
@@ -0,0 +1,57 @@
+package org.apache.helix.controller.changedetector;
+
+/*
+ * 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.Collection;
+import org.apache.helix.HelixConstants;
+
+/**
+ * ChangeDetector interface that will be used to track deltas in the cluster from one pipeline run
+ * to another. The interface methods are designed to be flexible for both the resource pipeline and
+ * the task pipeline.
+ * TODO: Consider splitting this up into two different ChangeDetector interfaces:
+ * TODO: PropertyBasedChangeDetector and PathBasedChangeDetector.
+ */
+public interface ChangeDetector {
+
+  /**
+   * Returns all types of changes detected.
+   * @return a collection of ChangeTypes
+   */
+  Collection<HelixConstants.ChangeType> getChangeTypes();
+
+  /**
+   * Returns the names of items that changed based on the change type given.
+   * @return a collection of names of items that changed
+   */
+  Collection<String> getChangesByType(HelixConstants.ChangeType changeType);
+
+  /**
+   * Returns the names of items that were added based on the change type given.
+   * @return a collection of names of items that were added
+   */
+  Collection<String> getAdditionsByType(HelixConstants.ChangeType changeType);
+
+  /**
+   * Returns the names of items that were removed based on the change type given.
+   * @return a collection of names of items that were removed
+   */
+  Collection<String> getRemovalsByType(HelixConstants.ChangeType changeType);
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
new file mode 100644
index 0000000..d65e609
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
@@ -0,0 +1,158 @@
+package org.apache.helix.controller.changedetector;
+
+/*
+ * 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 com.google.common.collect.Sets;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+
+/**
+ * ResourceChangeDetector implements ChangeDetector. It caches resource-related metadata from
+ * Helix's main resource pipeline cache (DataProvider) and the computation results of change
+ * detection.
+ * WARNING: the methods of this class are not thread-safe.
+ */
+public class ResourceChangeDetector implements ChangeDetector {
+
+  private ResourceChangeSnapshot _oldSnapshot; // snapshot for previous pipeline run
+  private ResourceChangeSnapshot _newSnapshot; // snapshot for this pipeline run
+
+  // The following caches the computation results
+  private Map<HelixConstants.ChangeType, Collection<String>> _changedItems = new HashMap<>();
+  private Map<HelixConstants.ChangeType, Collection<String>> _addedItems = new HashMap<>();
+  private Map<HelixConstants.ChangeType, Collection<String>> _removedItems = new HashMap<>();
+
+  public ResourceChangeDetector() {
+    _newSnapshot = new ResourceChangeSnapshot();
+  }
+
+  /**
+   * Compare the underlying HelixProperty objects and produce a collection of names of changed
+   * properties.
+   * @return
+   */
+  private Collection<String> getChangedItems(Map<String, ? extends HelixProperty> oldPropertyMap,
+      Map<String, ? extends HelixProperty> newPropertyMap) {
+    Collection<String> changedItems = new HashSet<>();
+    oldPropertyMap.forEach((name, property) -> {
+      if (newPropertyMap.containsKey(name)
+          && !property.getRecord().equals(newPropertyMap.get(name).getRecord())) {
+        changedItems.add(name);
+      }
+    });
+    return changedItems;
+  }
+
+  /**
+   * Return a collection of names that are newly added.
+   * @return
+   */
+  private Collection<String> getAddedItems(Map<String, ? extends HelixProperty> oldPropertyMap,
+      Map<String, ? extends HelixProperty> newPropertyMap) {
+    return Sets.difference(newPropertyMap.keySet(), oldPropertyMap.keySet());
+  }
+
+  /**
+   * Return a collection of names that were removed.
+   * @return
+   */
+  private Collection<String> getRemovedItems(Map<String, ? extends HelixProperty> oldPropertyMap,
+      Map<String, ? extends HelixProperty> newPropertyMap) {
+    return Sets.difference(oldPropertyMap.keySet(), newPropertyMap.keySet());
+  }
+
+  private void clearCachedComputation() {
+    _changedItems.clear();
+    _addedItems.clear();
+    _removedItems.clear();
+  }
+
+  /**
+   * Based on the change type given and propertyMap type, call the right getters for propertyMap.
+   * @param changeType
+   * @param snapshot
+   * @return
+   */
+  private Map<String, ? extends HelixProperty> determinePropertyMapByType(
+      HelixConstants.ChangeType changeType, ResourceChangeSnapshot snapshot) {
+    switch (changeType) {
+    case INSTANCE_CONFIG:
+      return snapshot.getInstanceConfigMap();
+    case IDEAL_STATE:
+      return snapshot.getIdealStateMap();
+    case RESOURCE_CONFIG:
+      return snapshot.getResourceConfigMap();
+    case LIVE_INSTANCE:
+      return snapshot.getLiveInstances();
+    default:
+      throw new HelixException(String.format(
+          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: %s",
+          changeType));
+    }
+  }
+
+  /**
+   * Makes the current newSnapshot the oldSnapshot and reads in the up-to-date snapshot for change
+   * computation. To be called in the controller pipeline.
+   * @param dataProvider newly refreshed DataProvider (cache)
+   */
+  public synchronized void updateSnapshots(ResourceControllerDataProvider dataProvider) {
+    // If there are changes, update internal states
+    _oldSnapshot = new ResourceChangeSnapshot(_newSnapshot);
+    _newSnapshot = new ResourceChangeSnapshot(dataProvider);
+    dataProvider.clearRefreshedChangeTypes();
+
+    // Invalidate cached computation
+    clearCachedComputation();
+  }
+
+  @Override
+  public Collection<HelixConstants.ChangeType> getChangeTypes() {
+    return Collections.unmodifiableSet(_newSnapshot.getChangedTypes());
+  }
+
+  @Override
+  public Collection<String> getChangesByType(HelixConstants.ChangeType changeType) {
+    return _changedItems.computeIfAbsent(changeType,
+        changedItems -> getChangedItems(determinePropertyMapByType(changeType, _oldSnapshot),
+            determinePropertyMapByType(changeType, _newSnapshot)));
+  }
+
+  @Override
+  public Collection<String> getAdditionsByType(HelixConstants.ChangeType changeType) {
+    return _addedItems.computeIfAbsent(changeType,
+        changedItems -> getAddedItems(determinePropertyMapByType(changeType, _oldSnapshot),
+            determinePropertyMapByType(changeType, _newSnapshot)));
+  }
+
+  @Override
+  public Collection<String> getRemovalsByType(HelixConstants.ChangeType changeType) {
+    return _removedItems.computeIfAbsent(changeType,
+        changedItems -> getRemovedItems(determinePropertyMapByType(changeType, _oldSnapshot),
+            determinePropertyMapByType(changeType, _newSnapshot)));
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeSnapshot.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeSnapshot.java
new file mode 100644
index 0000000..cbc3539
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeSnapshot.java
@@ -0,0 +1,105 @@
+package org.apache.helix.controller.changedetector;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.ResourceConfig;
+
+/**
+ * ResourceChangeSnapshot is a POJO that contains the following Helix metadata:
+ * 1. InstanceConfig
+ * 2. IdealState
+ * 3. ResourceConfig
+ * 4. LiveInstance
+ * 5. Changed property types
+ * It serves as a snapshot of the main controller cache to enable the difference (change)
+ * calculation between two rounds of the pipeline run.
+ */
+class ResourceChangeSnapshot {
+
+  private Set<HelixConstants.ChangeType> _changedTypes;
+  private Map<String, InstanceConfig> _instanceConfigMap;
+  private Map<String, IdealState> _idealStateMap;
+  private Map<String, ResourceConfig> _resourceConfigMap;
+  private Map<String, LiveInstance> _liveInstances;
+
+  /**
+   * Default constructor that constructs an empty snapshot.
+   */
+  ResourceChangeSnapshot() {
+    _changedTypes = new HashSet<>();
+    _instanceConfigMap = new HashMap<>();
+    _idealStateMap = new HashMap<>();
+    _resourceConfigMap = new HashMap<>();
+    _liveInstances = new HashMap<>();
+  }
+
+  /**
+   * Constructor using controller cache (ResourceControllerDataProvider).
+   * @param dataProvider
+   */
+  ResourceChangeSnapshot(ResourceControllerDataProvider dataProvider) {
+    _changedTypes = new HashSet<>(dataProvider.getRefreshedChangeTypes());
+    _instanceConfigMap = new HashMap<>(dataProvider.getInstanceConfigMap());
+    _idealStateMap = new HashMap<>(dataProvider.getIdealStates());
+    _resourceConfigMap = new HashMap<>(dataProvider.getResourceConfigMap());
+    _liveInstances = new HashMap<>(dataProvider.getLiveInstances());
+  }
+
+  /**
+   * Copy constructor for ResourceChangeCache.
+   * @param cache
+   */
+  ResourceChangeSnapshot(ResourceChangeSnapshot cache) {
+    _changedTypes = new HashSet<>(cache._changedTypes);
+    _instanceConfigMap = new HashMap<>(cache._instanceConfigMap);
+    _idealStateMap = new HashMap<>(cache._idealStateMap);
+    _resourceConfigMap = new HashMap<>(cache._resourceConfigMap);
+    _liveInstances = new HashMap<>(cache._liveInstances);
+  }
+
+  Set<HelixConstants.ChangeType> getChangedTypes() {
+    return _changedTypes;
+  }
+
+  Map<String, InstanceConfig> getInstanceConfigMap() {
+    return _instanceConfigMap;
+  }
+
+  Map<String, IdealState> getIdealStateMap() {
+    return _idealStateMap;
+  }
+
+  Map<String, ResourceConfig> getResourceConfigMap() {
+    return _resourceConfigMap;
+  }
+
+  Map<String, LiveInstance> getLiveInstances() {
+    return _liveInstances;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java b/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java
index b1dc215..9e1550a 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/dataproviders/ResourceControllerDataProvider.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import java.util.concurrent.ConcurrentHashMap;
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.PropertyKey;
@@ -64,6 +65,9 @@ public class ResourceControllerDataProvider extends BaseControllerDataProvider {
   private Map<String, Map<String, MissingTopStateRecord>> _missingTopStateMap;
   private Map<String, Map<String, String>> _lastTopStateLocationMap;
 
+  // Maintain a set of all ChangeTypes for change detection
+  private Set<HelixConstants.ChangeType> _refreshedChangeTypes;
+
   public ResourceControllerDataProvider() {
     this(AbstractDataCache.UNKNOWN_CLUSTER);
   }
@@ -106,19 +110,21 @@ public class ResourceControllerDataProvider extends BaseControllerDataProvider {
     _idealMappingCache = new HashMap<>();
     _missingTopStateMap = new HashMap<>();
     _lastTopStateLocationMap = new HashMap<>();
+    _refreshedChangeTypes = ConcurrentHashMap.newKeySet();
   }
 
   public synchronized void refresh(HelixDataAccessor accessor) {
     long startTime = System.currentTimeMillis();
 
     // Refresh base
-    Set<HelixConstants.ChangeType> propertyRefreshed = super.doRefresh(accessor);
+    Set<HelixConstants.ChangeType> changedTypes = super.doRefresh(accessor);
+    _refreshedChangeTypes.addAll(changedTypes);
 
     // Invalidate cached information if any of the important data has been refreshed
-    if (propertyRefreshed.contains(HelixConstants.ChangeType.IDEAL_STATE)
-        || propertyRefreshed.contains(HelixConstants.ChangeType.LIVE_INSTANCE)
-        || propertyRefreshed.contains(HelixConstants.ChangeType.INSTANCE_CONFIG)
-        || propertyRefreshed.contains(HelixConstants.ChangeType.RESOURCE_CONFIG)) {
+    if (changedTypes.contains(HelixConstants.ChangeType.IDEAL_STATE)
+        || changedTypes.contains(HelixConstants.ChangeType.LIVE_INSTANCE)
+        || changedTypes.contains(HelixConstants.ChangeType.INSTANCE_CONFIG)
+        || changedTypes.contains(HelixConstants.ChangeType.RESOURCE_CONFIG)) {
       clearCachedResourceAssignments();
     }
 
@@ -261,6 +267,23 @@ public class ResourceControllerDataProvider extends BaseControllerDataProvider {
     _idealMappingCache.put(resource, mapping);
   }
 
+  /**
+   * Return the set of all PropertyTypes that changed prior to this round of rebalance. The caller
+   * should clear this set by calling {@link #clearRefreshedChangeTypes()}.
+   * @return
+   */
+  public Set<HelixConstants.ChangeType> getRefreshedChangeTypes() {
+    return _refreshedChangeTypes;
+  }
+
+  /**
+   * Clears the set of all PropertyTypes that changed. The caller will have consumed all change
+   * types by calling {@link #getRefreshedChangeTypes()}.
+   */
+  public void clearRefreshedChangeTypes() {
+    _refreshedChangeTypes.clear();
+  }
+
   public void clearCachedResourceAssignments() {
     _resourceAssignmentCache.clear();
     _idealMappingCache.clear();
diff --git a/helix-core/src/test/java/org/apache/helix/controller/changedetector/TestResourceChangeDetector.java b/helix-core/src/test/java/org/apache/helix/controller/changedetector/TestResourceChangeDetector.java
new file mode 100644
index 0000000..3ef41e4
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/changedetector/TestResourceChangeDetector.java
@@ -0,0 +1,357 @@
+package org.apache.helix.controller.changedetector;
+
+/*
+ * 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.Collection;
+import org.apache.helix.AccessOption;
+import org.apache.helix.HelixConstants.ChangeType;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.manager.zk.ZKHelixDataAccessor;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.ResourceConfig;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+/**
+ * This test contains a series of unit tests for ResourceChangeDetector.
+ */
+public class TestResourceChangeDetector extends ZkTestBase {
+
+  // All possible change types for ResourceChangeDetector except for ClusterConfig
+  // since we don't provide the names of changed fields for ClusterConfig
+  private static final ChangeType[] RESOURCE_CHANGE_TYPES = {
+      ChangeType.IDEAL_STATE, ChangeType.INSTANCE_CONFIG, ChangeType.LIVE_INSTANCE,
+      ChangeType.RESOURCE_CONFIG
+  };
+
+  private static final String CLUSTER_NAME = TestHelper.getTestClassName();
+  private static final String RESOURCE_NAME = "TestDB";
+  private static final String NEW_RESOURCE_NAME = "TestDB2";
+  private static final String STATE_MODEL = "MasterSlave";
+  // There are 5 possible change types for ResourceChangeDetector
+  private static final int NUM_CHANGE_TYPES = 5;
+  private static final int NUM_RESOURCES = 1;
+  private static final int NUM_PARTITIONS = 10;
+  private static final int NUM_REPLICAS = 3;
+  private static final int NUM_NODES = 5;
+
+  // Create a mock of ResourceControllerDataProvider so that we could manipulate it
+  private ResourceControllerDataProvider _dataProvider;
+  private ResourceChangeDetector _resourceChangeDetector;
+  private ClusterControllerManager _controller;
+  private MockParticipantManager[] _participants = new MockParticipantManager[NUM_NODES];
+  private HelixDataAccessor _dataAccessor;
+  private PropertyKey.Builder _keyBuilder;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    super.beforeClass();
+
+    // Set up a mock cluster
+    TestHelper.setupCluster(CLUSTER_NAME, ZK_ADDR, 12918, // participant port
+        "localhost", // participant name prefix
+        RESOURCE_NAME, // resource name prefix
+        NUM_RESOURCES, // resources
+        NUM_PARTITIONS, // partitions per resource
+        NUM_NODES, // nodes
+        NUM_REPLICAS, // replicas
+        STATE_MODEL, true); // do rebalance
+
+    // Start a controller
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, "controller_0");
+    _controller.syncStart();
+
+    // Start Participants
+    for (int i = 0; i < NUM_NODES; i++) {
+      String instanceName = "localhost_" + (12918 + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].syncStart();
+    }
+
+    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
+    _keyBuilder = _dataAccessor.keyBuilder();
+    _resourceChangeDetector = new ResourceChangeDetector();
+
+    // Create a custom data provider
+    _dataProvider = new ResourceControllerDataProvider(CLUSTER_NAME);
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    for (MockParticipantManager participant : _participants) {
+      if (participant != null && participant.isConnected()) {
+        participant.syncStop();
+      }
+    }
+    _controller.syncStop();
+    deleteCluster(CLUSTER_NAME);
+    Assert.assertFalse(TestHelper.verify(() -> _dataAccessor.getBaseDataAccessor()
+        .exists("/" + CLUSTER_NAME, AccessOption.PERSISTENT), 20000L));
+  }
+
+  /**
+   * Tests the initialization of the change detector. It should tell us that there's been changes
+   * for every change type and for all items per type.
+   * @throws Exception
+   */
+  @Test
+  public void testResourceChangeDetectorInit() throws Exception {
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    Collection<ChangeType> changeTypes = _resourceChangeDetector.getChangeTypes();
+    Assert.assertEquals(changeTypes.size(), NUM_CHANGE_TYPES,
+        "Not all change types have been detected for ResourceChangeDetector!");
+
+    // Check that the right amount of resources show up as added
+    checkDetectionCounts(ChangeType.IDEAL_STATE, NUM_RESOURCES, 0, 0);
+
+    // Check that the right amount of instances show up as added
+    checkDetectionCounts(ChangeType.LIVE_INSTANCE, NUM_NODES, 0, 0);
+    checkDetectionCounts(ChangeType.INSTANCE_CONFIG, NUM_NODES, 0, 0);
+  }
+
+  /**
+   * Add a resource (IS and ResourceConfig) and see if the detector detects it.
+   */
+  @Test(dependsOnMethods = "testResourceChangeDetectorInit")
+  public void testAddResource() {
+    // Create an IS and ResourceConfig
+    _gSetupTool.getClusterManagementTool().addResource(CLUSTER_NAME, NEW_RESOURCE_NAME,
+        NUM_PARTITIONS, STATE_MODEL);
+    ResourceConfig resourceConfig = new ResourceConfig(NEW_RESOURCE_NAME);
+    _dataAccessor.setProperty(_keyBuilder.resourceConfig(NEW_RESOURCE_NAME), resourceConfig);
+    // Manually notify dataProvider
+    _dataProvider.notifyDataChange(ChangeType.IDEAL_STATE);
+    _dataProvider.notifyDataChange(ChangeType.RESOURCE_CONFIG);
+
+    // Refresh the data provider
+    _dataProvider.refresh(_dataAccessor);
+
+    // Update the detector
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.IDEAL_STATE, ChangeType.RESOURCE_CONFIG);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.IDEAL_STATE || type == ChangeType.RESOURCE_CONFIG) {
+        checkDetectionCounts(type, 1, 0, 0);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+    // Check that detector gives the right item
+    Assert.assertTrue(_resourceChangeDetector.getAdditionsByType(ChangeType.RESOURCE_CONFIG)
+        .contains(NEW_RESOURCE_NAME));
+  }
+
+  /**
+   * Modify a resource config for the new resource and test that detector detects it.
+   */
+  @Test(dependsOnMethods = "testAddResource")
+  public void testModifyResource() {
+    // Modify resource config
+    ResourceConfig resourceConfig =
+        _dataAccessor.getProperty(_keyBuilder.resourceConfig(NEW_RESOURCE_NAME));
+    resourceConfig.getRecord().setSimpleField("Did I change?", "Yes!");
+    _dataAccessor.updateProperty(_keyBuilder.resourceConfig(NEW_RESOURCE_NAME), resourceConfig);
+
+    // Notify data provider and check
+    _dataProvider.notifyDataChange(ChangeType.RESOURCE_CONFIG);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.RESOURCE_CONFIG);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.RESOURCE_CONFIG) {
+        checkDetectionCounts(type, 0, 1, 0);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+    Assert.assertTrue(_resourceChangeDetector.getChangesByType(ChangeType.RESOURCE_CONFIG)
+        .contains(NEW_RESOURCE_NAME));
+  }
+
+  /**
+   * Delete the new resource and test that detector detects it.
+   */
+  @Test(dependsOnMethods = "testModifyResource")
+  public void testDeleteResource() {
+    // Delete the newly added resource
+    _dataAccessor.removeProperty(_keyBuilder.idealStates(NEW_RESOURCE_NAME));
+    _dataAccessor.removeProperty(_keyBuilder.resourceConfig(NEW_RESOURCE_NAME));
+
+    // Notify data provider and check
+    _dataProvider.notifyDataChange(ChangeType.IDEAL_STATE);
+    _dataProvider.notifyDataChange(ChangeType.RESOURCE_CONFIG);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.RESOURCE_CONFIG, ChangeType.IDEAL_STATE);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.IDEAL_STATE || type == ChangeType.RESOURCE_CONFIG) {
+        checkDetectionCounts(type, 0, 0, 1);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+  }
+
+  /**
+   * Disconnect and reconnect a Participant and see if detector detects.
+   */
+  @Test(dependsOnMethods = "testDeleteResource")
+  public void testDisconnectReconnectInstance() {
+    // Disconnect a Participant
+    _participants[0].syncStop();
+    _dataProvider.notifyDataChange(ChangeType.LIVE_INSTANCE);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.LIVE_INSTANCE);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.LIVE_INSTANCE) {
+        checkDetectionCounts(type, 0, 0, 1);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+
+    // Reconnect the Participant
+    _participants[0] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, "localhost_12918");
+    _participants[0].syncStart();
+    _dataProvider.notifyDataChange(ChangeType.LIVE_INSTANCE);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.LIVE_INSTANCE);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.LIVE_INSTANCE) {
+        checkDetectionCounts(type, 1, 0, 0);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+  }
+
+  /**
+   * Remove an instance completely and see if detector detects.
+   */
+  @Test(dependsOnMethods = "testDisconnectReconnectInstance")
+  public void testRemoveInstance() {
+    _participants[0].syncStop();
+    InstanceConfig instanceConfig =
+        _dataAccessor.getProperty(_keyBuilder.instanceConfig(_participants[0].getInstanceName()));
+    _gSetupTool.getClusterManagementTool().dropInstance(CLUSTER_NAME, instanceConfig);
+
+    _dataProvider.notifyDataChange(ChangeType.LIVE_INSTANCE);
+    _dataProvider.notifyDataChange(ChangeType.INSTANCE_CONFIG);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.LIVE_INSTANCE, ChangeType.INSTANCE_CONFIG);
+    // Check the counts
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      if (type == ChangeType.LIVE_INSTANCE || type == ChangeType.INSTANCE_CONFIG) {
+        checkDetectionCounts(type, 0, 0, 1);
+      } else {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+  }
+
+  /**
+   * Modify cluster config and see if detector detects.
+   */
+  @Test(dependsOnMethods = "testRemoveInstance")
+  public void testModifyClusterConfig() {
+    // Modify cluster config
+    ClusterConfig clusterConfig = _dataAccessor.getProperty(_keyBuilder.clusterConfig());
+    clusterConfig.setTopology("Change");
+    _dataAccessor.updateProperty(_keyBuilder.clusterConfig(), clusterConfig);
+
+    _dataProvider.notifyDataChange(ChangeType.CLUSTER_CONFIG);
+    _dataProvider.refresh(_dataAccessor);
+    _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+    checkChangeTypes(ChangeType.CLUSTER_CONFIG);
+    // Check the counts for other types
+    for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+      checkDetectionCounts(type, 0, 0, 0);
+    }
+  }
+
+  /**
+   * Test that change detector gives correct results when there are no changes after updating
+   * snapshots.
+   */
+  @Test(dependsOnMethods = "testModifyClusterConfig")
+  public void testNoChange() {
+    // Test twice to make sure that no change is stable across different runs
+    for (int i = 0; i < 2; i++) {
+      _dataProvider.refresh(_dataAccessor);
+      _resourceChangeDetector.updateSnapshots(_dataProvider);
+
+      Assert.assertEquals(_resourceChangeDetector.getChangeTypes().size(), 0);
+      // Check the counts for all the other types
+      for (ChangeType type : RESOURCE_CHANGE_TYPES) {
+        checkDetectionCounts(type, 0, 0, 0);
+      }
+    }
+  }
+
+  /**
+   * Check that the given change types appear in detector's change types.
+   * @param types
+   */
+  private void checkChangeTypes(ChangeType... types) {
+    for (ChangeType type : types) {
+      Assert.assertTrue(_resourceChangeDetector.getChangeTypes().contains(type));
+    }
+  }
+
+  /**
+   * Convenience method for checking three types of detections.
+   * @param changeType
+   * @param additions
+   * @param changes
+   * @param deletions
+   */
+  private void checkDetectionCounts(ChangeType changeType, int additions, int changes,
+      int deletions) {
+    Assert.assertEquals(_resourceChangeDetector.getAdditionsByType(changeType).size(), additions);
+    Assert.assertEquals(_resourceChangeDetector.getChangesByType(changeType).size(), changes);
+    Assert.assertEquals(_resourceChangeDetector.getRemovalsByType(changeType).size(), deletions);
+  }
+}


[helix] 20/37: HardConstraints Implementation and unit tests (#433)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 43d3476918b476985542cd75f4c76ea2fc60ca74
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Mon Sep 9 13:57:28 2019 -0700

    HardConstraints Implementation and unit tests (#433)
    
    * Implement all of basic Hard Constraints
    1. Partitions count cannot exceed instance's upper limit
    2. Fault zone aware (no same partitions on the same zone)
    3. Partitions weight cannot exceed instance's capacity
    4. Cannot assign inactived partitions
    5. Same partition of different states cannot co-exist in one instance
    6. Instance doesn't have the tag of the replica
---
 .../constraints/FaultZoneAwareConstraint.java      | 43 ++++++++++++
 .../waged/constraints/NodeCapacityConstraint.java  | 50 ++++++++++++++
 .../NodeMaxPartitionLimitConstraint.java           | 40 +++++++++++
 .../constraints/ReplicaActivateConstraint.java     | 41 +++++++++++
 .../SamePartitionOnInstanceConstraint.java         | 39 +++++++++++
 .../waged/constraints/ValidGroupTagConstraint.java | 41 +++++++++++
 .../rebalancer/waged/model/AssignableNode.java     | 75 +++++++++++---------
 .../rebalancer/waged/model/AssignableReplica.java  |  4 ++
 .../constraints/TestFaultZoneAwareConstraint.java  | 79 ++++++++++++++++++++++
 .../constraints/TestNodeCapacityConstraint.java    | 54 +++++++++++++++
 .../TestNodeMaxPartitionLimitConstraint.java       | 56 +++++++++++++++
 .../TestPartitionActivateConstraint.java           | 64 ++++++++++++++++++
 .../TestSamePartitionOnInstanceConstraint.java     | 59 ++++++++++++++++
 .../constraints/TestValidGroupTagConstraint.java   | 66 ++++++++++++++++++
 14 files changed, 679 insertions(+), 32 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/FaultZoneAwareConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/FaultZoneAwareConstraint.java
new file mode 100644
index 0000000..c33419e
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/FaultZoneAwareConstraint.java
@@ -0,0 +1,43 @@
+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;
+
+class FaultZoneAwareConstraint extends HardConstraint {
+
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    if (!node.hasFaultZone()) {
+      return true;
+    }
+    return !clusterContext
+        .getPartitionsForResourceAndFaultZone(replica.getResourceName(), node.getFaultZone())
+        .contains(replica.getPartitionName());
+  }
+
+  @Override
+  String getDescription() {
+    return "A fault zone cannot contain more than 1 replica of same partition";
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java
new file mode 100644
index 0000000..5fc2faf
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.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 java.util.Map;
+
+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;
+
+class NodeCapacityConstraint extends HardConstraint {
+
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    Map<String, Integer> nodeCapacity = node.getCurrentCapacity();
+    Map<String, Integer> replicaCapacity = replica.getCapacity();
+
+    for (String key : replicaCapacity.keySet()) {
+      if (nodeCapacity.containsKey(key)) {
+        if (nodeCapacity.get(key) < replicaCapacity.get(key)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  String getDescription() {
+    return "Node has insufficient capacity";
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
new file mode 100644
index 0000000..9d0752b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
@@ -0,0 +1,40 @@
+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;
+
+class NodeMaxPartitionLimitConstraint extends HardConstraint {
+
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    return node.getAssignedReplicaCount() < node.getMaxPartition()
+        && node.getAssignedPartitionsByResource(replica.getResourceName()).size() < replica
+            .getResourceMaxPartitionsPerInstance();
+  }
+
+  @Override
+  String getDescription() {
+    return "Cannot exceed the maximum number of partitions limitation on node";
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ReplicaActivateConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ReplicaActivateConstraint.java
new file mode 100644
index 0000000..9152efe
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ReplicaActivateConstraint.java
@@ -0,0 +1,41 @@
+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 java.util.List;
+
+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;
+
+class ReplicaActivateConstraint extends HardConstraint {
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    List<String> disabledPartitions =
+        node.getDisabledPartitionsMap().get(replica.getResourceName());
+    return disabledPartitions == null || !disabledPartitions.contains(replica.getPartitionName());
+  }
+
+  @Override
+  String getDescription() {
+    return "Cannot assign the inactive replica";
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SamePartitionOnInstanceConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SamePartitionOnInstanceConstraint.java
new file mode 100644
index 0000000..202e49a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SamePartitionOnInstanceConstraint.java
@@ -0,0 +1,39 @@
+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;
+
+class SamePartitionOnInstanceConstraint extends HardConstraint {
+
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    return !node.getAssignedPartitionsByResource(replica.getResourceName())
+        .contains(replica.getPartitionName());
+  }
+
+  @Override
+  String getDescription() {
+    return "Same partition of different states cannot co-exist in one instance";
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ValidGroupTagConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ValidGroupTagConstraint.java
new file mode 100644
index 0000000..e31864f
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ValidGroupTagConstraint.java
@@ -0,0 +1,41 @@
+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;
+
+class ValidGroupTagConstraint extends HardConstraint {
+  @Override
+  boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    if (!replica.hasResourceInstanceGroupTag()) {
+      return true;
+    }
+
+    return node.getInstanceTags().contains(replica.getResourceInstanceGroupTag());
+  }
+
+  @Override
+  String getDescription() {
+    return "Instance doesn't have the tag of the replica";
+  }
+}
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
index 4141d20..f25c289 100644
--- 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
@@ -19,6 +19,8 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import static java.lang.Math.max;
+
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -35,8 +37,6 @@ import org.apache.helix.model.InstanceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.lang.Math.max;
-
 /**
  * This class represents a possible allocation of the replication.
  * Note that any usage updates to the AssignableNode are not thread safe.
@@ -52,7 +52,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
   private Map<String, Integer> _maxCapacity;
   private int _maxPartition; // maximum number of the partitions that can be assigned to the node.
 
-  // A map of <resource name, <partition name, replica>> that tracks the replicas assigned to the node.
+  // A map of <resource name, <partition name, replica>> that tracks the replicas assigned to the
+  // node.
   private Map<String, Map<String, AssignableReplica>> _currentAssignedReplicaMap;
   // A map of <capacity key, capacity value> that tracks the current available node capacity
   private Map<String, Integer> _currentCapacityMap;
@@ -78,13 +79,15 @@ public class AssignableNode implements Comparable<AssignableNode> {
   }
 
   /**
-   * Update the node with a ClusterDataCache. This resets the current assignment and recalculates currentCapacity.
-   * NOTE: While this is required to be used in the constructor, this can also be used when the clusterCache needs to be
-   * refreshed. This is under the assumption that the capacity mappings of InstanceConfig and ResourceConfig could
+   * Update the node with a ClusterDataCache. This resets the current assignment and recalculates
+   * currentCapacity.
+   * NOTE: While this is required to be used in the constructor, this can also be used when the
+   * clusterCache needs to be
+   * refreshed. This is under the assumption that the capacity mappings of InstanceConfig and
+   * ResourceConfig could
    * subject to change. If the assumption is no longer true, this function should become private.
-   *
-   * @param clusterConfig      - the Cluster Config of the cluster where the node is located
-   * @param instanceConfig     - the Instance Config of the node
+   * @param clusterConfig - the Cluster Config of the cluster where the node is located
+   * @param instanceConfig - the Instance Config of the node
    * @param existingAssignment - all the existing replicas that are current assigned to the node
    */
   private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig,
@@ -104,7 +107,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
 
   /**
    * Assign a replica to the node.
-   *
    * @param assignableReplica - the replica to be assigned
    */
   void assign(AssignableReplica assignableReplica) {
@@ -116,7 +118,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
   /**
    * Release a replica from the node.
    * If the replication is not on this node, the assignable node is not updated.
-   *
    * @param replica - the replica to be released
    */
   void release(AssignableReplica replica) throws IllegalArgumentException {
@@ -131,8 +132,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
     }
 
     Map<String, AssignableReplica> partitionMap = _currentAssignedReplicaMap.get(resourceName);
-    if (!partitionMap.containsKey(partitionName) || !partitionMap.get(partitionName)
-        .equals(replica)) {
+    if (!partitionMap.containsKey(partitionName)
+        || !partitionMap.get(partitionName).equals(replica)) {
       LOG.warn("Replica {} is not assigned to node {}. Ignore the release call.",
           replica.toString(), getInstanceName());
       return;
@@ -174,7 +175,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
 
   /**
    * @param resource Resource name
-   * @return A set of the current assigned replicas' partition names with the top state in the specified resource.
+   * @return A set of the current assigned replicas' partition names with the top state in the
+   *         specified resource.
    */
   public Set<String> getAssignedTopStatePartitionsByResource(String resource) {
     return _currentAssignedReplicaMap.getOrDefault(resource, Collections.emptyMap()).entrySet()
@@ -194,7 +196,7 @@ public class AssignableNode implements Comparable<AssignableNode> {
   /**
    * @return The total count of assigned replicas.
    */
-  public long getAssignedReplicaCount() {
+  public int getAssignedReplicaCount() {
     return _currentAssignedReplicaMap.values().stream().mapToInt(Map::size).sum();
   }
 
@@ -207,7 +209,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
 
   /**
    * Return the most concerning capacity utilization number for evenly partition assignment.
-   * The method dynamically returns the highest utilization number among all the capacity categories.
+   * The method dynamically returns the highest utilization number among all the capacity
+   * categories.
    * For example, if the current node usage is {CPU: 0.9, MEM: 0.4, DISK: 0.6}. Then this call shall
    * return 0.9.
    *
@@ -229,15 +232,21 @@ public class AssignableNode implements Comparable<AssignableNode> {
     return _faultZone;
   }
 
+  public boolean hasFaultZone() {
+    return _faultZone != null;
+  }
+
   /**
-   * @return A map of <resource name, set of partition names> contains all the partitions that are disabled on the node.
+   * @return A map of <resource name, set of partition names> contains all the partitions that are
+   *         disabled on the node.
    */
   public Map<String, List<String>> getDisabledPartitionsMap() {
     return _disabledPartitionsMap;
   }
 
   /**
-   * @return A map of <capacity category, capacity number> that describes the max capacity of the node.
+   * @return A map of <capacity category, capacity number> that describes the max capacity of the
+   *         node.
    */
   public Map<String, Integer> getMaxCapacity() {
     return _maxCapacity;
@@ -251,8 +260,10 @@ public class AssignableNode implements Comparable<AssignableNode> {
   }
 
   /**
-   * Computes the fault zone id based on the domain and fault zone type when topology is enabled. For example, when
-   * the domain is "zone=2, instance=testInstance" and the fault zone type is "zone", this function returns "2".
+   * Computes the fault zone id based on the domain and fault zone type when topology is enabled.
+   * For example, when
+   * the domain is "zone=2, instance=testInstance" and the fault zone type is "zone", this function
+   * returns "2".
    * If cannot find the fault zone id, this function leaves the fault zone id as the instance name.
    * TODO merge this logic with Topology.java tree building logic.
    * For now, the WAGED rebalancer has a more strict topology def requirement.
@@ -267,8 +278,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
       }
 
       String[] topologyDef = topologyStr.trim().split("/");
-      if (topologyDef.length == 0 || Arrays.stream(topologyDef)
-          .noneMatch(type -> type.equals(faultZoneType))) {
+      if (topologyDef.length == 0
+          || Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
         throw new HelixException(
             "The configured topology definition is empty or does not contain the fault zone type.");
       }
@@ -304,7 +315,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
   }
 
   /**
-   * This function should only be used to assign a set of new partitions that are not allocated on this node.
+   * This function should only be used to assign a set of new partitions that are not allocated on
+   * this node.
    * Using this function avoids the overhead of updating capacity repeatedly.
    */
   private void assignNewBatch(Collection<AssignableReplica> replicas) {
@@ -314,9 +326,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
       // increment the capacity requirement according to partition's capacity configuration.
       for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
         totalPartitionCapacity.compute(capacity.getKey(),
-            (key, totalValue) -> (totalValue == null) ?
-                capacity.getValue() :
-                totalValue + capacity.getValue());
+            (key, totalValue) -> (totalValue == null) ? capacity.getValue()
+                : totalValue + capacity.getValue());
       }
     }
 
@@ -332,12 +343,12 @@ public class AssignableNode implements Comparable<AssignableNode> {
   private void addToAssignmentRecord(AssignableReplica replica) {
     String resourceName = replica.getResourceName();
     String partitionName = replica.getPartitionName();
-    if (_currentAssignedReplicaMap.containsKey(resourceName) && _currentAssignedReplicaMap
-        .get(resourceName).containsKey(partitionName)) {
-      throw new HelixException(String
-          .format("Resource %s already has a replica with state %s from partition %s on node %s",
-              replica.getResourceName(), replica.getReplicaState(), replica.getPartitionName(),
-              getInstanceName()));
+    if (_currentAssignedReplicaMap.containsKey(resourceName)
+        && _currentAssignedReplicaMap.get(resourceName).containsKey(partitionName)) {
+      throw new HelixException(String.format(
+          "Resource %s already has a replica with state %s from partition %s on node %s",
+          replica.getResourceName(), replica.getReplicaState(), replica.getPartitionName(),
+          getInstanceName()));
     } else {
       _currentAssignedReplicaMap.computeIfAbsent(resourceName, key -> new HashMap<>())
           .put(partitionName, replica);
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
index 537bf70..66bd7b7 100644
--- 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
@@ -87,6 +87,10 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
     return _resourceInstanceGroupTag;
   }
 
+  public boolean hasResourceInstanceGroupTag() {
+    return _resourceInstanceGroupTag != null && !_resourceInstanceGroupTag.isEmpty();
+  }
+
   public int getResourceMaxPartitionsPerInstance() {
     return _resourceMaxPartitionsPerInstance;
   }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestFaultZoneAwareConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestFaultZoneAwareConstraint.java
new file mode 100644
index 0000000..9d2cb14
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestFaultZoneAwareConstraint.java
@@ -0,0 +1,79 @@
+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 static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class TestFaultZoneAwareConstraint {
+  private static final String TEST_PARTITION = "testPartition";
+  private static final String TEST_ZONE = "testZone";
+  private static final String TEST_RESOURCE = "testResource";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+
+  private final HardConstraint _faultZoneAwareConstraint = new FaultZoneAwareConstraint();
+
+  @BeforeMethod
+  public void init() {
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(TEST_PARTITION);
+    when(_testNode.getFaultZone()).thenReturn(TEST_ZONE);
+  }
+
+  @Test
+  public void inValidWhenFaultZoneAlreadyAssigned() {
+    when(_testNode.hasFaultZone()).thenReturn(true);
+    when(_clusterContext.getPartitionsForResourceAndFaultZone(TEST_RESOURCE, TEST_ZONE)).thenReturn(
+            ImmutableSet.of(TEST_PARTITION));
+
+    Assert.assertFalse(
+        _faultZoneAwareConstraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void validWhenEmptyAssignment() {
+    when(_testNode.hasFaultZone()).thenReturn(true);
+    when(_clusterContext.getPartitionsForResourceAndFaultZone(TEST_RESOURCE, TEST_ZONE)).thenReturn(Collections.emptySet());
+
+    Assert.assertTrue(
+        _faultZoneAwareConstraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void validWhenNoFaultZone() {
+    when(_testNode.hasFaultZone()).thenReturn(false);
+
+    Assert.assertTrue(
+        _faultZoneAwareConstraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java
new file mode 100644
index 0000000..511f881
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java
@@ -0,0 +1,54 @@
+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 static org.mockito.Mockito.when;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TestNodeCapacityConstraint {
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final HardConstraint _constraint = new NodeCapacityConstraint();
+
+  @Test
+  public void testConstraintValidWhenNodeHasEnoughSpace() {
+    String key = "testKey";
+    when(_testNode.getCurrentCapacity()).thenReturn(ImmutableMap.of(key,  10));
+    when(_testReplica.getCapacity()).thenReturn(ImmutableMap.of(key, 5));
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintInValidWhenNodeHasInsufficientSpace() {
+    String key = "testKey";
+    when(_testNode.getCurrentCapacity()).thenReturn(ImmutableMap.of(key,  1));
+    when(_testReplica.getCapacity()).thenReturn(ImmutableMap.of(key, 5));
+    Assert.assertFalse(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeMaxPartitionLimitConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeMaxPartitionLimitConstraint.java
new file mode 100644
index 0000000..4cb7466
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeMaxPartitionLimitConstraint.java
@@ -0,0 +1,56 @@
+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 static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestNodeMaxPartitionLimitConstraint {
+  private static final String TEST_RESOURCE = "TestResource";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final HardConstraint _constraint = new NodeMaxPartitionLimitConstraint();
+
+  @Test
+  public void testConstraintValid() {
+    when(_testNode.getAssignedReplicaCount()).thenReturn(0);
+    when(_testNode.getMaxPartition()).thenReturn(10);
+    when(_testNode.getAssignedPartitionsByResource(TEST_RESOURCE))
+        .thenReturn(Collections.emptySet());
+    when(_testReplica.getResourceMaxPartitionsPerInstance()).thenReturn(5);
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintInvalid() {
+    when(_testNode.getAssignedReplicaCount()).thenReturn(10);
+    when(_testNode.getMaxPartition()).thenReturn(5);
+    Assert.assertFalse(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionActivateConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionActivateConstraint.java
new file mode 100644
index 0000000..ecfdaa2
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionActivateConstraint.java
@@ -0,0 +1,64 @@
+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 static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+public class TestPartitionActivateConstraint {
+  private static final String TEST_PARTITION = "TestPartition";
+  private static final String TEST_RESOURCE = "TestResource";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final HardConstraint _constraint = new ReplicaActivateConstraint();
+
+  @Test
+  public void testConstraintValid() {
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(TEST_PARTITION);
+    when(_testNode.getDisabledPartitionsMap())
+        .thenReturn(ImmutableMap.of(TEST_PARTITION, Collections.emptyList()));
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+    when(_testNode.getDisabledPartitionsMap())
+        .thenReturn(ImmutableMap.of(TEST_PARTITION, ImmutableList.of("dummy")));
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintInvalidWhenReplicaIsDisabled() {
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(TEST_PARTITION);
+    when(_testNode.getDisabledPartitionsMap())
+        .thenReturn(ImmutableMap.of(TEST_PARTITION, ImmutableList.of(TEST_PARTITION)));
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSamePartitionOnInstanceConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSamePartitionOnInstanceConstraint.java
new file mode 100644
index 0000000..50b0c03
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSamePartitionOnInstanceConstraint.java
@@ -0,0 +1,59 @@
+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 static org.mockito.Mockito.when;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class TestSamePartitionOnInstanceConstraint {
+  private static final String TEST_RESOURCE = "TestResource";
+  private static final String TEST_PARTITIOIN = TEST_RESOURCE + "0";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final HardConstraint _constraint = new SamePartitionOnInstanceConstraint();
+
+  @Test
+  public void testConstraintValid() {
+    when(_testNode.getAssignedPartitionsByResource(TEST_RESOURCE))
+        .thenReturn(ImmutableSet.of("dummy"));
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(TEST_PARTITIOIN);
+
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintInValid() {
+    when(_testNode.getAssignedPartitionsByResource(TEST_RESOURCE))
+        .thenReturn(ImmutableSet.of(TEST_PARTITIOIN));
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(TEST_PARTITIOIN);
+    Assert.assertFalse(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestValidGroupTagConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestValidGroupTagConstraint.java
new file mode 100644
index 0000000..8d02b3d
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestValidGroupTagConstraint.java
@@ -0,0 +1,66 @@
+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 static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class TestValidGroupTagConstraint {
+  private static final String TEST_TAG = "testTag";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final HardConstraint _constraint = new ValidGroupTagConstraint();
+
+  @Test
+  public void testConstraintValid() {
+    when(_testReplica.hasResourceInstanceGroupTag()).thenReturn(true);
+    when(_testReplica.getResourceInstanceGroupTag()).thenReturn(TEST_TAG);
+    when(_testNode.getInstanceTags()).thenReturn(ImmutableSet.of(TEST_TAG));
+
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintInValid() {
+    when(_testReplica.hasResourceInstanceGroupTag()).thenReturn(true);
+    when(_testReplica.getResourceInstanceGroupTag()).thenReturn(TEST_TAG);
+    when(_testNode.getInstanceTags()).thenReturn(Collections.emptySet());
+
+    Assert.assertFalse(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+
+  @Test
+  public void testConstraintWhenReplicaHasNoTag() {
+    when(_testReplica.hasResourceInstanceGroupTag()).thenReturn(false);
+
+    Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
+  }
+}


[helix] 25/37: Implement MaxCapacityUsageInstanceConstraint soft constraint (#463)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 52e1ba5971afa8ecdf88ceca273bdef6d1098f77
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Sat Sep 14 22:09:10 2019 -0700

    Implement MaxCapacityUsageInstanceConstraint soft constraint (#463)
    
    The constraint evaluates the score by checking the max used capacity key out of all the capacity
    keys.
    The higher the maximum usage value for the capacity key, the lower the score will be, implying
    that it is that much less desirable to assign anything on the given node.
    It is a greedy approach since it evaluates only the most used capacity key.
---
 .../InstancePartitionsCountConstraint.java         |  1 +
 ...ava => MaxCapacityUsageInstanceConstraint.java} | 24 +++++-----
 .../waged/constraints/SoftConstraint.java          |  3 +-
 .../constraints/SoftConstraintWeightModel.java     |  2 +-
 .../TestInstancePartitionsCountConstraint.java     |  9 ++--
 .../TestMaxCapacityUsageInstanceConstraint.java    | 56 ++++++++++++++++++++++
 6 files changed, 78 insertions(+), 17 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
index ca05cf8..feee05a 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
@@ -27,6 +27,7 @@ import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
  * Evaluate by instance's current partition count versus estimated max partition count
  * Intuitively, Encourage the assignment if the instance's occupancy rate is below average;
  * Discourage the assignment if the instance's occupancy rate is above average
+ * The normalized score will be within [0, 1]
  */
 class InstancePartitionsCountConstraint extends SoftConstraint {
   private static final float MAX_SCORE = 1f;
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/MaxCapacityUsageInstanceConstraint.java
similarity index 61%
copy from helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
copy to helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/MaxCapacityUsageInstanceConstraint.java
index ca05cf8..2fe94c6 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/MaxCapacityUsageInstanceConstraint.java
@@ -9,7 +9,7 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
  * "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
+ *     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
@@ -24,24 +24,24 @@ import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
 
 /**
- * Evaluate by instance's current partition count versus estimated max partition count
- * Intuitively, Encourage the assignment if the instance's occupancy rate is below average;
- * Discourage the assignment if the instance's occupancy rate is above average
+ * The constraint evaluates the score by checking the max used capacity key out of all the capacity
+ * keys.
+ * The higher the maximum usage value for the capacity key, the lower the score will be, implying
+ * that it is that much less desirable to assign anything on the given node.
+ * It is a greedy approach since it evaluates only on the most used capacity key.
  */
-class InstancePartitionsCountConstraint extends SoftConstraint {
-  private static final float MAX_SCORE = 1f;
-  private static final float MIN_SCORE = 0f;
+class MaxCapacityUsageInstanceConstraint extends SoftConstraint {
+  private static final float MIN_SCORE = 0;
+  private static final float MAX_SCORE = 1;
 
-  InstancePartitionsCountConstraint() {
+  MaxCapacityUsageInstanceConstraint() {
     super(MAX_SCORE, MIN_SCORE);
   }
 
   @Override
   protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext) {
-    float doubleEstimatedMaxPartitionCount = 2 * clusterContext.getEstimatedMaxPartitionCount();
-    float currentPartitionCount = node.getAssignedReplicaCount();
-    return Math.max((doubleEstimatedMaxPartitionCount - currentPartitionCount)
-        / doubleEstimatedMaxPartitionCount, 0);
+    float maxCapacityUsage = node.getHighestCapacityUtilization();
+    return 1.0f - maxCapacityUsage / 2.0f;
   }
 }
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
index 0f2bdbc..f44d262 100644
--- 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
@@ -24,7 +24,8 @@ import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
 
 /**
- * The "soft" constraint evaluates the optimality of an assignment by giving it a score of a scale of [minScore, maxScore]
+ * The "soft" constraint evaluates the optimality of an assignment by giving it a score of a scale
+ * of [minScore, maxScore]
  * The higher the score, the better the assignment; Intuitively, the assignment is encouraged.
  * The lower score the score, the worse the assignment; Intuitively, the assignment is penalized.
  */
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
index a961936..953005c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
@@ -35,7 +35,7 @@ class SoftConstraintWeightModel {
   }
 
   static {
-    //TODO update the weight
+    // TODO update the weight
     MODEL = ImmutableMap.<Class, Float> builder().put(InstancePartitionsCountConstraint.class, 1.0f)
         .build();
   }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java
index 7ffc40b..63622e2 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java
@@ -38,7 +38,8 @@ public class TestInstancePartitionsCountConstraint {
   @Test
   public void testWhenInstanceIsIdle() {
     when(_testNode.getAssignedReplicaCount()).thenReturn(0);
-    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    float score =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
     Assert.assertEquals(score, 1.0f);
   }
 
@@ -46,7 +47,8 @@ public class TestInstancePartitionsCountConstraint {
   public void testWhenInstanceIsFull() {
     when(_testNode.getAssignedReplicaCount()).thenReturn(10);
     when(_clusterContext.getEstimatedMaxPartitionCount()).thenReturn(10);
-    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    float score =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
     Assert.assertEquals(score, 0.5f);
   }
 
@@ -54,7 +56,8 @@ public class TestInstancePartitionsCountConstraint {
   public void testWhenInstanceHalfOccupied() {
     when(_testNode.getAssignedReplicaCount()).thenReturn(10);
     when(_clusterContext.getEstimatedMaxPartitionCount()).thenReturn(20);
-    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    float score =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
     Assert.assertEquals(score, 0.75f);
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestMaxCapacityUsageInstanceConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestMaxCapacityUsageInstanceConstraint.java
new file mode 100644
index 0000000..7026dee
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestMaxCapacityUsageInstanceConstraint.java
@@ -0,0 +1,56 @@
+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 static org.mockito.Mockito.CALLS_REAL_METHODS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+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;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+public class TestMaxCapacityUsageInstanceConstraint {
+  private AssignableReplica _testReplica;
+  private AssignableNode _testNode;
+  private ClusterContext _clusterContext;
+  private final SoftConstraint _constraint = new MaxCapacityUsageInstanceConstraint();
+
+  @BeforeMethod
+  public void setUp() {
+    _testNode = mock(AssignableNode.class, CALLS_REAL_METHODS);
+    _testReplica = mock(AssignableReplica.class);
+    _clusterContext = mock(ClusterContext.class);
+  }
+
+  @Test
+  public void testGetNormalizedScore() {
+    when(_testNode.getHighestCapacityUtilization()).thenReturn(0.8f);
+    float score =
+            _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.6f);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(normalizedScore, 0.6f);
+  }
+}


[helix] 36/37: Load soft constraint weight from resources/properties file (#492)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 75b60d126a85a9f663b1faf8a2e5d3d779ad0d21
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Wed Oct 2 16:29:40 2019 -0700

    Load soft constraint weight from resources/properties file (#492)
    
    Load the soft constraint's weight from a properties file.
    It makes easier for us to adjust weights in the future.
---
 .../java/org/apache/helix/SystemPropertyKeys.java  |  2 +
 .../ConstraintBasedAlgorithmFactory.java           | 48 +++++++++++++++++-----
 .../resources/soft-constraint-weight.properties    | 24 +++++++++++
 3 files changed, 63 insertions(+), 11 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/SystemPropertyKeys.java b/helix-core/src/main/java/org/apache/helix/SystemPropertyKeys.java
index 1a6a797..d316986 100644
--- a/helix-core/src/main/java/org/apache/helix/SystemPropertyKeys.java
+++ b/helix-core/src/main/java/org/apache/helix/SystemPropertyKeys.java
@@ -6,6 +6,8 @@ public class SystemPropertyKeys {
 
   // ZKHelixManager
   public static final String CLUSTER_MANAGER_VERSION = "cluster-manager-version.properties";
+  // soft constraints weight definitions
+  public static final String SOFT_CONSTRAINT_WEIGHTS = "soft-constraint-weight.properties";
 
   public static final String FLAPPING_TIME_WINDOW = "helixmanager.flappingTimeWindow";
 
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
index 657fc82..f70de9a 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
@@ -19,14 +19,19 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
  * under the License.
  */
 
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
+import org.apache.helix.HelixManagerProperties;
+import org.apache.helix.SystemPropertyKeys;
 import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
 import org.apache.helix.model.ClusterConfig;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+
 /**
  * The factory class to create an instance of {@link ConstraintBasedAlgorithm}
  */
@@ -36,11 +41,29 @@ public class ConstraintBasedAlgorithmFactory {
   // enlarge the overall weight of the evenness constraints compared with the movement constraint.
   // TODO: Tune or make the following factor configurable.
   private static final int EVENNESS_PREFERENCE_NORMALIZE_FACTOR = 50;
+  private static final Map<String, Float> MODEL = new HashMap<>() {
+    {
+      // The default setting
+      put(PartitionMovementConstraint.class.getSimpleName(), 1f);
+      put(InstancePartitionsCountConstraint.class.getSimpleName(), 0.3f);
+      put(ResourcePartitionAntiAffinityConstraint.class.getSimpleName(), 0.1f);
+      put(ResourceTopStateAntiAffinityConstraint.class.getSimpleName(), 0.1f);
+      put(MaxCapacityUsageInstanceConstraint.class.getSimpleName(), 0.5f);
+    }
+  };
+
+  static {
+    Properties properties =
+        new HelixManagerProperties(SystemPropertyKeys.SOFT_CONSTRAINT_WEIGHTS).getProperties();
+    // overwrite the default value with data load from property file
+    properties.forEach((constraintName, weight) -> MODEL.put(String.valueOf(constraintName),
+        Float.valueOf(String.valueOf(weight))));
+  }
 
   public static RebalanceAlgorithm getInstance(
       Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
-    List<HardConstraint> hardConstraints = ImmutableList
-        .of(new FaultZoneAwareConstraint(), new NodeCapacityConstraint(),
+    List<HardConstraint> hardConstraints =
+        ImmutableList.of(new FaultZoneAwareConstraint(), new NodeCapacityConstraint(),
             new ReplicaActivateConstraint(), new NodeMaxPartitionLimitConstraint(),
             new ValidGroupTagConstraint(), new SamePartitionOnInstanceConstraint());
 
@@ -52,13 +75,16 @@ public class ConstraintBasedAlgorithmFactory {
     float evennessRatio = (float) evennessPreference / (evennessPreference + movementPreference);
     float movementRatio = (float) movementPreference / (evennessPreference + movementPreference);
 
-    Map<SoftConstraint, Float> softConstraints = ImmutableMap.<SoftConstraint, Float>builder()
-        .put(new PartitionMovementConstraint(), movementRatio)
-        .put(new InstancePartitionsCountConstraint(), 0.3f * evennessRatio)
-        .put(new ResourcePartitionAntiAffinityConstraint(), 0.1f * evennessRatio)
-        .put(new ResourceTopStateAntiAffinityConstraint(), 0.1f * evennessRatio)
-        .put(new MaxCapacityUsageInstanceConstraint(), 0.5f * evennessRatio).build();
+    List<SoftConstraint> softConstraints = ImmutableList.of(new PartitionMovementConstraint(),
+        new InstancePartitionsCountConstraint(), new ResourcePartitionAntiAffinityConstraint(),
+        new ResourceTopStateAntiAffinityConstraint(), new MaxCapacityUsageInstanceConstraint());
+    Map<SoftConstraint, Float> softConstraintsWithWeight = Maps.toMap(softConstraints, key -> {
+      String name = key.getClass().getSimpleName();
+      float weight = MODEL.get(name);
+      return name.equals(PartitionMovementConstraint.class.getSimpleName()) ? movementRatio * weight
+          : evennessRatio * weight;
+    });
 
-    return new ConstraintBasedAlgorithm(hardConstraints, softConstraints);
+    return new ConstraintBasedAlgorithm(hardConstraints, softConstraintsWithWeight);
   }
 }
diff --git a/helix-core/src/main/resources/soft-constraint-weight.properties b/helix-core/src/main/resources/soft-constraint-weight.properties
new file mode 100644
index 0000000..3e87c9d
--- /dev/null
+++ b/helix-core/src/main/resources/soft-constraint-weight.properties
@@ -0,0 +1,24 @@
+#
+# 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.
+#
+
+PartitionMovementConstraint=1f
+InstancePartitionsCountConstraint=0.3f
+ResourcePartitionAntiAffinityConstraint=0.1f
+ResourceTopStateAntiAffinityConstraint=0.1f
+MaxCapacityUsageInstanceConstraint=0.5f
\ No newline at end of file


[helix] 34/37: Enable maintenance mode for the WAGED rebalancer.

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1712a88724ee69d771e4da5bb234c56eb30898c8
Author: Jiajun Wang <jj...@linkedin.com>
AuthorDate: Sun Sep 29 15:50:25 2019 -0700

    Enable maintenance mode for the WAGED rebalancer.
    
    The maintenance mode rebalance logic keeps the same as the previous feature.
    Add more tests about partition migration and node swap that requires maintenance mode.
---
 .../ConstraintBasedAlgorithmFactory.java           |  19 +-
 .../stages/BestPossibleStateCalcStage.java         |  20 +-
 .../TestPartitionMigrationBase.java                |  15 +-
 .../PartitionMigration/TestWagedExpandCluster.java |  65 +++++
 .../TestWagedRebalancerMigration.java              | 108 ++++++++
 .../rebalancer/WagedRebalancer/TestNodeSwap.java   | 291 +++++++++++++++++++++
 6 files changed, 500 insertions(+), 18 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
index 8568444..657fc82 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
@@ -22,32 +22,37 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
-import org.apache.helix.model.ClusterConfig;
-
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
+import org.apache.helix.model.ClusterConfig;
 
 /**
  * The factory class to create an instance of {@link ConstraintBasedAlgorithm}
  */
 public class ConstraintBasedAlgorithmFactory {
+  // Evenness constraints tend to score within a smaller range.
+  // In order to let their scores cause enough difference in the final evaluation result, we need to
+  // enlarge the overall weight of the evenness constraints compared with the movement constraint.
+  // TODO: Tune or make the following factor configurable.
+  private static final int EVENNESS_PREFERENCE_NORMALIZE_FACTOR = 50;
 
   public static RebalanceAlgorithm getInstance(
       Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
-    List<HardConstraint> hardConstraints =
-        ImmutableList.of(new FaultZoneAwareConstraint(), new NodeCapacityConstraint(),
+    List<HardConstraint> hardConstraints = ImmutableList
+        .of(new FaultZoneAwareConstraint(), new NodeCapacityConstraint(),
             new ReplicaActivateConstraint(), new NodeMaxPartitionLimitConstraint(),
             new ValidGroupTagConstraint(), new SamePartitionOnInstanceConstraint());
 
     int evennessPreference =
-        preferences.getOrDefault(ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS, 1);
+        preferences.getOrDefault(ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS, 1)
+            * EVENNESS_PREFERENCE_NORMALIZE_FACTOR;
     int movementPreference =
         preferences.getOrDefault(ClusterConfig.GlobalRebalancePreferenceKey.LESS_MOVEMENT, 1);
     float evennessRatio = (float) evennessPreference / (evennessPreference + movementPreference);
     float movementRatio = (float) movementPreference / (evennessPreference + movementPreference);
 
-    Map<SoftConstraint, Float> softConstraints = ImmutableMap.<SoftConstraint, Float> builder()
+    Map<SoftConstraint, Float> softConstraints = ImmutableMap.<SoftConstraint, Float>builder()
         .put(new PartitionMovementConstraint(), movementRatio)
         .put(new InstancePartitionsCountConstraint(), 0.3f * evennessRatio)
         .put(new ResourcePartitionAntiAffinityConstraint(), 0.1f * evennessRatio)
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index cd7ab59..8c082f1 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -235,6 +236,11 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       ResourceControllerDataProvider cache, CurrentStateOutput currentStateOutput,
       HelixManager helixManager, Map<String, Resource> resourceMap, BestPossibleStateOutput output,
       List<String> failureResources) {
+    if (cache.isMaintenanceModeEnabled()) {
+      // The WAGED rebalancer won't be used while maintenance mode is enabled.
+      return Collections.emptyMap();
+    }
+
     // Find the compatible resources: 1. FULL_AUTO 2. Configured to use the WAGED rebalancer
     Map<String, Resource> wagedRebalancedResourceMap =
         resourceMap.entrySet().stream().filter(resourceEntry -> {
@@ -394,10 +400,9 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     }
   }
 
-  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState,
-      String resourceName, boolean isMaintenanceModeEnabled) {
+  private Rebalancer<ResourceControllerDataProvider> getCustomizedRebalancer(
+      String rebalancerClassName, String resourceName) {
     Rebalancer<ResourceControllerDataProvider> customizedRebalancer = null;
-    String rebalancerClassName = idealState.getRebalancerClassName();
     if (rebalancerClassName != null) {
       if (logger.isDebugEnabled()) {
         LogUtil.logDebug(logger, _eventId,
@@ -411,13 +416,19 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
             "Exception while invoking custom rebalancer class:" + rebalancerClassName, e);
       }
     }
+    return customizedRebalancer;
+  }
 
+  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState,
+      String resourceName, boolean isMaintenanceModeEnabled) {
     Rebalancer<ResourceControllerDataProvider> rebalancer = null;
     switch (idealState.getRebalanceMode()) {
     case FULL_AUTO:
       if (isMaintenanceModeEnabled) {
         rebalancer = new MaintenanceRebalancer();
       } else {
+        Rebalancer<ResourceControllerDataProvider> customizedRebalancer =
+            getCustomizedRebalancer(idealState.getRebalancerClassName(), resourceName);
         if (customizedRebalancer != null) {
           rebalancer = customizedRebalancer;
         } else {
@@ -433,14 +444,13 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       break;
     case USER_DEFINED:
     case TASK:
-      rebalancer = customizedRebalancer;
+      rebalancer = getCustomizedRebalancer(idealState.getRebalancerClassName(), resourceName);
       break;
     default:
       LogUtil.logError(logger, _eventId,
           "Fail to find the rebalancer, invalid rebalance mode " + idealState.getRebalanceMode());
       break;
     }
-
     return rebalancer;
   }
 
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
index 7a559a5..61d72a2 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestPartitionMigrationBase.java
@@ -57,14 +57,14 @@ public class TestPartitionMigrationBase extends ZkTestBase {
   protected ClusterControllerManager _controller;
 
   List<MockParticipantManager> _participants = new ArrayList<>();
-  int _replica = 3;
-  int _minActiveReplica = _replica - 1;
+  protected int _replica = 3;
+  protected int _minActiveReplica = _replica - 1;
   ZkHelixClusterVerifier _clusterVerifier;
-  List<String> _testDBs = new ArrayList<>();
+  protected List<String> _testDBs = new ArrayList<>();
 
   MigrationStateVerifier _migrationVerifier;
   HelixManager _manager;
-  ConfigAccessor _configAccessor;
+  protected ConfigAccessor _configAccessor;
 
 
   @BeforeClass
@@ -84,8 +84,7 @@ public class TestPartitionMigrationBase extends ZkTestBase {
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    _clusterVerifier = getVerifier();
 
     enablePersistIntermediateAssignment(_gZkClient, CLUSTER_NAME, true);
 
@@ -95,6 +94,10 @@ public class TestPartitionMigrationBase extends ZkTestBase {
     _configAccessor = new ConfigAccessor(_gZkClient);
   }
 
+  protected ZkHelixClusterVerifier getVerifier() {
+    return new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+  }
+
   protected MockParticipantManager createAndStartParticipant(String instancename) {
     _gSetupTool.addInstanceToCluster(CLUSTER_NAME, instancename);
 
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedExpandCluster.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedExpandCluster.java
new file mode 100644
index 0000000..d303e87
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedExpandCluster.java
@@ -0,0 +1,65 @@
+package org.apache.helix.integration.rebalancer.PartitionMigration;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+
+public class TestWagedExpandCluster extends TestExpandCluster {
+// TODO check the movements in between
+  protected ZkHelixClusterVerifier getVerifier() {
+    Set<String> dbNames = new HashSet<>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      dbNames.add("Test-DB-" + i++);
+    }
+    return new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setResources(dbNames)
+        .setZkAddr(ZK_ADDR).build();
+  }
+
+  protected Map<String, IdealState> createTestDBs(long delayTime) {
+    Map<String, IdealState> idealStateMap = new HashMap<>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+          _minActiveReplica);
+      _testDBs.add(db);
+    }
+    for (String db : _testDBs) {
+      IdealState is =
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      idealStateMap.put(db, is);
+    }
+    ClusterConfig clusterConfig = _configAccessor.getClusterConfig(CLUSTER_NAME);
+    clusterConfig.setDelayRebalaceEnabled(true);
+    clusterConfig.setRebalanceDelayTime(delayTime);
+    _configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+    return idealStateMap;
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedRebalancerMigration.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedRebalancerMigration.java
new file mode 100644
index 0000000..1a13496
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/PartitionMigration/TestWagedRebalancerMigration.java
@@ -0,0 +1,108 @@
+package org.apache.helix.integration.rebalancer.PartitionMigration;
+
+/*
+ * 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.Collections;
+
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
+import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+public class TestWagedRebalancerMigration extends TestPartitionMigrationBase {
+  ConfigAccessor _configAccessor;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    super.beforeClass();
+    _configAccessor = new ConfigAccessor(_gZkClient);
+  }
+
+  @DataProvider(name = "stateModels")
+  public static Object[][] stateModels() {
+    return new Object[][] { { BuiltInStateModelDefinitions.MasterSlave.name(), true },
+        { BuiltInStateModelDefinitions.OnlineOffline.name(), true },
+        { BuiltInStateModelDefinitions.LeaderStandby.name(), true },
+        { BuiltInStateModelDefinitions.MasterSlave.name(), false },
+        { BuiltInStateModelDefinitions.OnlineOffline.name(), false },
+        { BuiltInStateModelDefinitions.LeaderStandby.name(), false },
+    };
+  }
+
+  // TODO check the movements in between
+  @Test(dataProvider = "stateModels")
+  public void testMigrateToWagedRebalancerWhileExpandCluster(String stateModel,
+      boolean delayEnabled) throws Exception {
+    String db = "Test-DB-" + stateModel;
+    if (delayEnabled) {
+      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+          _replica - 1, 3000000, CrushRebalanceStrategy.class.getName());
+    } else {
+      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+          _replica, 0, CrushRebalanceStrategy.class.getName());
+    }
+    IdealState idealState =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+    ClusterConfig config = _configAccessor.getClusterConfig(CLUSTER_NAME);
+    config.setDelayRebalaceEnabled(delayEnabled);
+    config.setRebalanceDelayTime(3000000);
+    _configAccessor.setClusterConfig(CLUSTER_NAME, config);
+
+    // add new instance to the cluster
+    int numNodes = _participants.size();
+    for (int i = numNodes; i < numNodes + NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      MockParticipantManager participant = createAndStartParticipant(storageNodeName);
+      _participants.add(participant);
+      Thread.sleep(100);
+    }
+    Thread.sleep(2000);
+    ZkHelixClusterVerifier clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME)
+            .setResources(Collections.singleton(db)).setZkAddr(ZK_ADDR).build();
+    Assert.assertTrue(clusterVerifier.verifyByPolling());
+
+    _migrationVerifier =
+        new MigrationStateVerifier(Collections.singletonMap(db, idealState), _manager);
+
+    _migrationVerifier.reset();
+    _migrationVerifier.start();
+
+    IdealState currentIdealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+    currentIdealState.setRebalancerClassName(WagedRebalancer.class.getName());
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, currentIdealState);
+    Thread.sleep(2000);
+    Assert.assertTrue(clusterVerifier.verifyByPolling());
+
+    Assert.assertFalse(_migrationVerifier.hasLessReplica());
+    Assert.assertFalse(_migrationVerifier.hasMoreReplica());
+
+    _migrationVerifier.stop();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestNodeSwap.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestNodeSwap.java
new file mode 100644
index 0000000..360e495
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestNodeSwap.java
@@ -0,0 +1,291 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestNodeSwap extends ZkTestBase {
+  final int NUM_NODE = 6;
+  protected static final int START_PORT = 12918;
+  protected static final int _PARTITIONS = 20;
+
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  protected ClusterControllerManager _controller;
+  protected HelixClusterVerifier _clusterVerifier;
+
+  List<MockParticipantManager> _participants = new ArrayList<>();
+  Set<String> _allDBs = new HashSet<>();
+  int _replica = 3;
+
+  String[] _testModels = { BuiltInStateModelDefinitions.OnlineOffline.name(),
+      BuiltInStateModelDefinitions.MasterSlave.name(),
+      BuiltInStateModelDefinitions.LeaderStandby.name()
+  };
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+    clusterConfig.setTopology("/zone/instance");
+    clusterConfig.setFaultZoneType("zone");
+    clusterConfig.setDelayRebalaceEnabled(true);
+    // Set a long enough time to ensure delayed rebalance is activate
+    clusterConfig.setRebalanceDelayTime(3000000);
+    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+    Set<String> nodes = new HashSet<>();
+    for (int i = 0; i < NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+      String zone = "zone-" + i % 3;
+      String domain = String.format("zone=%s,instance=%s", zone, storageNodeName);
+
+      InstanceConfig instanceConfig =
+          configAccessor.getInstanceConfig(CLUSTER_NAME, storageNodeName);
+      instanceConfig.setDomain(domain);
+      _gSetupTool.getClusterManagementTool()
+          .setInstanceConfig(CLUSTER_NAME, storageNodeName, instanceConfig);
+      nodes.add(storageNodeName);
+    }
+
+    // start dummy participants
+    for (String node : nodes) {
+      MockParticipantManager participant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, node);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+    enableTopologyAwareRebalance(_gZkClient, CLUSTER_NAME, true);
+
+    int i = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+          _replica - 1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(1000);
+
+    _clusterVerifier = new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verify(5000));
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    _controller.syncStop();
+    for (MockParticipantManager p : _participants) {
+      p.syncStop();
+    }
+    deleteCluster(CLUSTER_NAME);
+  }
+
+  @Test
+  public void testNodeSwap() throws Exception {
+    Map<String, ExternalView> record = new HashMap<>();
+    for (String db : _allDBs) {
+      record.put(db,
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db));
+    }
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+
+    // 1. disable an old node
+    MockParticipantManager oldParticipant = _participants.get(0);
+    String oldParticipantName = oldParticipant.getInstanceName();
+    final InstanceConfig instanceConfig =
+        _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, oldParticipantName);
+    instanceConfig.setInstanceEnabled(false);
+    _gSetupTool.getClusterManagementTool()
+        .setInstanceConfig(CLUSTER_NAME, oldParticipantName, instanceConfig);
+    Assert.assertTrue(_clusterVerifier.verify(10000));
+
+    // 2. then entering maintenance mode and remove it from topology
+    _gSetupTool.getClusterManagementTool()
+        .manuallyEnableMaintenanceMode(CLUSTER_NAME, true, "NodeSwap", Collections.emptyMap());
+    oldParticipant.syncStop();
+    _participants.remove(oldParticipant);
+    Thread.sleep(2000);
+    _gSetupTool.getClusterManagementTool().dropInstance(CLUSTER_NAME, instanceConfig);
+
+    // 3. create new participant with same topology
+    String newParticipantName = "RandomParticipant_" + START_PORT;
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, newParticipantName);
+    InstanceConfig newConfig = configAccessor.getInstanceConfig(CLUSTER_NAME, newParticipantName);
+    String zone = instanceConfig.getDomainAsMap().get("zone");
+    String domain = String.format("zone=%s,instance=%s", zone, newParticipantName);
+    newConfig.setDomain(domain);
+    _gSetupTool.getClusterManagementTool()
+        .setInstanceConfig(CLUSTER_NAME, newParticipantName, newConfig);
+
+    MockParticipantManager participant =
+        new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, newParticipantName);
+    participant.syncStart();
+    _participants.add(0, participant);
+
+    // 4. exit maintenance mode and rebalance
+    _gSetupTool.getClusterManagementTool()
+        .manuallyEnableMaintenanceMode(CLUSTER_NAME, false, "NodeSwapDone", Collections.emptyMap());
+
+    Thread.sleep(2000);
+    _clusterVerifier = new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verify(5000));
+
+    // Since only one node temporary down, the same partitions will be moved to the newly added node.
+    for (String db : _allDBs) {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      ExternalView oldEv = record.get(db);
+      for (String partition : ev.getPartitionSet()) {
+        Map<String, String> stateMap = ev.getStateMap(partition);
+        Map<String, String> oldStateMap = oldEv.getStateMap(partition);
+        Assert.assertTrue(oldStateMap != null && stateMap != null);
+        Assert.assertEquals(stateMap.size(), _replica);
+        // Note the WAGED rebalanacer won't ensure the same state, because moving the top states
+        // back to the replaced node might be unnecessary and causing overhead.
+        Set<String> instanceSet = new HashSet<>(stateMap.keySet());
+        if (instanceSet.remove(newParticipantName)) {
+          instanceSet.add(oldParticipantName);
+        }
+        Assert.assertEquals(oldStateMap.keySet(), instanceSet);
+      }
+    }
+  }
+
+  @Test(dependsOnMethods = "testNodeSwap")
+  public void testFaultZoneSwap() throws Exception {
+    Map<String, ExternalView> record = new HashMap<>();
+    for (String db : _allDBs) {
+      record.put(db,
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db));
+    }
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+
+    // 1. disable a whole fault zone
+    Map<String, InstanceConfig> removedInstanceConfigMap = new HashMap<>();
+    for (MockParticipantManager participant : _participants) {
+      String instanceName = participant.getInstanceName();
+      InstanceConfig instanceConfig =
+          _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instanceName);
+      if (instanceConfig.getDomainAsMap().get("zone").equals("zone-0")) {
+        instanceConfig.setInstanceEnabled(false);
+        _gSetupTool.getClusterManagementTool()
+            .setInstanceConfig(CLUSTER_NAME, instanceName, instanceConfig);
+        removedInstanceConfigMap.put(instanceName, instanceConfig);
+      }
+    }
+    Assert.assertTrue(_clusterVerifier.verify(10000));
+
+    // 2. then entering maintenance mode and remove all the zone-0 nodes from topology
+    _gSetupTool.getClusterManagementTool()
+        .manuallyEnableMaintenanceMode(CLUSTER_NAME, true, "NodeSwap", Collections.emptyMap());
+    Iterator<MockParticipantManager> iter = _participants.iterator();
+    while(iter.hasNext()) {
+      MockParticipantManager participant = iter.next();
+      String instanceName = participant.getInstanceName();
+      if (removedInstanceConfigMap.containsKey(instanceName)) {
+        participant.syncStop();
+        iter.remove();
+        Thread.sleep(1000);
+        _gSetupTool.getClusterManagementTool()
+            .dropInstance(CLUSTER_NAME, removedInstanceConfigMap.get(instanceName));
+      }
+    }
+
+    // 3. create new participants with same topology
+    Set<String> newInstanceNames = new HashSet<>();
+    for (int i = 0; i < removedInstanceConfigMap.size(); i++) {
+      String newParticipantName = "NewParticipant_" + (START_PORT + i++);
+      newInstanceNames.add(newParticipantName);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, newParticipantName);
+      InstanceConfig newConfig = configAccessor.getInstanceConfig(CLUSTER_NAME, newParticipantName);
+      String domain = String.format("zone=zone-0,instance=%s", newParticipantName);
+      newConfig.setDomain(domain);
+      _gSetupTool.getClusterManagementTool()
+          .setInstanceConfig(CLUSTER_NAME, newParticipantName, newConfig);
+
+      MockParticipantManager participant =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, newParticipantName);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // 4. exit maintenance mode and rebalance
+    _gSetupTool.getClusterManagementTool()
+        .manuallyEnableMaintenanceMode(CLUSTER_NAME, false, "NodeSwapDone", Collections.emptyMap());
+
+    Thread.sleep(2000);
+    _clusterVerifier = new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verify(5000));
+
+    for (String db : _allDBs) {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      ExternalView oldEv = record.get(db);
+      for (String partition : ev.getPartitionSet()) {
+        Map<String, String> stateMap = ev.getStateMap(partition);
+        Map<String, String> oldStateMap = oldEv.getStateMap(partition);
+        Assert.assertTrue(oldStateMap != null && stateMap != null);
+        Assert.assertEquals(stateMap.size(), _replica);
+        Set<String> instanceSet = new HashSet<>(stateMap.keySet());
+        instanceSet.removeAll(oldStateMap.keySet());
+        // All the different instances in the new mapping are the newly added instance
+        Assert.assertTrue(newInstanceNames.containsAll(instanceSet));
+        instanceSet = new HashSet<>(oldStateMap.keySet());
+        instanceSet.removeAll(stateMap.keySet());
+        // All the different instances in the old mapping are the removed instance
+        Assert.assertTrue(removedInstanceConfigMap.keySet().containsAll(instanceSet));
+      }
+    }
+  }
+}


[helix] 29/37: PartitionMovementSoftConstraint Implementation (#474)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 50e8ceccd35dfa791b066ff85e8e6a78555511b8
Author: Yi Wang <yw...@linkedin.com>
AuthorDate: Tue Sep 17 15:09:45 2019 -0700

    PartitionMovementSoftConstraint Implementation (#474)
    
    Add soft constraint: partition movement constraint
    
    Evaluate the proposed assignment according to the potential partition movements cost.
    The cost is evaluated based on the difference between the old assignment and the new assignment.
---
 .../constraints/PartitionMovementConstraint.java   |  96 ++++++++++++++++
 .../rebalancer/waged/model/ClusterContext.java     |  43 +++++--
 .../rebalancer/waged/model/ClusterModel.java       |  27 +----
 .../waged/model/ClusterModelProvider.java          |   5 +-
 .../TestPartitionMovementConstraint.java           | 127 +++++++++++++++++++++
 .../waged/model/ClusterModelTestHelper.java        |   6 +-
 .../rebalancer/waged/model/TestClusterContext.java |  17 +--
 .../rebalancer/waged/model/TestClusterModel.java   |  15 ++-
 8 files changed, 278 insertions(+), 58 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/PartitionMovementConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/PartitionMovementConstraint.java
new file mode 100644
index 0000000..a781afc
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/PartitionMovementConstraint.java
@@ -0,0 +1,96 @@
+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 java.util.Collections;
+import java.util.Map;
+
+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;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.ResourceAssignment;
+
+/**
+ * Evaluate the proposed assignment according to the potential partition movements cost.
+ * The cost is evaluated based on the difference between the old assignment and the new assignment.
+ * In detail, we consider the following two previous assignments as the base.
+ * - Baseline assignment that is calculated regardless of the node state (online/offline).
+ * - Previous Best Possible assignment.
+ * Any change to these two assignments will increase the partition movements cost, so that the
+ * evaluated score will become lower.
+ */
+class PartitionMovementConstraint extends SoftConstraint {
+  private static final float MAX_SCORE = 1f;
+  private static final float MIN_SCORE = 0f;
+  //TODO: these factors will be tuned based on user's preference
+  // This factor indicates the default score that is evaluated if only partition allocation matches
+  // (states are different).
+  private static final float ALLOCATION_MATCH_FACTOR = 0.5f;
+  // This factor indicates the contribution of the Baseline assignment matching to the final score.
+  private static final float BASELINE_MATCH_FACTOR = 0.25f;
+
+  PartitionMovementConstraint() {
+    super(MAX_SCORE, MIN_SCORE);
+  }
+
+  @Override
+  protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    Map<String, String> bestPossibleStateMap =
+        getStateMap(replica, clusterContext.getBestPossibleAssignment());
+    Map<String, String> baselineStateMap =
+        getStateMap(replica, clusterContext.getBaselineAssignment());
+
+    // Prioritize the matching of the previous Best Possible assignment.
+    float scale = calculateAssignmentScale(node, replica, bestPossibleStateMap);
+    // If the baseline is also provided, adjust the final score accordingly.
+    scale = scale * (1 - BASELINE_MATCH_FACTOR)
+        + calculateAssignmentScale(node, replica, baselineStateMap) * BASELINE_MATCH_FACTOR;
+
+    return scale;
+  }
+
+  @Override
+  NormalizeFunction getNormalizeFunction() {
+    return score -> score * (getMaxScore() - getMinScore()) + getMinScore();
+  }
+
+  private Map<String, String> getStateMap(AssignableReplica replica,
+      Map<String, ResourceAssignment> assignment) {
+    String resourceName = replica.getResourceName();
+    String partitionName = replica.getPartitionName();
+    if (assignment == null || !assignment.containsKey(resourceName)) {
+      return Collections.emptyMap();
+    }
+    return assignment.get(resourceName).getReplicaMap(new Partition(partitionName));
+  }
+
+  private float calculateAssignmentScale(AssignableNode node, AssignableReplica replica,
+      Map<String, String> instanceToStateMap) {
+    String instanceName = node.getInstanceName();
+    if (!instanceToStateMap.containsKey(instanceName)) {
+      return 0;
+    } else {
+      return (instanceToStateMap.get(instanceName).equals(replica.getReplicaState()) ? 1
+          : ALLOCATION_MATCH_FACTOR);
+    }
+  }
+}
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
index a0c841a..892cad3 100644
--- 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
@@ -28,6 +28,8 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.helix.HelixException;
+import org.apache.helix.model.ResourceAssignment;
+
 
 /**
  * This class tracks the rebalance-related global cluster status.
@@ -44,30 +46,47 @@ public class ClusterContext {
 
   // map{zoneName : map{resourceName : set(partitionNames)}}
   private Map<String, Map<String, Set<String>>> _assignmentForFaultZoneMap = new HashMap<>();
+  // Records about the previous assignment
+  // <ResourceName, ResourceAssignment contains the baseline assignment>
+  private final Map<String, ResourceAssignment> _baselineAssignment;
+  // <ResourceName, ResourceAssignment contains the best possible assignment>
+  private final Map<String, ResourceAssignment> _bestPossibleAssignment;
 
   /**
    * Construct the cluster context based on the current instance status.
    * @param replicaSet All the partition replicas that are managed by the rebalancer
    * @param instanceCount The count of all the active instances that can be used to host partitions.
    */
-  ClusterContext(Set<AssignableReplica> replicaSet, int instanceCount) {
+  ClusterContext(Set<AssignableReplica> replicaSet, int instanceCount,
+      Map<String, ResourceAssignment> baselineAssignment, Map<String, ResourceAssignment> bestPossibleAssignment) {
     int totalReplicas = 0;
     int totalTopStateReplicas = 0;
 
     for (Map.Entry<String, List<AssignableReplica>> entry : replicaSet.stream()
-        .collect(Collectors.groupingBy(AssignableReplica::getResourceName)).entrySet()) {
+        .collect(Collectors.groupingBy(AssignableReplica::getResourceName))
+        .entrySet()) {
       int replicas = entry.getValue().size();
       totalReplicas += replicas;
 
       int replicaCnt = Math.max(1, estimateAvgReplicaCount(replicas, instanceCount));
       _estimatedMaxPartitionByResource.put(entry.getKey(), replicaCnt);
 
-      totalTopStateReplicas +=
-          entry.getValue().stream().filter(AssignableReplica::isReplicaTopState).count();
+      totalTopStateReplicas += entry.getValue().stream().filter(AssignableReplica::isReplicaTopState).count();
     }
 
     _estimatedMaxPartitionCount = estimateAvgReplicaCount(totalReplicas, instanceCount);
     _estimatedMaxTopStateCount = estimateAvgReplicaCount(totalTopStateReplicas, instanceCount);
+    _baselineAssignment = baselineAssignment;
+    _bestPossibleAssignment = bestPossibleAssignment;
+  }
+
+  public Map<String, ResourceAssignment> getBaselineAssignment() {
+    return _baselineAssignment == null || _baselineAssignment.isEmpty() ? Collections.emptyMap() : _baselineAssignment;
+  }
+
+  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
+    return _bestPossibleAssignment == null || _bestPossibleAssignment.isEmpty() ? Collections.emptyMap()
+        : _bestPossibleAssignment;
   }
 
   public Map<String, Map<String, Set<String>>> getAssignmentForFaultZoneMap() {
@@ -93,25 +112,25 @@ public class ClusterContext {
 
   void addPartitionToFaultZone(String faultZoneId, String resourceName, String partition) {
     if (!_assignmentForFaultZoneMap.computeIfAbsent(faultZoneId, k -> new HashMap<>())
-        .computeIfAbsent(resourceName, k -> new HashSet<>()).add(partition)) {
+        .computeIfAbsent(resourceName, k -> new HashSet<>())
+        .add(partition)) {
       throw new HelixException(
-          String.format("Resource %s already has a replica from partition %s in fault zone %s",
-              resourceName, partition, faultZoneId));
+          String.format("Resource %s already has a replica from partition %s in fault zone %s", resourceName, partition,
+              faultZoneId));
     }
   }
 
   boolean removePartitionFromFaultZone(String faultZoneId, String resourceName, String partition) {
     return _assignmentForFaultZoneMap.getOrDefault(faultZoneId, Collections.emptyMap())
-        .getOrDefault(resourceName, Collections.emptySet()).remove(partition);
+        .getOrDefault(resourceName, Collections.emptySet())
+        .remove(partition);
   }
 
-  void setAssignmentForFaultZoneMap(
-      Map<String, Map<String, Set<String>>> assignmentForFaultZoneMap) {
+  void setAssignmentForFaultZoneMap(Map<String, Map<String, Set<String>>> assignmentForFaultZoneMap) {
     _assignmentForFaultZoneMap = assignmentForFaultZoneMap;
   }
 
   private int estimateAvgReplicaCount(int replicaCount, int instanceCount) {
-    return (int) Math
-        .ceil((float) replicaCount / instanceCount * ERROR_MARGIN_FOR_ESTIMATED_MAX_COUNT);
+    return (int) Math.ceil((float) replicaCount / instanceCount * ERROR_MARGIN_FOR_ESTIMATED_MAX_COUNT);
   }
 }
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
index 6c4e67b..3d31c04 100644
--- 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
@@ -19,14 +19,13 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
-import org.apache.helix.model.ResourceAssignment;
-
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.helix.HelixException;
+
 /**
  * This class wraps the required input for the rebalance algorithm.
  */
@@ -39,23 +38,14 @@ public class ClusterModel {
   private final Map<String, Map<String, AssignableReplica>> _assignableReplicaIndex;
   private final Map<String, AssignableNode> _assignableNodeMap;
 
-  // Records about the previous assignment
-  // <ResourceName, ResourceAssignment contains the baseline assignment>
-  private final Map<String, ResourceAssignment> _baselineAssignment;
-  // <ResourceName, ResourceAssignment contains the best possible assignment>
-  private final Map<String, ResourceAssignment> _bestPossibleAssignment;
-
   /**
    * @param clusterContext         The initialized cluster context.
    * @param assignableReplicas     The replicas to be assigned.
    *                               Note that the replicas in this list shall not be included while initializing the context and assignable nodes.
    * @param assignableNodes        The active instances.
-   * @param baselineAssignment     The recorded baseline assignment.
-   * @param bestPossibleAssignment The current best possible assignment.
    */
   ClusterModel(ClusterContext clusterContext, Set<AssignableReplica> assignableReplicas,
-      Set<AssignableNode> assignableNodes, Map<String, ResourceAssignment> baselineAssignment,
-      Map<String, ResourceAssignment> bestPossibleAssignment) {
+      Set<AssignableNode> assignableNodes) {
     _clusterContext = clusterContext;
 
     // Save all the to be assigned replication
@@ -70,9 +60,6 @@ public class ClusterModel {
 
     _assignableNodeMap = assignableNodes.stream()
         .collect(Collectors.toMap(AssignableNode::getInstanceName, node -> node));
-
-    _baselineAssignment = baselineAssignment;
-    _bestPossibleAssignment = bestPossibleAssignment;
   }
 
   public ClusterContext getContext() {
@@ -87,14 +74,6 @@ public class ClusterModel {
     return _assignableReplicaMap;
   }
 
-  public Map<String, ResourceAssignment> getBaseline() {
-    return _baselineAssignment;
-  }
-
-  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
-    return _bestPossibleAssignment;
-  }
-
   /**
    * Assign the given replica to the specified instance and record the assignment in the cluster model.
    * The cluster usage information will be updated accordingly.
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 20024c7..af1a8d8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -86,12 +86,11 @@ public class ClusterModelProvider {
     // Construct and initialize cluster context.
     ClusterContext context = new ClusterContext(
         replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()),
-        activeInstances.size());
+        activeInstances.size(), baselineAssignment, bestPossibleAssignment);
     // Initial the cluster context with the allocated assignments.
     context.setAssignmentForFaultZoneMap(mapAssignmentToFaultZone(assignableNodes));
 
-    return new ClusterModel(context, toBeAssignedReplicas, assignableNodes, baselineAssignment,
-        bestPossibleAssignment);
+    return new ClusterModel(context, toBeAssignedReplicas, assignableNodes);
   }
 
   /**
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionMovementConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionMovementConstraint.java
new file mode 100644
index 0000000..d3af35e
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestPartitionMovementConstraint.java
@@ -0,0 +1,127 @@
+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 static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Collections;
+import java.util.Map;
+
+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;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.ResourceAssignment;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TestPartitionMovementConstraint {
+  private static final String INSTANCE = "TestInstance";
+  private static final String RESOURCE = "TestResource";
+  private static final String PARTITION = "TestPartition";
+  private AssignableNode _testNode;
+  private AssignableReplica _testReplica;
+  private ClusterContext _clusterContext;
+  private SoftConstraint _constraint = new PartitionMovementConstraint();
+
+  @BeforeMethod
+  public void init() {
+    _testNode = mock(AssignableNode.class);
+    _testReplica = mock(AssignableReplica.class);
+    _clusterContext = mock(ClusterContext.class);
+    when(_testReplica.getResourceName()).thenReturn(RESOURCE);
+    when(_testReplica.getPartitionName()).thenReturn(PARTITION);
+    when(_testNode.getInstanceName()).thenReturn(INSTANCE);
+  }
+
+  @Test
+  public void testGetAssignmentScoreWhenBestPossibleBaselineMissing() {
+    when(_clusterContext.getBaselineAssignment()).thenReturn(Collections.emptyMap());
+    when(_clusterContext.getBestPossibleAssignment()).thenReturn(Collections.emptyMap());
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0f);
+    Assert.assertEquals(normalizedScore, 0f);
+  }
+
+  @Test
+  public void testGetAssignmentScoreWhenBestPossibleBaselineSame() {
+    ResourceAssignment mockResourceAssignment = mock(ResourceAssignment.class);
+    when(mockResourceAssignment.getReplicaMap(new Partition(PARTITION)))
+        .thenReturn(ImmutableMap.of(INSTANCE, "Master"));
+    Map<String, ResourceAssignment> assignmentMap =
+        ImmutableMap.of(RESOURCE, mockResourceAssignment);
+    when(_clusterContext.getBaselineAssignment()).thenReturn(assignmentMap);
+    when(_clusterContext.getBestPossibleAssignment()).thenReturn(assignmentMap);
+    // when the calculated states are both equal to the replica's current state
+    when(_testReplica.getReplicaState()).thenReturn("Master");
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+
+    Assert.assertEquals(score, 1f);
+    Assert.assertEquals(normalizedScore, 1f);
+    // when the calculated states are both different from the replica's current state
+    when(_testReplica.getReplicaState()).thenReturn("Slave");
+    score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+
+    Assert.assertEquals(score, 0.5f);
+    Assert.assertEquals(normalizedScore, 0.5f);
+  }
+
+  @Test
+  public void testGetAssignmentScoreWhenBestPossibleBaselineOpposite() {
+    ResourceAssignment bestPossibleResourceAssignment = mock(ResourceAssignment.class);
+    when(bestPossibleResourceAssignment.getReplicaMap(new Partition(PARTITION)))
+        .thenReturn(ImmutableMap.of(INSTANCE, "Master"));
+    ResourceAssignment baselineResourceAssignment = mock(ResourceAssignment.class);
+    when(baselineResourceAssignment.getReplicaMap(new Partition(PARTITION)))
+        .thenReturn(ImmutableMap.of(INSTANCE, "Slave"));
+    when(_clusterContext.getBaselineAssignment())
+        .thenReturn(ImmutableMap.of(RESOURCE, baselineResourceAssignment));
+    when(_clusterContext.getBestPossibleAssignment())
+        .thenReturn(ImmutableMap.of(RESOURCE, bestPossibleResourceAssignment));
+    // when the replica's state matches with best possible only
+    when(_testReplica.getReplicaState()).thenReturn("Master");
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+
+    Assert.assertEquals(score, 0.875f);
+    Assert.assertEquals(normalizedScore, 0.875f);
+    // when the replica's state matches with baseline only
+    when(_testReplica.getReplicaState()).thenReturn("Slave");
+    score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+
+    // The calculated score is lower than previous 0.875f cause the replica's state matches with
+    // best possible is preferred
+    Assert.assertEquals(score, 0.625f);
+    Assert.assertEquals(normalizedScore, 0.625f);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
index 08143c6..585c26f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
@@ -34,9 +34,9 @@ public class ClusterModelTestHelper extends AbstractTestClusterModel {
     Set<AssignableReplica> assignableReplicas = generateReplicas(testCache);
     Set<AssignableNode> assignableNodes = generateNodes(testCache);
 
-    ClusterContext context = new ClusterContext(assignableReplicas, 2);
-    return new ClusterModel(context, assignableReplicas, assignableNodes, Collections.emptyMap(),
-        Collections.emptyMap());
+    ClusterContext context =
+        new ClusterContext(assignableReplicas, 2, Collections.emptyMap(), Collections.emptyMap());
+    return new ClusterModel(context, assignableReplicas, assignableNodes);
   }
 
   private Set<AssignableNode> generateNodes(ResourceControllerDataProvider testCache) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java
index 8206f29..d8b93c0 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java
@@ -19,18 +19,19 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
-import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.testng.Assert;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
 public class TestClusterContext extends AbstractTestClusterModel {
   @BeforeClass
   public void initialize() {
@@ -43,7 +44,7 @@ public class TestClusterContext extends AbstractTestClusterModel {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
     Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
 
-    ClusterContext context = new ClusterContext(assignmentSet, 2);
+    ClusterContext context = new ClusterContext(assignmentSet, 2, new HashMap<>(), new HashMap<>());
 
     // Note that we left some margin for the max estimation.
     Assert.assertEquals(context.getEstimatedMaxPartitionCount(), 3);
@@ -80,7 +81,7 @@ public class TestClusterContext extends AbstractTestClusterModel {
   public void testDuplicateAssign() throws IOException {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
     Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
-    ClusterContext context = new ClusterContext(assignmentSet, 2);
+    ClusterContext context = new ClusterContext(assignmentSet, 2, new HashMap<>(), new HashMap<>());
     context
         .addPartitionToFaultZone(_testFaultZoneId, _resourceNames.get(0), _partitionNames.get(0));
     // Insert again and trigger the error.
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
index 5112413..12146b7 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
@@ -19,17 +19,17 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.helix.HelixException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
 public class TestClusterModel extends AbstractTestClusterModel {
   @BeforeClass
   public void initialize() {
@@ -54,10 +54,9 @@ public class TestClusterModel extends AbstractTestClusterModel {
     Set<AssignableReplica> assignableReplicas = generateReplicas(testCache);
     Set<AssignableNode> assignableNodes = generateNodes(testCache);
 
-    ClusterContext context = new ClusterContext(assignableReplicas, 2);
+    ClusterContext context = new ClusterContext(assignableReplicas, 2, Collections.emptyMap(), Collections.emptyMap());
     ClusterModel clusterModel =
-        new ClusterModel(context, assignableReplicas, assignableNodes, Collections.emptyMap(),
-            Collections.emptyMap());
+        new ClusterModel(context, assignableReplicas, assignableNodes);
 
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
         .allMatch(resourceMap -> resourceMap.values().isEmpty()));


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

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jiajunwang pushed a commit to branch wagedRebalancer
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 { }


[helix] 10/37: Refine the WAGED rebalancer related interfaces for integration (#431)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c3d8501e1fea597be9bb7fbfee914ced0e3869a9
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Wed Aug 28 22:28:10 2019 -0700

    Refine the WAGED rebalancer related interfaces for integration (#431)
    
    * Refine the WAGED rebalancer related interfaces and initial integrate with the BestPossibleStateCalStage.
    
    - Modify the BestPossibleStateCalStage logic to plugin the WAGED rebalancer.
    - Refine ClusterModel to integrate with the ClusterDataDetector implementation.
    - Enabling getting the changed details for Cluster Config in the change detector. Which is required by the WAGED rebalancer.
---
 .../org/apache/helix/HelixRebalanceException.java  | 43 ++++++++++
 .../changedetector/ResourceChangeDetector.java     | 20 +++--
 .../controller/rebalancer/GlobalRebalancer.java    | 67 ---------------
 .../rebalancer/waged/ClusterDataDetector.java      | 73 ----------------
 .../rebalancer/waged/ClusterDataProvider.java      | 54 ------------
 .../rebalancer/waged/WagedRebalancer.java          | 65 ++++++++++----
 .../waged/model/ClusterModelProvider.java          | 25 +++---
 .../stages/BestPossibleStateCalcStage.java         | 98 ++++++++++++++++------
 .../waged/model/TestClusterModelProvider.java      |  6 +-
 9 files changed, 196 insertions(+), 255 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
new file mode 100644
index 0000000..c01b173
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
@@ -0,0 +1,43 @@
+package org.apache.helix;
+
+/*
+ * 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.
+ */
+
+/**
+ * Exception thrown by Helix due to rebalance failures.
+ */
+public class HelixRebalanceException extends Exception {
+  enum RebalanceFailureType {
+    INVALID_CLUSTER_STATUS,
+    INVALID_REBALANCER_STATUS,
+    FAILED_TO_CALCULATE,
+    UNKNOWN_FAILURE
+  }
+
+  private final RebalanceFailureType _type;
+
+  public HelixRebalanceException(String message, RebalanceFailureType type, Throwable cause) {
+    super(String.format("%s. Failure Type: %s", message, type.name()), cause);
+    _type = type;
+  }
+
+  public RebalanceFailureType getFailureType() {
+    return _type;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
index d65e609..611f4b2 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/changedetector/ResourceChangeDetector.java
@@ -20,15 +20,17 @@ package org.apache.helix.controller.changedetector;
  */
 
 import com.google.common.collect.Sets;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
-import org.apache.helix.HelixConstants;
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixProperty;
-import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 
 /**
  * ResourceChangeDetector implements ChangeDetector. It caches resource-related metadata from
@@ -37,6 +39,7 @@ import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
  * WARNING: the methods of this class are not thread-safe.
  */
 public class ResourceChangeDetector implements ChangeDetector {
+  private static final Logger LOG = LoggerFactory.getLogger(ResourceChangeDetector.class.getName());
 
   private ResourceChangeSnapshot _oldSnapshot; // snapshot for previous pipeline run
   private ResourceChangeSnapshot _newSnapshot; // snapshot for this pipeline run
@@ -108,10 +111,13 @@ public class ResourceChangeDetector implements ChangeDetector {
       return snapshot.getResourceConfigMap();
     case LIVE_INSTANCE:
       return snapshot.getLiveInstances();
+    case CONFIG:
+      return Collections.emptyMap();
     default:
-      throw new HelixException(String.format(
-          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: %s",
-          changeType));
+      LOG.warn(
+          "ResourceChangeDetector cannot compute the names of changes for the given ChangeType: {}",
+          changeType);
+      return Collections.emptyMap();
     }
   }
 
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
deleted file mode 100644
index a3b9b32..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/GlobalRebalancer.java
+++ /dev/null
@@ -1,67 +0,0 @@
-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/ClusterDataDetector.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
deleted file mode 100644
index 0423edf..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataDetector.java
+++ /dev/null
@@ -1,73 +0,0 @@
-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 {
-    BaselineAssignmentChange,
-    InstanceConfigChange,
-    ClusterConfigChange,
-    ResourceConfigChange,
-    ResourceIdealStatesChange,
-    InstanceStateChange,
-    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
deleted file mode 100644
index 387666c..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/ClusterDataProvider.java
+++ /dev/null
@@ -1,54 +0,0 @@
-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.ResourceAssignment;
-
-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, ResourceAssignment> baselineAssignment,
-      Map<String, ResourceAssignment> bestPossibleAssignment) {
-    // TODO finish the implementation.
-    return null;
-  }
-}
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
index aa3cfee..fd740e6 100644
--- 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
@@ -19,10 +19,13 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.changedetector.ResourceChangeDetector;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.GlobalRebalancer;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintsRebalanceAlgorithm;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Resource;
@@ -36,23 +39,57 @@ 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>
+ * @see <a href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
+ * Design Document
+ * </a>
  */
-public class WagedRebalancer implements GlobalRebalancer<ResourceControllerDataProvider> {
+public class WagedRebalancer {
   private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
 
-  @Override
-  public void init(HelixManager manager) { }
+  // --------- The following fields are placeholders and need replacement. -----------//
+  // TODO Shall we make the metadata store a static threadlocal object as well to avoid reinitialization?
+  private final AssignmentMetadataStore _assignmentMetadataStore;
+  private final RebalanceAlgorithm _rebalanceAlgorithm;
+  // ------------------------------------------------------------------------------------//
 
-  @Override
-  public Map<String, IdealState> computeNewIdealState(CurrentStateOutput currentStateOutput,
-      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
-      throws HelixException {
-    return new HashMap<>();
+  // The cluster change detector is a stateful object. Make it static to avoid unnecessary
+  // reinitialization.
+  private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
+      new ThreadLocal<>();
+  private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
+
+  private ResourceChangeDetector getChangeDetector() {
+    if (CHANGE_DETECTOR_THREAD_LOCAL.get() == null) {
+      CHANGE_DETECTOR_THREAD_LOCAL.set(new ResourceChangeDetector());
+    }
+    return CHANGE_DETECTOR_THREAD_LOCAL.get();
+  }
+
+  public WagedRebalancer(HelixManager helixManager) {
+    // TODO init the metadata store according to their requirement when integrate, or change to final static method if possible.
+    _assignmentMetadataStore = new AssignmentMetadataStore();
+    // TODO init the algorithm according to the requirement when integrate.
+    _rebalanceAlgorithm = new ConstraintsRebalanceAlgorithm();
+
+    // Use the mapping calculator in DelayedAutoRebalancer for calculating the final assignment
+    // output.
+    // This calculator will translate the best possible assignment into an applicable state mapping
+    // based on the current states.
+    // TODO abstract and separate the mapping calculator logic from the DelayedAutoRebalancer
+    _mappingCalculator = new DelayedAutoRebalancer();
   }
 
-  @Override
-  public RebalanceFailureReason getFailureReason() {
-    return new RebalanceFailureReason(RebalanceFailureType.UNKNOWN_FAILURE);
+  /**
+   * Compute the new IdealStates for all the resources input. The IdealStates include both the new
+   * partition assignment (in the listFiles) and the new replica state mapping (in the mapFields).
+   * @param clusterData        The Cluster status data provider.
+   * @param resourceMap        A map containing all the rebalancing resources.
+   * @param currentStateOutput The present Current State of the cluster.
+   * @return A map containing the computed new IdealStates.
+   */
+  public Map<String, IdealState> computeNewIdealStates(ResourceControllerDataProvider clusterData,
+      Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
+      throws HelixRebalanceException {
+    return new HashMap<>();
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 9de023b..c4f7d02 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -19,9 +19,9 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -59,7 +59,7 @@ public class ClusterModelProvider {
    */
   public static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
       Map<String, Resource> resourceMap, Set<String> activeInstances,
-      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
       Map<String, ResourceAssignment> baselineAssignment,
       Map<String, ResourceAssignment> bestPossibleAssignment) {
     // Generate replica objects for all the resource partitions.
@@ -108,14 +108,13 @@ public class ClusterModelProvider {
    */
   private static Set<AssignableReplica> findToBeAssignedReplicas(
       Map<String, Set<AssignableReplica>> replicaMap,
-      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
       Map<String, ResourceAssignment> bestPossibleAssignment,
       Map<String, Set<AssignableReplica>> allocatedReplicas) {
     Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
-    if (clusterChanges.containsKey(ClusterDataDetector.ChangeType.ClusterConfigChange)
-        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.InstanceConfigChange)
-        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.BaselineAssignmentChange)) {
-      // If the cluster topology or baseline assignment has been modified, need to reassign all replicas
+    if (clusterChanges.containsKey(HelixConstants.ChangeType.CONFIG)
+        || clusterChanges.containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
+      // If the cluster topology has been modified, need to reassign all replicas
       toBeAssignedReplicas
           .addAll(replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()));
     } else {
@@ -124,11 +123,13 @@ public class ClusterModelProvider {
         Set<AssignableReplica> replicas = replicaMap.get(resourceName);
         // 1. if the resource config/idealstate is changed, need to reassign.
         // 2. if the resource does appear in the best possible assignment, need to reassign.
-        if (clusterChanges.getOrDefault(ClusterDataDetector.ChangeType.ResourceConfigChange,
-            Collections.emptySet()).contains(resourceName) || clusterChanges
-            .getOrDefault(ClusterDataDetector.ChangeType.ResourceIdealStatesChange,
-                Collections.emptySet()).contains(resourceName) || !bestPossibleAssignment
-            .containsKey(resourceName)) {
+        if (clusterChanges
+            .getOrDefault(HelixConstants.ChangeType.RESOURCE_CONFIG, Collections.emptySet())
+            .contains(resourceName)
+            || clusterChanges
+            .getOrDefault(HelixConstants.ChangeType.IDEAL_STATE, Collections.emptySet())
+            .contains(resourceName)
+            || !bestPossibleAssignment.containsKey(resourceName)) {
           toBeAssignedReplicas.addAll(replicas);
           continue; // go to check next resource
         } else {
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 49a72e0..ba4da88 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.stages;
  */
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -27,6 +28,7 @@ import java.util.concurrent.Callable;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
+import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
@@ -37,6 +39,7 @@ import org.apache.helix.controller.rebalancer.MaintenanceRebalancer;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.MaintenanceSignal;
@@ -56,18 +59,19 @@ import org.slf4j.LoggerFactory;
  * IdealState,StateModel,LiveInstance
  */
 public class BestPossibleStateCalcStage extends AbstractBaseStage {
-  private static final Logger logger = LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
+  private static final Logger logger =
+      LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
-    CurrentStateOutput currentStateOutput =
-        event.getAttribute(AttributeName.CURRENT_STATE.name());
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
     final Map<String, Resource> resourceMap =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     final ClusterStatusMonitor clusterStatusMonitor =
         event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
     if (currentStateOutput == null || resourceMap == null || cache == null) {
       throw new StageException(
@@ -90,8 +94,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
                     resourceMap, stateModelDefMap);
           }
         } catch (Exception e) {
-          LogUtil
-              .logError(logger, _eventId, "Could not update cluster status metrics!", e);
+          LogUtil.logError(logger, _eventId, "Could not update cluster status metrics!", e);
         }
         return null;
       }
@@ -100,7 +103,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
   private BestPossibleStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
       CurrentStateOutput currentStateOutput) {
-    ResourceControllerDataProvider cache = event.getAttribute(AttributeName.ControllerDataProvider.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
     BestPossibleStateOutput output = new BestPossibleStateOutput();
 
     HelixManager helixManager = event.getAttribute(AttributeName.helixmanager.name());
@@ -112,19 +116,50 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     boolean isValid = validateOfflineInstancesLimit(cache,
         (HelixManager) event.getAttribute(AttributeName.helixmanager.name()));
 
+    // 1. Rebalance with the WAGED rebalancer
+    // The rebalancer only calculates the new ideal assignment for all the resources that are
+    // configured to use the WAGED rebalancer.
+    // For the other resources, the legacy rebalancers will be triggered in the next step.
+    Map<String, IdealState> newIdealStates = new HashMap<>();
+    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager);
+    try {
+      newIdealStates
+          .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, currentStateOutput));
+    } catch (HelixRebalanceException ex) {
+      // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
+      // Since it calculates for all the eligible resources globally, a partial result is invalid.
+      // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
+      LogUtil.logError(logger, _eventId, String
+          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
+              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
+    }
+
     final List<String> failureResources = new ArrayList<>();
     Iterator<Resource> itr = resourceMap.values().iterator();
     while (itr.hasNext()) {
       Resource resource = itr.next();
       boolean result = false;
-      try {
-        result =
-            computeResourceBestPossibleState(event, cache, currentStateOutput, resource, output);
-      } catch (HelixException ex) {
-        LogUtil.logError(logger, _eventId,
-            "Exception when calculating best possible states for " + resource.getResourceName(),
-            ex);
-
+      IdealState is = newIdealStates.get(resource.getResourceName());
+      if (is != null) {
+        // 2. Check if the WAGED rebalancer has calculated for this resource or not.
+        result = checkBestPossibleStateCalculation(is);
+        if (result) {
+          // The WAGED rebalancer calculates a valid result, record in the output
+          updateBestPossibleStateOutput(output, resource, is);
+        }
+      } else {
+        // 3. The WAGED rebalancer skips calculating the resource assignment, fallback to use a
+        // legacy resource rebalancer if applicable.
+        // If this calculation fails, the resource will be reported in the failureResources list.
+        try {
+          result =
+              computeSingleResourceBestPossibleState(event, cache, currentStateOutput, resource,
+                  output);
+        } catch (HelixException ex) {
+          LogUtil.logError(logger, _eventId,
+              "Exception when calculating best possible states for " + resource.getResourceName(),
+              ex);
+        }
       }
       if (!result) {
         failureResources.add(resource.getResourceName());
@@ -185,8 +220,9 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
         if (manager != null) {
           if (manager.getHelixDataAccessor()
               .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
-            manager.getClusterManagmentTool().autoEnableMaintenanceMode(manager.getClusterName(),
-                true, errMsg, MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
+            manager.getClusterManagmentTool()
+                .autoEnableMaintenanceMode(manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_OFFLINE_INSTANCES_EXCEEDED);
             LogUtil.logWarn(logger, _eventId, errMsg);
           }
         } else {
@@ -199,8 +235,19 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     return true;
   }
 
-  private boolean computeResourceBestPossibleState(ClusterEvent event, ResourceControllerDataProvider cache,
-      CurrentStateOutput currentStateOutput, Resource resource, BestPossibleStateOutput output) {
+  private void updateBestPossibleStateOutput(BestPossibleStateOutput output, Resource resource,
+      IdealState computedIdealState) {
+    output.setPreferenceLists(resource.getResourceName(), computedIdealState.getPreferenceLists());
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> newStateMap =
+          computedIdealState.getInstanceStateMap(partition.getPartitionName());
+      output.setState(resource.getResourceName(), partition, newStateMap);
+    }
+  }
+
+  private boolean computeSingleResourceBestPossibleState(ClusterEvent event,
+      ResourceControllerDataProvider cache, CurrentStateOutput currentStateOutput,
+      Resource resource, BestPossibleStateOutput output) {
     // for each ideal state
     // read the state model def
     // for each resource
@@ -229,12 +276,13 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
     Rebalancer<ResourceControllerDataProvider> rebalancer =
         getRebalancer(idealState, resourceName, cache.isMaintenanceModeEnabled());
-    MappingCalculator<ResourceControllerDataProvider> mappingCalculator = getMappingCalculator(rebalancer, resourceName);
+    MappingCalculator<ResourceControllerDataProvider> mappingCalculator =
+        getMappingCalculator(rebalancer, resourceName);
 
     if (rebalancer == null || mappingCalculator == null) {
-      LogUtil.logError(logger, _eventId,
-          "Error computing assignment for resource " + resourceName + ". no rebalancer found. rebalancer: " + rebalancer
-              + " mappingCalculator: " + mappingCalculator);
+      LogUtil.logError(logger, _eventId, "Error computing assignment for resource " + resourceName
+          + ". no rebalancer found. rebalancer: " + rebalancer + " mappingCalculator: "
+          + mappingCalculator);
     }
 
     if (rebalancer != null && mappingCalculator != null) {
@@ -299,8 +347,8 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     }
   }
 
-  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState, String resourceName,
-      boolean isMaintenanceModeEnabled) {
+  private Rebalancer<ResourceControllerDataProvider> getRebalancer(IdealState idealState,
+      String resourceName, boolean isMaintenanceModeEnabled) {
     Rebalancer<ResourceControllerDataProvider> customizedRebalancer = null;
     String rebalancerClassName = idealState.getRebalancerClassName();
     if (rebalancerClassName != null) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index f92a66c..1221b6f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -19,8 +19,8 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixConstants;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.IdealState;
@@ -177,7 +177,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     // 5. test with best possible assignment but cluster topology is changed
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ClusterConfigChange,
+        _instances, Collections.singletonMap(HelixConstants.ChangeType.CONFIG,
             Collections.emptySet()), Collections.emptyMap(), bestPossibleAssignment);
     // There should be no existing assignment since the topology change invalidates all existing assignment
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
@@ -194,7 +194,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     String changedResourceName = _resourceNames.get(0);
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ResourceConfigChange,
+        _instances, Collections.singletonMap(HelixConstants.ChangeType.RESOURCE_CONFIG,
             Collections.singleton(changedResourceName)), Collections.emptyMap(),
         bestPossibleAssignment);
     // There should be no existing assignment for all the resource except for resource2.


[helix] 18/37: Validate the instance capacity/partition weight configuration while constructing the assignable instances (#451)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a4b61eeeb74cd3363a281214dab11bb53ec013d8
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Fri Sep 6 19:17:19 2019 -0700

    Validate the instance capacity/partition weight configuration while constructing the assignable instances (#451)
    
    Compare the configure items with the required capacity keys that are defined in the cluster config when build the assignable instances.
    - According to the design, all the required capacity keys must appear in the instance capacity config.
    - As for the partition weights, the corresponding weight item will be filled with value 0 if the required capacity key is not specified in the resource config.
---
 .../rebalancer/waged/model/AssignableNode.java     | 41 ++++++++++++----
 .../rebalancer/waged/model/AssignableReplica.java  | 27 ++++++++---
 .../waged/model/ClusterModelProvider.java          | 31 ++++++-------
 .../waged/model/AbstractTestClusterModel.java      | 23 ++++-----
 .../rebalancer/waged/model/TestAssignableNode.java | 53 +++++++++++++--------
 .../waged/model/TestAssignableReplica.java         | 54 +++++++++++++++++-----
 6 files changed, 153 insertions(+), 76 deletions(-)

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
index 35c3c38..33677e5 100644
--- 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
@@ -19,12 +19,6 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.InstanceConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -35,6 +29,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.helix.HelixException;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import static java.lang.Math.max;
 
 /**
@@ -91,10 +91,7 @@ public class AssignableNode {
       Collection<AssignableReplica> existingAssignment) {
     reset();
 
-    Map<String, Integer> instanceCapacity = instanceConfig.getInstanceCapacityMap();
-    if (instanceCapacity.isEmpty()) {
-      instanceCapacity = clusterConfig.getDefaultInstanceCapacityMap();
-    }
+    Map<String, Integer> instanceCapacity = fetchInstanceCapacity(clusterConfig, instanceConfig);
     _currentCapacityMap.putAll(instanceCapacity);
     _faultZone = computeFaultZone(clusterConfig, instanceConfig);
     _instanceTags = new HashSet<>(instanceConfig.getTags());
@@ -213,6 +210,7 @@ public class AssignableNode {
    * The method dynamically returns the highest utilization number among all the capacity categories.
    * For example, if the current node usage is {CPU: 0.9, MEM: 0.4, DISK: 0.6}. Then this call shall
    * return 0.9.
+   *
    * @return The highest utilization number of the node among all the capacity category.
    */
   public float getHighestCapacityUtilization() {
@@ -359,6 +357,29 @@ public class AssignableNode {
     // a NOP; in other words, this node will be treated as if it has unlimited capacity.
   }
 
+  /**
+   * Get and validate the instance capacity from instance config.
+   *
+   * @throws HelixException if any required capacity key is not configured in the instance config.
+   */
+  private Map<String, Integer> fetchInstanceCapacity(ClusterConfig clusterConfig,
+      InstanceConfig instanceConfig) {
+    List<String> requiredCapacityKeys = clusterConfig.getInstanceCapacityKeys();
+    Map<String, Integer> instanceCapacity = instanceConfig.getInstanceCapacityMap();
+    if (instanceCapacity.isEmpty()) {
+      instanceCapacity = clusterConfig.getDefaultInstanceCapacityMap();
+    }
+    // Remove all the non-required capacity items from the map.
+    instanceCapacity.keySet().retainAll(requiredCapacityKeys);
+    // All the required keys must exist in the instance config.
+    if (!instanceCapacity.keySet().containsAll(requiredCapacityKeys)) {
+      throw new HelixException(String.format(
+          "The required capacity keys %s are not fully configured in the instance %s capacity map %s.",
+          requiredCapacityKeys.toString(), _instanceName, instanceCapacity.toString()));
+    }
+    return instanceCapacity;
+  }
+
   @Override
   public int hashCode() {
     return _instanceName.hashCode();
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
index ade04bf..537bf70 100644
--- 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
@@ -19,12 +19,14 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.model.ResourceConfig;
-import org.apache.helix.model.StateModelDefinition;
-
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.model.StateModelDefinition;
+
 /**
  * This class represents a partition replication that needs to be allocated.
  */
@@ -40,18 +42,19 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
   private final String _replicaState;
 
   /**
+   * @param clusterConfig  The cluster config.
    * @param resourceConfig The resource config for the resource which contains the replication.
    * @param partitionName  The replication's partition name.
    * @param replicaState   The state of the replication.
    * @param statePriority  The priority of the replication's state.
    */
-  AssignableReplica(ResourceConfig resourceConfig, String partitionName, String replicaState,
-      int statePriority) {
+  AssignableReplica(ClusterConfig clusterConfig, ResourceConfig resourceConfig,
+      String partitionName, String replicaState, int statePriority) {
     _partitionName = partitionName;
     _replicaState = replicaState;
     _statePriority = statePriority;
     _resourceName = resourceConfig.getResourceName();
-    _capacityUsage = fetchCapacityUsage(partitionName, resourceConfig);
+    _capacityUsage = fetchCapacityUsage(partitionName, resourceConfig, clusterConfig);
     _resourceInstanceGroupTag = resourceConfig.getInstanceGroupTag();
     _resourceMaxPartitionsPerInstance = resourceConfig.getMaxPartitionsPerInstance();
   }
@@ -127,7 +130,7 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
    * Parse the resource config for the partition weight.
    */
   private Map<String, Integer> fetchCapacityUsage(String partitionName,
-      ResourceConfig resourceConfig) {
+      ResourceConfig resourceConfig, ClusterConfig clusterConfig) {
     Map<String, Map<String, Integer>> capacityMap;
     try {
       capacityMap = resourceConfig.getPartitionCapacityMap();
@@ -146,6 +149,16 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
           "The capacity usage of the specified partition %s is not configured in the Resource Config %s. No default partition capacity is configured neither.",
           partitionName, resourceConfig.getResourceName()));
     }
+
+    List<String> requiredCapacityKeys = clusterConfig.getInstanceCapacityKeys();
+    // Remove the non-required capacity items.
+    partitionCapacity.keySet().retainAll(requiredCapacityKeys);
+    // If any required capacity key is not configured in the resource config, fill the partition
+    // capacity map with 0 usage.
+    for (String capacityKey : requiredCapacityKeys) {
+      partitionCapacity.putIfAbsent(capacityKey, 0);
+    }
+
     return partitionCapacity;
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 61f5d8d..3570164 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -19,6 +19,14 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
@@ -30,14 +38,6 @@ import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
-
 /**
  * This util class generates Cluster Model object based on the controller's data cache.
  */
@@ -112,8 +112,8 @@ public class ClusterModelProvider {
       Map<String, ResourceAssignment> bestPossibleAssignment,
       Map<String, Set<AssignableReplica>> allocatedReplicas) {
     Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
-    if (clusterChanges.containsKey(HelixConstants.ChangeType.CLUSTER_CONFIG)
-        || clusterChanges.containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
+    if (clusterChanges.containsKey(HelixConstants.ChangeType.CLUSTER_CONFIG) || clusterChanges
+        .containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
       // If the cluster topology has been modified, need to reassign all replicas
       toBeAssignedReplicas
           .addAll(replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()));
@@ -125,11 +125,9 @@ public class ClusterModelProvider {
         // 2. if the resource does appear in the best possible assignment, need to reassign.
         if (clusterChanges
             .getOrDefault(HelixConstants.ChangeType.RESOURCE_CONFIG, Collections.emptySet())
-            .contains(resourceName)
-            || clusterChanges
+            .contains(resourceName) || clusterChanges
             .getOrDefault(HelixConstants.ChangeType.IDEAL_STATE, Collections.emptySet())
-            .contains(resourceName)
-            || !bestPossibleAssignment.containsKey(resourceName)) {
+            .contains(resourceName) || !bestPossibleAssignment.containsKey(resourceName)) {
           toBeAssignedReplicas.addAll(replicas);
           continue; // go to check next resource
         } else {
@@ -196,9 +194,10 @@ public class ClusterModelProvider {
       ResourceControllerDataProvider dataProvider, Map<String, Resource> resourceMap,
       int instanceCount) {
     Map<String, Set<AssignableReplica>> totalReplicaMap = new HashMap<>();
+    ClusterConfig clusterConfig = dataProvider.getClusterConfig();
 
     for (String resourceName : resourceMap.keySet()) {
-      ResourceConfig config = dataProvider.getResourceConfig(resourceName);
+      ResourceConfig resourceConfig = dataProvider.getResourceConfig(resourceName);
       IdealState is = dataProvider.getIdealState(resourceName);
       if (is == null) {
         throw new HelixException(
@@ -220,7 +219,7 @@ public class ClusterModelProvider {
           String state = entry.getKey();
           for (int i = 0; i < entry.getValue(); i++) {
             totalReplicaMap.computeIfAbsent(resourceName, key -> new HashSet<>()).add(
-                new AssignableReplica(config, partition, state,
+                new AssignableReplica(clusterConfig, resourceConfig, partition, state,
                     def.getStatePriorityMap().get(state)));
           }
         }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
index d99a3fb..a8a5de5 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -19,6 +19,15 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
@@ -29,15 +38,6 @@ import org.apache.helix.model.ResourceConfig;
 import org.mockito.Mockito;
 import org.testng.annotations.BeforeClass;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import static org.mockito.Mockito.when;
 
 public abstract class AbstractTestClusterModel {
@@ -104,6 +104,7 @@ public abstract class AbstractTestClusterModel {
     testClusterConfig.setMaxPartitionsPerInstance(5);
     testClusterConfig.setDisabledInstances(Collections.emptyMap());
     testClusterConfig.setTopologyAwareEnabled(false);
+    testClusterConfig.setInstanceCapacityKeys(new ArrayList<>(_capacityDataMap.keySet()));
     when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
 
     // 3. Mock the live instance node for the default instance.
@@ -179,8 +180,8 @@ public abstract class AbstractTestClusterModel {
       ResourceConfig resourceConfig = dataProvider.getResourceConfig(cs.getResourceName());
       // Construct one AssignableReplica for each partition in the current state.
       cs.getPartitionStateMap().entrySet().stream().forEach(entry -> assignmentSet.add(
-          new AssignableReplica(resourceConfig, entry.getKey(), entry.getValue(),
-              entry.getValue().equals("MASTER") ? 1 : 2)));
+          new AssignableReplica(dataProvider.getClusterConfig(), resourceConfig, entry.getKey(),
+              entry.getValue(), entry.getValue().equals("MASTER") ? 1 : 2)));
     }
     return assignmentSet;
   }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index 34a03a9..92a6998 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -19,22 +19,24 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
-import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.model.ClusterConfig;
-import org.apache.helix.model.InstanceConfig;
-import org.testng.Assert;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
 import static org.mockito.Mockito.when;
 
 public class TestAssignableNode extends AbstractTestClusterModel {
@@ -87,9 +89,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
         expectedTopStateAssignmentSet1.size() + expectedTopStateAssignmentSet2.size());
 
     // Test 2 - release assignment from the AssignableNode
-    AssignableReplica removingReplica =
-        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
-            _partitionNames.get(2), "MASTER", 1);
+    AssignableReplica removingReplica = new AssignableReplica(testCache.getClusterConfig(),
+        testCache.getResourceConfig(_resourceNames.get(1)), _partitionNames.get(2), "MASTER", 1);
     expectedAssignment.get(_resourceNames.get(1)).remove(_partitionNames.get(2));
     expectedCapacityMap.put("item1", 9);
     expectedCapacityMap.put("item2", 18);
@@ -128,9 +129,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
         expectedTopStateAssignmentSet1.size() + expectedTopStateAssignmentSet2.size());
 
     // Test 3 - add assignment to the AssignableNode
-    AssignableReplica addingReplica =
-        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
-            _partitionNames.get(2), "SLAVE", 2);
+    AssignableReplica addingReplica = new AssignableReplica(testCache.getClusterConfig(),
+        testCache.getResourceConfig(_resourceNames.get(1)), _partitionNames.get(2), "SLAVE", 2);
     expectedAssignment.get(_resourceNames.get(1)).add(_partitionNames.get(2));
     expectedCapacityMap.put("item1", 4);
     expectedCapacityMap.put("item2", 8);
@@ -169,9 +169,9 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
         Collections.emptyList());
-    AssignableReplica removingReplica =
-        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
-            _partitionNames.get(2) + "non-exist", "MASTER", 1);
+    AssignableReplica removingReplica = new AssignableReplica(testCache.getClusterConfig(),
+        testCache.getResourceConfig(_resourceNames.get(1)), _partitionNames.get(2) + "non-exist",
+        "MASTER", 1);
 
     // Release shall pass.
     assignableNode.release(removingReplica);
@@ -184,9 +184,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
-    AssignableReplica duplicateReplica =
-        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(0)),
-            _partitionNames.get(0), "SLAVE", 2);
+    AssignableReplica duplicateReplica = new AssignableReplica(testCache.getClusterConfig(),
+        testCache.getResourceConfig(_resourceNames.get(0)), _partitionNames.get(0), "SLAVE", 2);
     assignableNode.assign(duplicateReplica);
   }
 
@@ -264,4 +263,18 @@ public class TestAssignableNode extends AbstractTestClusterModel {
             Collections.emptyList());
     Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
   }
+
+  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "The required capacity keys \\[item2, item1, item3, AdditionalCapacityKey\\] are not fully configured in the instance testInstanceId capacity map \\{item2=40, item1=20, item3=30\\}.")
+  public void testIncompleteInstanceCapacity() {
+    ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
+    List<String> requiredCapacityKeys = new ArrayList<>(_capacityDataMap.keySet());
+    requiredCapacityKeys.add("AdditionalCapacityKey");
+    testClusterConfig.setInstanceCapacityKeys(requiredCapacityKeys);
+
+    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
+    testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
+
+    new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId,
+        Collections.emptyList());
+  }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java
index d069ced..a247537 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java
@@ -19,16 +19,19 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.model.ResourceConfig;
-import org.apache.helix.model.StateModelDefinition;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.model.StateModelDefinition;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
 public class TestAssignableReplica {
   String resourceName = "Resource";
   String partitionNamePrefix = "partition";
@@ -38,7 +41,7 @@ public class TestAssignableReplica {
   int slavePriority = 2;
 
   @Test
-  public void testConstructRepliaWithResourceConfig() throws IOException {
+  public void testConstructReplicaWithResourceConfig() throws IOException {
     // Init assignable replica with a basic config object
     Map<String, Integer> capacityDataMapResource1 = new HashMap<>();
     capacityDataMapResource1.put("item1", 3);
@@ -46,11 +49,13 @@ public class TestAssignableReplica {
     ResourceConfig testResourceConfigResource = new ResourceConfig(resourceName);
     testResourceConfigResource.setPartitionCapacityMap(
         Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMapResource1));
+    ClusterConfig testClusterConfig = new ClusterConfig("testCluster");
+    testClusterConfig.setInstanceCapacityKeys(new ArrayList<>(capacityDataMapResource1.keySet()));
 
     String partitionName = partitionNamePrefix + 1;
     AssignableReplica replica =
-        new AssignableReplica(testResourceConfigResource, partitionName, masterState,
-            masterPriority);
+        new AssignableReplica(testClusterConfig, testResourceConfigResource, partitionName,
+            masterState, masterPriority);
     Assert.assertEquals(replica.getResourceName(), resourceName);
     Assert.assertEquals(replica.getPartitionName(), partitionName);
     Assert.assertEquals(replica.getReplicaState(), masterState);
@@ -79,14 +84,14 @@ public class TestAssignableReplica {
         .setIntField(ResourceConfig.ResourceConfigProperty.MAX_PARTITIONS_PER_INSTANCE.name(),
             maxPartition);
 
-    replica = new AssignableReplica(testResourceConfigResource, partitionName, masterState,
-        masterPriority);
+    replica = new AssignableReplica(testClusterConfig, testResourceConfigResource, partitionName,
+        masterState, masterPriority);
     Assert.assertEquals(replica.getCapacity(), capacityDataMapResource1);
     Assert.assertEquals(replica.getResourceInstanceGroupTag(), group);
     Assert.assertEquals(replica.getResourceMaxPartitionsPerInstance(), maxPartition);
 
-    replica = new AssignableReplica(testResourceConfigResource, partitionName2, slaveState,
-        slavePriority);
+    replica = new AssignableReplica(testClusterConfig, testResourceConfigResource, partitionName2,
+        slaveState, slavePriority);
     Assert.assertEquals(replica.getResourceName(), resourceName);
     Assert.assertEquals(replica.getPartitionName(), partitionName2);
     Assert.assertEquals(replica.getReplicaState(), slaveState);
@@ -96,4 +101,29 @@ public class TestAssignableReplica {
     Assert.assertEquals(replica.getResourceInstanceGroupTag(), group);
     Assert.assertEquals(replica.getResourceMaxPartitionsPerInstance(), maxPartition);
   }
+
+  @Test
+  public void testIncompletePartitionWeightConfig() throws IOException {
+    // Init assignable replica with a basic config object
+    Map<String, Integer> capacityDataMapResource = new HashMap<>();
+    capacityDataMapResource.put("item1", 3);
+    capacityDataMapResource.put("item2", 6);
+    ResourceConfig testResourceConfigResource = new ResourceConfig(resourceName);
+    testResourceConfigResource.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMapResource));
+    ClusterConfig testClusterConfig = new ClusterConfig("testCluster");
+    List<String> requiredCapacityKeys = new ArrayList<>(capacityDataMapResource.keySet());
+    // Remove one required key, so it becomes a unnecessary item.
+    String unnecessaryCapacityKey = requiredCapacityKeys.remove(0);
+    // Add one new required key, so it does not exist in the resource config.
+    String newCapacityKey = "newCapacityKey";
+    requiredCapacityKeys.add(newCapacityKey);
+    testClusterConfig.setInstanceCapacityKeys(requiredCapacityKeys);
+
+    AssignableReplica replica = new AssignableReplica(testClusterConfig, testResourceConfigResource,
+        partitionNamePrefix + 1, masterState, masterPriority);
+    Assert.assertTrue(replica.getCapacity().keySet().containsAll(requiredCapacityKeys));
+    Assert.assertEquals(replica.getCapacity().get(newCapacityKey).intValue(), 0);
+    Assert.assertFalse(replica.getCapacity().containsKey(unnecessaryCapacityKey));
+  }
 }


[helix] 37/37: Add latency metric components for WAGED rebalancer (#490)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 4353f67192adc6c9d6fec1c718f3e1690a60ad0f
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Fri Oct 4 13:50:43 2019 -0700

    Add latency metric components for WAGED rebalancer (#490)
    
    Add WAGED rebalancer metric framework and latency metric implementation
    
    Changelist:
    1. Add WAGED rebalancer metric interface
    2. Implement latency-related metrics
    3. Integrate latency metrics into WAGED rebalancer
    4. Add tests
---
 .../rebalancer/waged/WagedRebalancer.java          | 192 ++++++++++++++-------
 .../ConstraintBasedAlgorithmFactory.java           |   2 +-
 .../stages/BestPossibleStateCalcStage.java         |  44 ++++-
 .../monitoring/mbeans/MonitorDomainNames.java      |   3 +-
 .../helix/monitoring/metrics/MetricCollector.java  | 100 +++++++++++
 .../metrics/WagedRebalancerMetricCollector.java    |  80 +++++++++
 .../implementation/RebalanceLatencyGauge.java      | 104 +++++++++++
 .../model/CountMetric.java}                        |  27 ++-
 .../monitoring/metrics/model/LatencyMetric.java    |  52 ++++++
 .../model/Metric.java}                             |  40 ++++-
 .../rebalancer/waged/TestWagedRebalancer.java      |  22 +--
 .../waged/TestWagedRebalancerMetrics.java          | 132 ++++++++++++++
 .../waged/model/AbstractTestClusterModel.java      |  16 +-
 13 files changed, 698 insertions(+), 116 deletions(-)

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
index d211884..53c9840 100644
--- 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
@@ -48,6 +48,9 @@ import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.ResourceAssignment;
 import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.monitoring.metrics.MetricCollector;
+import org.apache.helix.monitoring.metrics.WagedRebalancerMetricCollector;
+import org.apache.helix.monitoring.metrics.model.LatencyMetric;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,10 +67,8 @@ public class WagedRebalancer {
   // When any of the following change happens, the rebalancer needs to do a global rebalance which
   // contains 1. baseline recalculate, 2. partial rebalance that is based on the new baseline.
   private static final Set<HelixConstants.ChangeType> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES =
-      ImmutableSet.of(
-          HelixConstants.ChangeType.RESOURCE_CONFIG,
-          HelixConstants.ChangeType.CLUSTER_CONFIG,
-          HelixConstants.ChangeType.INSTANCE_CONFIG);
+      ImmutableSet.of(HelixConstants.ChangeType.RESOURCE_CONFIG,
+          HelixConstants.ChangeType.CLUSTER_CONFIG, HelixConstants.ChangeType.INSTANCE_CONFIG);
   // The cluster change detector is a stateful object.
   // Make it static to avoid unnecessary reinitialization.
   private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
@@ -76,6 +77,7 @@ public class WagedRebalancer {
   private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
   private final AssignmentMetadataStore _assignmentMetadataStore;
   private final RebalanceAlgorithm _rebalanceAlgorithm;
+  private MetricCollector _metricCollector;
 
   private static AssignmentMetadataStore constructAssignmentStore(HelixManager helixManager) {
     AssignmentMetadataStore assignmentMetadataStore = null;
@@ -90,7 +92,8 @@ public class WagedRebalancer {
   }
 
   public WagedRebalancer(HelixManager helixManager,
-      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
+      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences,
+      MetricCollector metricCollector) {
     this(constructAssignmentStore(helixManager),
         ConstraintBasedAlgorithmFactory.getInstance(preferences),
         // Use DelayedAutoRebalancer as the mapping calculator for the final assignment output.
@@ -99,16 +102,37 @@ public class WagedRebalancer {
         // TODO abstract and separate the main mapping calculator logic from DelayedAutoRebalancer
         new DelayedAutoRebalancer(),
         // Helix Manager is required for the rebalancer scheduler
-        helixManager);
+        helixManager, metricCollector);
   }
 
+  /**
+   * This constructor will use null for HelixManager and MetricCollector. With null HelixManager,
+   * the rebalancer will rebalance solely based on CurrentStates. With null MetricCollector, the
+   * rebalancer will not emit JMX metrics.
+   * @param assignmentMetadataStore
+   * @param algorithm
+   * @param mappingCalculator
+   */
   protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
       RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator) {
-    this(assignmentMetadataStore, algorithm, mappingCalculator, null);
+    this(assignmentMetadataStore, algorithm, mappingCalculator, null, null);
+  }
+
+  /**
+   * This constructor will use null for HelixManager and MetricCollector. With null HelixManager,
+   * the rebalancer will rebalance solely based on CurrentStates.
+   * @param assignmentMetadataStore
+   * @param algorithm
+   * @param metricCollector
+   */
+  protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
+      RebalanceAlgorithm algorithm, MetricCollector metricCollector) {
+    this(assignmentMetadataStore, algorithm, new DelayedAutoRebalancer(), null, metricCollector);
   }
 
   private WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
-      RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator, HelixManager manager) {
+      RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator, HelixManager manager,
+      MetricCollector metricCollector) {
     if (assignmentMetadataStore == null) {
       LOG.warn("Assignment Metadata Store is not configured properly."
           + " The rebalancer will not access the assignment store during the rebalance.");
@@ -117,6 +141,10 @@ public class WagedRebalancer {
     _rebalanceAlgorithm = algorithm;
     _mappingCalculator = mappingCalculator;
     _manager = manager;
+    // If metricCollector is null, instantiate a version that does not register metrics in order to
+    // allow rebalancer to proceed
+    _metricCollector =
+        metricCollector == null ? new WagedRebalancerMetricCollector() : metricCollector;
   }
 
   // Release all the resources.
@@ -138,8 +166,7 @@ public class WagedRebalancer {
       Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
       throws HelixRebalanceException {
     if (resourceMap.isEmpty()) {
-      LOG.warn("There is no resource to be rebalanced by {}",
-          this.getClass().getSimpleName());
+      LOG.warn("There is no resource to be rebalanced by {}", this.getClass().getSimpleName());
       return Collections.emptyMap();
     }
 
@@ -169,7 +196,6 @@ public class WagedRebalancer {
             newStateMap == null ? Collections.emptyMap() : newStateMap);
       }
     }
-
     LOG.info("Finish computing new ideal states for resources: {}",
         resourceMap.keySet().toString());
     return newIdealStates;
@@ -191,24 +217,25 @@ public class WagedRebalancer {
               return itemKeys;
             }));
 
+    // Perform Global Baseline Calculation
     if (clusterChanges.keySet().stream()
-        .anyMatch(changeType -> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES.contains(changeType))) {
+        .anyMatch(GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES::contains)) {
       refreshBaseline(clusterData, clusterChanges, resourceMap, currentStateOutput);
       // Inject a cluster config change for large scale partial rebalance once the baseline changed.
       clusterChanges.putIfAbsent(HelixConstants.ChangeType.CLUSTER_CONFIG, Collections.emptySet());
     }
 
-    Set<String> activeNodes = DelayedRebalanceUtil
-        .getActiveNodes(clusterData.getAllInstances(), clusterData.getEnabledLiveInstances(),
-            clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
-            clusterData.getInstanceConfigMap(), clusterData.getClusterConfig());
+    Set<String> activeNodes = DelayedRebalanceUtil.getActiveNodes(clusterData.getAllInstances(),
+        clusterData.getEnabledLiveInstances(), clusterData.getInstanceOfflineTimeMap(),
+        clusterData.getLiveInstances().keySet(), clusterData.getInstanceConfigMap(),
+        clusterData.getClusterConfig());
 
     // Schedule (or unschedule) delayed rebalance according to the delayed rebalance config.
     delayedRebalanceSchedule(clusterData, activeNodes, resourceMap.keySet());
 
+    // Perform partial rebalance
     Map<String, ResourceAssignment> newAssignment =
-        partialRebalance(clusterData, clusterChanges, resourceMap, activeNodes,
-            currentStateOutput);
+        partialRebalance(clusterData, clusterChanges, resourceMap, activeNodes, currentStateOutput);
 
     // <ResourceName, <State, Priority>>
     Map<String, Map<String, Integer>> resourceStatePriorityMap = new HashMap<>();
@@ -238,16 +265,15 @@ public class WagedRebalancer {
     // some delayed rebalanced assignments.
     if (!activeNodes.equals(clusterData.getEnabledLiveInstances())) {
       applyRebalanceOverwrite(finalIdealStateMap, clusterData, resourceMap, clusterChanges,
-          resourceStatePriorityMap,
-          getBaselineAssignment(_assignmentMetadataStore, currentStateOutput,
-              resourceMap.keySet()));
+          resourceStatePriorityMap, getBaselineAssignment(_assignmentMetadataStore,
+              currentStateOutput, resourceMap.keySet()));
     }
     // Replace the assignment if user-defined preference list is configured.
     // Note the user-defined list is intentionally applied to the final mapping after calculation.
     // This is to avoid persisting it into the assignment store, which impacts the long term
     // assignment evenness and partition movements.
-    finalIdealStateMap.entrySet().stream().forEach(
-        idealStateEntry -> applyUserDefinedPreferenceList(
+    finalIdealStateMap.entrySet().stream()
+        .forEach(idealStateEntry -> applyUserDefinedPreferenceList(
             clusterData.getResourceConfig(idealStateEntry.getKey()), idealStateEntry.getValue()));
 
     return finalIdealStateMap;
@@ -258,19 +284,31 @@ public class WagedRebalancer {
       Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap,
       final CurrentStateOutput currentStateOutput) throws HelixRebalanceException {
     LOG.info("Start calculating the new baseline.");
+    LatencyMetric globalBaselineCalcLatency = _metricCollector.getMetric(
+        WagedRebalancerMetricCollector.WagedRebalancerMetricNames.GlobalBaselineCalcLatencyGauge
+            .name(),
+        LatencyMetric.class);
+    globalBaselineCalcLatency.startMeasuringLatency();
+    // Read the baseline from metadata store
     Map<String, ResourceAssignment> currentBaseline =
         getBaselineAssignment(_assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
+
     // For baseline calculation
     // 1. Ignore node status (disable/offline).
     // 2. Use the baseline as the previous best possible assignment since there is no "baseline" for
     // the baseline.
-    Map<String, ResourceAssignment> newBaseline =
-        calculateAssignment(clusterData, clusterChanges, resourceMap, clusterData.getAllInstances(),
-            Collections.emptyMap(), currentBaseline);
+    Map<String, ResourceAssignment> newBaseline = calculateAssignment(clusterData, clusterChanges,
+        resourceMap, clusterData.getAllInstances(), Collections.emptyMap(), currentBaseline);
 
+    // Write the new baseline to metadata store
     if (_assignmentMetadataStore != null) {
       try {
+        LatencyMetric writeLatency = _metricCollector.getMetric(
+            WagedRebalancerMetricCollector.WagedRebalancerMetricNames.StateWriteLatencyGauge.name(),
+            LatencyMetric.class);
+        writeLatency.startMeasuringLatency();
         _assignmentMetadataStore.persistBaseline(newBaseline);
+        writeLatency.endMeasuringLatency();
       } catch (Exception ex) {
         throw new HelixRebalanceException("Failed to persist the new baseline assignment.",
             HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
@@ -278,7 +316,7 @@ public class WagedRebalancer {
     } else {
       LOG.debug("Assignment Metadata Store is empty. Skip persist the baseline assignment.");
     }
-
+    globalBaselineCalcLatency.endMeasuringLatency();
     LOG.info("Finish calculating the new baseline.");
   }
 
@@ -288,20 +326,34 @@ public class WagedRebalancer {
       Set<String> activeNodes, final CurrentStateOutput currentStateOutput)
       throws HelixRebalanceException {
     LOG.info("Start calculating the new best possible assignment.");
+    LatencyMetric partialRebalanceLatency = _metricCollector.getMetric(
+        WagedRebalancerMetricCollector.WagedRebalancerMetricNames.PartialRebalanceLatencyGauge
+            .name(),
+        LatencyMetric.class);
+    partialRebalanceLatency.startMeasuringLatency();
+    // TODO: Consider combining the metrics for both baseline/best possible?
+    // Read the baseline from metadata store
     Map<String, ResourceAssignment> currentBaseline =
         getBaselineAssignment(_assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
-    Map<String, ResourceAssignment> currentBestPossibleAssignment =
-        getBestPossibleAssignment(_assignmentMetadataStore, currentStateOutput,
-            resourceMap.keySet());
-    Map<String, ResourceAssignment> newAssignment =
-        calculateAssignment(clusterData, clusterChanges, resourceMap, activeNodes, currentBaseline,
-            currentBestPossibleAssignment);
+
+    // Read the best possible assignment from metadata store
+    Map<String, ResourceAssignment> currentBestPossibleAssignment = getBestPossibleAssignment(
+        _assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
+
+    // Compute the new assignment
+    Map<String, ResourceAssignment> newAssignment = calculateAssignment(clusterData, clusterChanges,
+        resourceMap, activeNodes, currentBaseline, currentBestPossibleAssignment);
 
     if (_assignmentMetadataStore != null) {
       try {
+        LatencyMetric writeLatency = _metricCollector.getMetric(
+            WagedRebalancerMetricCollector.WagedRebalancerMetricNames.StateWriteLatencyGauge.name(),
+            LatencyMetric.class);
+        writeLatency.startMeasuringLatency();
         // TODO Test to confirm if persisting the final assignment (with final partition states)
         // would be a better option.
         _assignmentMetadataStore.persistBestPossibleAssignment(newAssignment);
+        writeLatency.endMeasuringLatency();
       } catch (Exception ex) {
         throw new HelixRebalanceException("Failed to persist the new best possible assignment.",
             HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
@@ -309,20 +361,20 @@ public class WagedRebalancer {
     } else {
       LOG.debug("Assignment Metadata Store is empty. Skip persist the baseline assignment.");
     }
-
+    partialRebalanceLatency.endMeasuringLatency();
     LOG.info("Finish calculating the new best possible assignment.");
     return newAssignment;
   }
 
   /**
    * Generate the cluster model based on the input and calculate the optimal assignment.
-   * @param clusterData                the cluster data cache.
-   * @param clusterChanges             the detected cluster changes.
-   * @param resourceMap                the rebalancing resources.
-   * @param activeNodes                the alive and enabled nodes.
-   * @param baseline                   the baseline assignment for the algorithm as a reference.
+   * @param clusterData the cluster data cache.
+   * @param clusterChanges the detected cluster changes.
+   * @param resourceMap the rebalancing resources.
+   * @param activeNodes the alive and enabled nodes.
+   * @param baseline the baseline assignment for the algorithm as a reference.
    * @param prevBestPossibleAssignment the previous best possible assignment for the algorithm as a
-   *                                   reference.
+   *          reference.
    * @return the new optimal assignment for the resources.
    */
   private Map<String, ResourceAssignment> calculateAssignment(
@@ -415,7 +467,7 @@ public class WagedRebalancer {
    * @param currentStateOutput
    * @param resources
    * @return The current baseline assignment. If record does not exist in the
-   * assignmentMetadataStore, return the current state assignment.
+   *         assignmentMetadataStore, return the current state assignment.
    * @throws HelixRebalanceException
    */
   private Map<String, ResourceAssignment> getBaselineAssignment(
@@ -424,7 +476,12 @@ public class WagedRebalancer {
     Map<String, ResourceAssignment> currentBaseline = Collections.emptyMap();
     if (assignmentMetadataStore != null) {
       try {
+        LatencyMetric stateReadLatency = _metricCollector.getMetric(
+            WagedRebalancerMetricCollector.WagedRebalancerMetricNames.StateReadLatencyGauge.name(),
+            LatencyMetric.class);
+        stateReadLatency.startMeasuringLatency();
         currentBaseline = assignmentMetadataStore.getBaseline();
+        stateReadLatency.endMeasuringLatency();
       } catch (HelixException ex) {
         // Report error. and use empty mapping instead.
         LOG.error("Failed to get the current baseline assignment.", ex);
@@ -435,8 +492,7 @@ public class WagedRebalancer {
       }
     }
     if (currentBaseline.isEmpty()) {
-      LOG.warn(
-          "The current baseline assignment record is empty. Use the current states instead.");
+      LOG.warn("The current baseline assignment record is empty. Use the current states instead.");
       currentBaseline = getCurrentStateAssingment(currentStateOutput, resources);
     }
     return currentBaseline;
@@ -447,7 +503,7 @@ public class WagedRebalancer {
    * @param currentStateOutput
    * @param resources
    * @return The current best possible assignment. If record does not exist in the
-   * assignmentMetadataStore, return the current state assignment.
+   *         assignmentMetadataStore, return the current state assignment.
    * @throws HelixRebalanceException
    */
   private Map<String, ResourceAssignment> getBestPossibleAssignment(
@@ -456,7 +512,12 @@ public class WagedRebalancer {
     Map<String, ResourceAssignment> currentBestAssignment = Collections.emptyMap();
     if (assignmentMetadataStore != null) {
       try {
+        LatencyMetric stateReadLatency = _metricCollector.getMetric(
+            WagedRebalancerMetricCollector.WagedRebalancerMetricNames.StateReadLatencyGauge.name(),
+            LatencyMetric.class);
+        stateReadLatency.startMeasuringLatency();
         currentBestAssignment = assignmentMetadataStore.getBestPossibleAssignment();
+        stateReadLatency.endMeasuringLatency();
       } catch (HelixException ex) {
         // Report error. and use empty mapping instead.
         LOG.error("Failed to get the current best possible assignment.", ex);
@@ -483,8 +544,8 @@ public class WagedRebalancer {
       if (!currentStateMap.isEmpty()) {
         ResourceAssignment newResourceAssignment = new ResourceAssignment(resourceName);
         currentStateMap.entrySet().stream().forEach(currentStateEntry -> {
-          newResourceAssignment
-              .addReplicaMap(currentStateEntry.getKey(), currentStateEntry.getValue());
+          newResourceAssignment.addReplicaMap(currentStateEntry.getKey(),
+              currentStateEntry.getValue());
         });
         currentStateAssignment.put(resourceName, newResourceAssignment);
       }
@@ -507,11 +568,10 @@ public class WagedRebalancer {
       Set<String> offlineOrDisabledInstances = new HashSet<>(delayedActiveNodes);
       offlineOrDisabledInstances.removeAll(clusterData.getEnabledLiveInstances());
       for (String resource : resourceSet) {
-        DelayedRebalanceUtil
-            .setRebalanceScheduler(resource, delayedRebalanceEnabled, offlineOrDisabledInstances,
-                clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
-                clusterData.getInstanceConfigMap(), clusterConfig.getRebalanceDelayTime(),
-                clusterConfig, _manager);
+        DelayedRebalanceUtil.setRebalanceScheduler(resource, delayedRebalanceEnabled,
+            offlineOrDisabledInstances, clusterData.getInstanceOfflineTimeMap(),
+            clusterData.getLiveInstances().keySet(), clusterData.getInstanceConfigMap(),
+            clusterConfig.getRebalanceDelayTime(), clusterConfig, _manager);
       }
     } else {
       LOG.warn("Skip scheduling a delayed rebalancer since HelixManager is not specified.");
@@ -523,30 +583,30 @@ public class WagedRebalancer {
    * Since the rebalancing might be done with the delayed logic, the rebalanced ideal states
    * might include inactive nodes.
    * This overwrite will adjust the final mapping, so as to ensure the result is completely valid.
-   * @param idealStateMap            the calculated ideal states.
-   * @param clusterData              the cluster data cache.
-   * @param resourceMap              the rebalanaced resource map.
-   * @param clusterChanges           the detected cluster changes that triggeres the rebalance.
+   * @param idealStateMap the calculated ideal states.
+   * @param clusterData the cluster data cache.
+   * @param resourceMap the rebalanaced resource map.
+   * @param clusterChanges the detected cluster changes that triggeres the rebalance.
    * @param resourceStatePriorityMap the state priority map for each resource.
-   * @param baseline                 the baseline assignment
+   * @param baseline the baseline assignment
    */
   private void applyRebalanceOverwrite(Map<String, IdealState> idealStateMap,
       ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap,
       Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
       Map<String, Map<String, Integer>> resourceStatePriorityMap,
-      Map<String, ResourceAssignment> baseline)
-      throws HelixRebalanceException {
+      Map<String, ResourceAssignment> baseline) throws HelixRebalanceException {
     Set<String> enabledLiveInstances = clusterData.getEnabledLiveInstances();
-    // Note that the calculation used the baseline as the input only. This is for minimizing unnecessary partition movement.
-    Map<String, ResourceAssignment> activeAssignment =
-        calculateAssignment(clusterData, clusterChanges, resourceMap, enabledLiveInstances,
-            Collections.emptyMap(), baseline);
+    // Note that the calculation used the baseline as the input only. This is for minimizing
+    // unnecessary partition movement.
+    Map<String, ResourceAssignment> activeAssignment = calculateAssignment(clusterData,
+        clusterChanges, resourceMap, enabledLiveInstances, Collections.emptyMap(), baseline);
     for (String resourceName : idealStateMap.keySet()) {
       IdealState is = idealStateMap.get(resourceName);
       if (!activeAssignment.containsKey(resourceName)) {
         throw new HelixRebalanceException(
             "Failed to calculate the complete partition assignment with all active nodes. Cannot find the resource assignment for "
-                + resourceName, HelixRebalanceException.Type.FAILED_TO_CALCULATE);
+                + resourceName,
+            HelixRebalanceException.Type.FAILED_TO_CALCULATE);
       }
       IdealState currentIdealState = clusterData.getIdealState(resourceName);
       IdealState newActiveIdealState =
@@ -555,9 +615,9 @@ public class WagedRebalancer {
 
       int numReplia = currentIdealState.getReplicaCount(enabledLiveInstances.size());
       int minActiveReplica = DelayedRebalanceUtil.getMinActiveReplica(currentIdealState, numReplia);
-      Map<String, List<String>> finalPreferenceLists = DelayedRebalanceUtil
-          .getFinalDelayedMapping(newActiveIdealState.getPreferenceLists(), is.getPreferenceLists(),
-              enabledLiveInstances, Math.min(minActiveReplica, numReplia));
+      Map<String, List<String>> finalPreferenceLists =
+          DelayedRebalanceUtil.getFinalDelayedMapping(newActiveIdealState.getPreferenceLists(),
+              is.getPreferenceLists(), enabledLiveInstances, Math.min(minActiveReplica, numReplia));
 
       is.setPreferenceLists(finalPreferenceLists);
     }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
index f70de9a..fbf8b19 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
@@ -41,7 +41,7 @@ public class ConstraintBasedAlgorithmFactory {
   // enlarge the overall weight of the evenness constraints compared with the movement constraint.
   // TODO: Tune or make the following factor configurable.
   private static final int EVENNESS_PREFERENCE_NORMALIZE_FACTOR = 50;
-  private static final Map<String, Float> MODEL = new HashMap<>() {
+  private static final Map<String, Float> MODEL = new HashMap<String, Float>() {
     {
       // The default setting
       put(PartitionMovementConstraint.class.getSimpleName(), 1f);
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 8c082f1..6f442ea 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -26,8 +26,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
-
 import java.util.stream.Collectors;
+
+import javax.management.JMException;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
@@ -42,6 +43,8 @@ import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
+import org.apache.helix.monitoring.metrics.MetricCollector;
+import org.apache.helix.monitoring.metrics.WagedRebalancerMetricCollector;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
@@ -64,6 +67,8 @@ import org.slf4j.LoggerFactory;
 public class BestPossibleStateCalcStage extends AbstractBaseStage {
   private static final Logger logger =
       LoggerFactory.getLogger(BestPossibleStateCalcStage.class.getName());
+  // Create a ThreadLocal of MetricCollector. Metrics could only be updated by the controller thread only.
+  private static final ThreadLocal<MetricCollector> METRIC_COLLECTOR_THREAD_LOCAL = new ThreadLocal<>();
 
   @Override
   public void process(ClusterEvent event) throws Exception {
@@ -253,20 +258,41 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     Map<String, IdealState> newIdealStates = new HashMap<>();
 
     // Init rebalancer with the rebalance preferences.
-    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences = cache.getClusterConfig()
-        .getGlobalRebalancePreference();
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences =
+        cache.getClusterConfig().getGlobalRebalancePreference();
+
+    if (METRIC_COLLECTOR_THREAD_LOCAL.get() == null) {
+      try {
+        // If HelixManager is null, we just pass in null for MetricCollector so that a
+        // non-functioning WagedRebalancerMetricCollector would be created in WagedRebalancer's
+        // constructor. This is to handle two cases: 1. HelixManager is null for non-testing cases -
+        // in this case, WagedRebalancer will not read/write to metadata store and just use
+        // CurrentState-based rebalancing. 2. Tests that require instrumenting the rebalancer for
+        // verifying whether the cluster has converged.
+        METRIC_COLLECTOR_THREAD_LOCAL.set(helixManager == null ? null
+            : new WagedRebalancerMetricCollector(helixManager.getClusterName()));
+      } catch (JMException e) {
+        LogUtil.logWarn(logger, _eventId, String.format(
+            "MetricCollector instantiation failed! WagedRebalancer will not emit metrics due to JMException %s",
+            e));
+      }
+    }
+
     // TODO avoid creating the rebalancer on every rebalance call for performance enhancement
-    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager, preferences);
+    WagedRebalancer wagedRebalancer =
+        new WagedRebalancer(helixManager, preferences, METRIC_COLLECTOR_THREAD_LOCAL.get());
     try {
-      newIdealStates.putAll(wagedRebalancer
-          .computeNewIdealStates(cache, wagedRebalancedResourceMap, currentStateOutput));
+      newIdealStates.putAll(wagedRebalancer.computeNewIdealStates(cache, wagedRebalancedResourceMap,
+          currentStateOutput));
     } catch (HelixRebalanceException ex) {
       // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
       // Since it calculates for all the eligible resources globally, a partial result is invalid.
       // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
-      LogUtil.logError(logger, _eventId, String
-          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
-              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
+      LogUtil.logError(logger, _eventId,
+          String.format(
+              "Failed to calculate the new Ideal States using the rebalancer %s due to %s",
+              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()),
+          ex);
     } finally {
       wagedRebalancer.close();
     }
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
index 73bf057..fee9099 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
@@ -28,5 +28,6 @@ public enum MonitorDomainNames {
   HelixThreadPoolExecutor,
   HelixCallback,
   RoutingTableProvider,
-  CLMParticipantReport
+  CLMParticipantReport,
+  Rebalancer
 }
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/metrics/MetricCollector.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/MetricCollector.java
new file mode 100644
index 0000000..764557a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/MetricCollector.java
@@ -0,0 +1,100 @@
+package org.apache.helix.monitoring.metrics;
+
+/*
+ * 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.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import javax.management.JMException;
+import javax.management.ObjectName;
+import org.apache.helix.HelixException;
+import org.apache.helix.monitoring.metrics.model.Metric;
+import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMBeanProvider;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+
+/**
+ * Collects and manages all metrics that implement the {@link Metric} interface.
+ */
+public abstract class MetricCollector extends DynamicMBeanProvider {
+  private static final String CLUSTER_NAME_KEY = "ClusterName";
+  private static final String ENTITY_NAME_KEY = "EntityName";
+  private final String _monitorDomainName;
+  private final String _clusterName;
+  private final String _entityName;
+  private Map<String, Metric> _metricMap;
+
+  public MetricCollector(String monitorDomainName, String clusterName, String entityName) {
+    _monitorDomainName = monitorDomainName;
+    _clusterName = clusterName;
+    _entityName = entityName;
+    _metricMap = new HashMap<>();
+  }
+
+  @Override
+  public DynamicMBeanProvider register() throws JMException {
+    // First cast all Metric objects to DynamicMetrics
+    Collection<DynamicMetric<?, ?>> dynamicMetrics = new HashSet<>();
+    _metricMap.values().forEach(metric -> dynamicMetrics.add(metric.getDynamicMetric()));
+
+    // Define MBeanName and ObjectName
+    // MBean name has two key-value pairs:
+    // ------ 1) ClusterName KV pair (first %s=%s)
+    // ------ 2) EntityName KV pair (second %s=%s)
+    String mbeanName =
+        String.format("%s=%s, %s=%s", CLUSTER_NAME_KEY, _clusterName, ENTITY_NAME_KEY, _entityName);
+
+    // ObjectName has one key-value pair:
+    // ------ 1) Monitor domain name KV pair where value is the MBean name
+    doRegister(dynamicMetrics,
+        new ObjectName(String.format("%s:%s", _monitorDomainName, mbeanName)));
+    return this;
+  }
+
+  @Override
+  public String getSensorName() {
+    return String.format("%s.%s.%s", MonitorDomainNames.Rebalancer.name(), _clusterName,
+        _entityName);
+  }
+
+  void addMetric(Metric metric) {
+    if (metric instanceof DynamicMetric) {
+      _metricMap.putIfAbsent(metric.getMetricName(), metric);
+    } else {
+      throw new HelixException("MetricCollector only supports Metrics that are DynamicMetric!");
+    }
+  }
+
+  /**
+   * Returns a desired type of the metric.
+   * @param metricName
+   * @param metricClass Desired type
+   * @param <T> Casted result of the metric
+   * @return
+   */
+  public <T extends DynamicMetric> T getMetric(String metricName, Class<T> metricClass) {
+    return metricClass.cast(_metricMap.get(metricName));
+  }
+
+  public Map<String, Metric> getMetricMap() {
+    return _metricMap;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/metrics/WagedRebalancerMetricCollector.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/WagedRebalancerMetricCollector.java
new file mode 100644
index 0000000..04d804d
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/WagedRebalancerMetricCollector.java
@@ -0,0 +1,80 @@
+package org.apache.helix.monitoring.metrics;
+
+/*
+ * 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 javax.management.JMException;
+import org.apache.helix.monitoring.mbeans.MonitorDomainNames;
+import org.apache.helix.monitoring.metrics.implementation.RebalanceLatencyGauge;
+import org.apache.helix.monitoring.metrics.model.LatencyMetric;
+
+public class WagedRebalancerMetricCollector extends MetricCollector {
+  private static final String WAGED_REBALANCER_ENTITY_NAME = "WagedRebalancer";
+
+  /**
+   * This enum class contains all metric names defined for WagedRebalancer. Note that all enums are
+   * in camel case for readability.
+   */
+  public enum WagedRebalancerMetricNames {
+    // Per-stage latency metrics
+    GlobalBaselineCalcLatencyGauge,
+    PartialRebalanceLatencyGauge,
+
+    // The following latency metrics are related to AssignmentMetadataStore
+    StateReadLatencyGauge,
+    StateWriteLatencyGauge
+  }
+
+  public WagedRebalancerMetricCollector(String clusterName) throws JMException {
+    super(MonitorDomainNames.Rebalancer.name(), clusterName, WAGED_REBALANCER_ENTITY_NAME);
+    createMetrics();
+    register();
+  }
+
+  /**
+   * This constructor will create but will not register metrics. This constructor will be used in
+   * case of JMException so that the rebalancer could proceed without registering and emitting
+   * metrics.
+   */
+  public WagedRebalancerMetricCollector() {
+    super(MonitorDomainNames.Rebalancer.name(), null, null);
+    createMetrics();
+  }
+
+  /**
+   * Creates and registers all metrics in MetricCollector for WagedRebalancer.
+   */
+  private void createMetrics() {
+    // Define all metrics
+    LatencyMetric globalBaselineCalcLatencyGauge = new RebalanceLatencyGauge(
+        WagedRebalancerMetricNames.GlobalBaselineCalcLatencyGauge.name(), getResetIntervalInMs());
+    LatencyMetric partialRebalanceLatencyGauge = new RebalanceLatencyGauge(
+        WagedRebalancerMetricNames.PartialRebalanceLatencyGauge.name(), getResetIntervalInMs());
+    LatencyMetric stateReadLatencyGauge = new RebalanceLatencyGauge(
+        WagedRebalancerMetricNames.StateReadLatencyGauge.name(), getResetIntervalInMs());
+    LatencyMetric stateWriteLatencyGauge = new RebalanceLatencyGauge(
+        WagedRebalancerMetricNames.StateWriteLatencyGauge.name(), getResetIntervalInMs());
+
+    // Add metrics to WagedRebalancerMetricCollector
+    addMetric(globalBaselineCalcLatencyGauge);
+    addMetric(partialRebalanceLatencyGauge);
+    addMetric(stateReadLatencyGauge);
+    addMetric(stateWriteLatencyGauge);
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/metrics/implementation/RebalanceLatencyGauge.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/implementation/RebalanceLatencyGauge.java
new file mode 100644
index 0000000..e96a589
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/implementation/RebalanceLatencyGauge.java
@@ -0,0 +1,104 @@
+package org.apache.helix.monitoring.metrics.implementation;
+
+/*
+ * 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 com.codahale.metrics.Histogram;
+import com.codahale.metrics.SlidingTimeWindowArrayReservoir;
+import java.util.concurrent.TimeUnit;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+import org.apache.helix.monitoring.metrics.model.LatencyMetric;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RebalanceLatencyGauge extends LatencyMetric {
+  private static final Logger LOG = LoggerFactory.getLogger(RebalanceLatencyGauge.class);
+  private static final long VALUE_NOT_SET = -1;
+  private long _lastEmittedMetricValue = VALUE_NOT_SET;
+
+  /**
+   * Instantiates a new Histogram dynamic metric.
+   * @param metricName the metric name
+   */
+  public RebalanceLatencyGauge(String metricName, long slidingTimeWindow) {
+    super(metricName, new Histogram(
+        new SlidingTimeWindowArrayReservoir(slidingTimeWindow, TimeUnit.MILLISECONDS)));
+    _metricName = metricName;
+  }
+
+  /**
+   * WARNING: this method is not thread-safe.
+   * Calling this method multiple times would simply overwrite the previous state. This is because
+   * the rebalancer could fail at any point, and we want it to recover gracefully by resetting the
+   * internal state of this metric.
+   */
+  @Override
+  public void startMeasuringLatency() {
+    reset();
+    _startTime = System.currentTimeMillis();
+  }
+
+  /**
+   * WARNING: this method is not thread-safe.
+   */
+  @Override
+  public void endMeasuringLatency() {
+    if (_startTime == VALUE_NOT_SET || _endTime != VALUE_NOT_SET) {
+      LOG.error(
+          "Needs to call startMeasuringLatency first! Ignoring and resetting the metric. Metric name: {}",
+          _metricName);
+      reset();
+      return;
+    }
+    _endTime = System.currentTimeMillis();
+    _lastEmittedMetricValue = _endTime - _startTime;
+    updateValue(_lastEmittedMetricValue);
+    reset();
+  }
+
+  @Override
+  public String getMetricName() {
+    return _metricName;
+  }
+
+  @Override
+  public void reset() {
+    _startTime = VALUE_NOT_SET;
+    _endTime = VALUE_NOT_SET;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Metric %s's latency is %d", _metricName, getLastEmittedMetricValue());
+  }
+
+  /**
+   * Returns the most recently emitted metric value at the time of the call.
+   * @return
+   */
+  @Override
+  public long getLastEmittedMetricValue() {
+    return _lastEmittedMetricValue;
+  }
+
+  @Override
+  public DynamicMetric getDynamicMetric() {
+    return this;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/CountMetric.java
similarity index 55%
copy from helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
copy to helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/CountMetric.java
index 73bf057..5a7f0ca 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/CountMetric.java
@@ -1,4 +1,4 @@
-package org.apache.helix.monitoring.mbeans;
+package org.apache.helix.monitoring.metrics.model;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -19,14 +19,23 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.SimpleDynamicMetric;
+
 /**
- * This enum defines all of domain names used with various Helix monitor mbeans.
+ * Represents a count metric and defines methods to help with calculation. A count metric gives a
+ * gauge value of a certain property.
  */
-public enum MonitorDomainNames {
-  ClusterStatus,
-  HelixZkClient,
-  HelixThreadPoolExecutor,
-  HelixCallback,
-  RoutingTableProvider,
-  CLMParticipantReport
+public abstract class CountMetric<V> extends SimpleDynamicMetric<V> implements Metric {
+  protected V _count;
+
+  /**
+   * Instantiates a new Simple dynamic metric.
+   * @param metricName the metric name
+   * @param metricObject the metric object
+   */
+  public CountMetric(String metricName, V metricObject) {
+    super(metricName, metricObject);
+  }
+
+  public abstract void setCount(Object count);
 }
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/LatencyMetric.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/LatencyMetric.java
new file mode 100644
index 0000000..c8ba5ae
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/LatencyMetric.java
@@ -0,0 +1,52 @@
+package org.apache.helix.monitoring.metrics.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.
+ */
+
+import com.codahale.metrics.Histogram;
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.HistogramDynamicMetric;
+
+/**
+ * Represents a latency metric and defines methods to help with calculation. A latency metric gives
+ * how long a particular stage in the logic took in milliseconds.
+ */
+public abstract class LatencyMetric extends HistogramDynamicMetric implements Metric {
+  protected long _startTime;
+  protected long _endTime;
+  protected String _metricName;
+
+  /**
+   * Instantiates a new Histogram dynamic metric.
+   * @param metricName the metric name
+   * @param metricObject the metric object
+   */
+  public LatencyMetric(String metricName, Histogram metricObject) {
+    super(metricName, metricObject);
+  }
+
+  /**
+   * Starts measuring the latency.
+   */
+  public abstract void startMeasuringLatency();
+
+  /**
+   * Ends measuring the latency.
+   */
+  public abstract void endMeasuringLatency();
+}
diff --git a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/Metric.java
similarity index 55%
copy from helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
copy to helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/Metric.java
index 73bf057..ba59b4f 100644
--- a/helix-core/src/main/java/org/apache/helix/monitoring/mbeans/MonitorDomainNames.java
+++ b/helix-core/src/main/java/org/apache/helix/monitoring/metrics/model/Metric.java
@@ -1,4 +1,4 @@
-package org.apache.helix.monitoring.mbeans;
+package org.apache.helix.monitoring.metrics.model;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -19,14 +19,36 @@ package org.apache.helix.monitoring.mbeans;
  * under the License.
  */
 
+import org.apache.helix.monitoring.mbeans.dynamicMBeans.DynamicMetric;
+
 /**
- * This enum defines all of domain names used with various Helix monitor mbeans.
+ * Defines a generic metric interface.
  */
-public enum MonitorDomainNames {
-  ClusterStatus,
-  HelixZkClient,
-  HelixThreadPoolExecutor,
-  HelixCallback,
-  RoutingTableProvider,
-  CLMParticipantReport
+public interface Metric {
+
+  /**
+   * Gets the name of the metric.
+   */
+  String getMetricName();
+
+  /**
+   * Resets the internal state of this metric.
+   */
+  void reset();
+
+  /**
+   * Prints the metric along with its name.
+   */
+  String toString();
+
+  /**
+   * Returns the most recently emitted value for the metric at the time of the call.
+   * @return metric value
+   */
+  long getLastEmittedMetricValue();
+
+  /**
+   * Returns the underlying DynamicMetric.
+   */
+  DynamicMetric getDynamicMetric();
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
index 96b6523..df368cb 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
@@ -133,8 +133,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   }
 
   @Test(dependsOnMethods = "testRebalance")
-  public void testPartialRebalance()
-      throws IOException, HelixRebalanceException {
+  public void testPartialRebalance() throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
     WagedRebalancer rebalancer =
         new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
@@ -184,9 +183,9 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         String resourceName = csEntry.getKey();
         CurrentState cs = csEntry.getValue();
         for (Map.Entry<String, String> partitionStateEntry : cs.getPartitionStateMap().entrySet()) {
-          currentStateOutput
-              .setCurrentState(resourceName, new Partition(partitionStateEntry.getKey()),
-                  instanceName, partitionStateEntry.getValue());
+          currentStateOutput.setCurrentState(resourceName,
+              new Partition(partitionStateEntry.getKey()), instanceName,
+              partitionStateEntry.getValue());
         }
       }
     }
@@ -220,8 +219,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   }
 
   @Test(dependsOnMethods = "testRebalance", expectedExceptions = HelixRebalanceException.class, expectedExceptionsMessageRegExp = "Input contains invalid resource\\(s\\) that cannot be rebalanced by the WAGED rebalancer. \\[Resource1\\] Failure Type: INVALID_INPUT")
-  public void testNonCompatibleConfiguration()
-      throws IOException, HelixRebalanceException {
+  public void testNonCompatibleConfiguration() throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
     WagedRebalancer rebalancer =
         new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
@@ -243,8 +241,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   // TODO test with invalid capacity configuration which will fail the cluster model constructing.
   @Test(dependsOnMethods = "testRebalance")
-  public void testInvalidClusterStatus()
-      throws IOException {
+  public void testInvalidClusterStatus() throws IOException {
     _metadataStore.clearMetadataStore();
     WagedRebalancer rebalancer =
         new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
@@ -291,7 +288,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   }
 
   @Test(dependsOnMethods = "testRebalance")
-  public void testAlgorithmExepction() throws IOException, HelixRebalanceException {
+  public void testAlgorithmException() throws IOException, HelixRebalanceException {
     RebalanceAlgorithm badAlgorithm = Mockito.mock(RebalanceAlgorithm.class);
     when(badAlgorithm.calculate(any())).thenThrow(new HelixRebalanceException("Algorithm fails.",
         HelixRebalanceException.Type.FAILED_TO_CALCULATE));
@@ -430,9 +427,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
       Assert.assertTrue(newIdealStates.containsKey(resourceName));
       IdealState is = newIdealStates.get(resourceName);
       ResourceAssignment assignment = expectedResult.get(resourceName);
-      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(
-          assignment.getMappedPartitions().stream().map(partition -> partition.getPartitionName())
-              .collect(Collectors.toSet())));
+      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(assignment.getMappedPartitions()
+          .stream().map(partition -> partition.getPartitionName()).collect(Collectors.toSet())));
       for (String partitionName : is.getPartitionSet()) {
         Assert.assertEquals(is.getInstanceStateMap(partitionName),
             assignment.getReplicaMap(new Partition(partitionName)));
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancerMetrics.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancerMetrics.java
new file mode 100644
index 0000000..dc0c89e
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancerMetrics.java
@@ -0,0 +1,132 @@
+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 java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.management.JMException;
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.constraints.MockRebalanceAlgorithm;
+import org.apache.helix.controller.rebalancer.waged.model.AbstractTestClusterModel;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Resource;
+import org.apache.helix.monitoring.metrics.MetricCollector;
+import org.apache.helix.monitoring.metrics.WagedRebalancerMetricCollector;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+public class TestWagedRebalancerMetrics extends AbstractTestClusterModel {
+  private static final String TEST_STRING = "TEST";
+  private MetricCollector _metricCollector;
+  private Set<String> _instances;
+  private MockRebalanceAlgorithm _algorithm;
+  private MockAssignmentMetadataStore _metadataStore;
+
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+    _instances = new HashSet<>();
+    _instances.add(_testInstanceId);
+    _algorithm = new MockRebalanceAlgorithm();
+
+    // Initialize a mock assignment metadata store
+    _metadataStore = new MockAssignmentMetadataStore();
+  }
+
+  @Test
+  public void testMetricValuePropagation()
+      throws JMException, HelixRebalanceException, IOException {
+    _metadataStore.clearMetadataStore();
+    _metricCollector = new WagedRebalancerMetricCollector(TEST_STRING);
+    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm, _metricCollector);
+
+    // Generate the input for the rebalancer.
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+
+    // Check that there exists a non-zero value in the metrics
+    Assert.assertTrue(_metricCollector.getMetricMap().values().stream()
+        .anyMatch(metric -> metric.getLastEmittedMetricValue() > 0L));
+  }
+
+  @Override
+  protected ResourceControllerDataProvider setupClusterDataCache() throws IOException {
+    ResourceControllerDataProvider testCache = super.setupClusterDataCache();
+
+    // Set up mock idealstate
+    Map<String, IdealState> isMap = new HashMap<>();
+    for (String resource : _resourceNames) {
+      IdealState is = new IdealState(resource);
+      is.setNumPartitions(_partitionNames.size());
+      is.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO);
+      is.setStateModelDefRef("MasterSlave");
+      is.setReplicas("100");
+      is.setRebalancerClassName(WagedRebalancer.class.getName());
+      _partitionNames.stream()
+          .forEach(partition -> is.setPreferenceList(partition, Collections.emptyList()));
+      isMap.put(resource, is);
+    }
+    when(testCache.getIdealState(anyString())).thenAnswer(
+        (Answer<IdealState>) invocationOnMock -> isMap.get(invocationOnMock.getArguments()[0]));
+    when(testCache.getIdealStates()).thenReturn(isMap);
+
+    // Set up 2 more instances
+    for (int i = 1; i < 3; i++) {
+      String instanceName = _testInstanceId + i;
+      _instances.add(instanceName);
+      // 1. Set up the default instance information with capacity configuration.
+      InstanceConfig testInstanceConfig = createMockInstanceConfig(instanceName);
+      Map<String, InstanceConfig> instanceConfigMap = testCache.getInstanceConfigMap();
+      instanceConfigMap.put(instanceName, testInstanceConfig);
+      when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+      // 2. Mock the live instance node for the default instance.
+      LiveInstance testLiveInstance = createMockLiveInstance(instanceName);
+      Map<String, LiveInstance> liveInstanceMap = testCache.getLiveInstances();
+      liveInstanceMap.put(instanceName, testLiveInstance);
+      when(testCache.getLiveInstances()).thenReturn(liveInstanceMap);
+      when(testCache.getEnabledInstances()).thenReturn(liveInstanceMap.keySet());
+      when(testCache.getEnabledLiveInstances()).thenReturn(liveInstanceMap.keySet());
+    }
+
+    return testCache;
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
index 91db076..7cb1da2 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -27,7 +27,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
@@ -116,11 +115,11 @@ public abstract class AbstractTestClusterModel {
     // 4. Mock two resources, each with 2 partitions on the default instance.
     // The instance will have the following partitions assigned
     // Resource 1:
-    //          partition 1 - MASTER
-    //          partition 2 - SLAVE
+    // -------------- partition 1 - MASTER
+    // -------------- partition 2 - SLAVE
     // Resource 2:
-    //          partition 3 - MASTER
-    //          partition 4 - SLAVE
+    // -------------- partition 3 - MASTER
+    // -------------- partition 4 - SLAVE
     CurrentState testCurrentStateResource1 = Mockito.mock(CurrentState.class);
     Map<String, String> partitionStateMap1 = new HashMap<>();
     partitionStateMap1.put(_partitionNames.get(0), "MASTER");
@@ -179,9 +178,10 @@ public abstract class AbstractTestClusterModel {
     for (CurrentState cs : currentStatemap.values()) {
       ResourceConfig resourceConfig = dataProvider.getResourceConfig(cs.getResourceName());
       // Construct one AssignableReplica for each partition in the current state.
-      cs.getPartitionStateMap().entrySet().stream().forEach(entry -> assignmentSet.add(
-          new AssignableReplica(dataProvider.getClusterConfig(), resourceConfig, entry.getKey(),
-              entry.getValue(), entry.getValue().equals("MASTER") ? 1 : 2)));
+      cs.getPartitionStateMap().entrySet().stream()
+          .forEach(entry -> assignmentSet
+              .add(new AssignableReplica(dataProvider.getClusterConfig(), resourceConfig,
+                  entry.getKey(), entry.getValue(), entry.getValue().equals("MASTER") ? 1 : 2)));
     }
     return assignmentSet;
   }


[helix] 05/37: Convert all the internal assignment state objects to be ResourceAssignment. (#399)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 912ed34b0cb16dba350c480c6e962e557189528b
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Fri Aug 9 16:36:46 2019 -0700

    Convert all the internal assignment state objects to be ResourceAssignment. (#399)
    
    This is to avoid unnecessary information being recorded or read.
---
 .../rebalancer/waged/AssignmentMetadataStore.java          | 14 +++++++-------
 .../controller/rebalancer/waged/ClusterDataProvider.java   |  5 +++--
 .../controller/rebalancer/waged/RebalanceAlgorithm.java    |  3 +--
 .../waged/constraints/ConstraintsRebalanceAlgorithm.java   |  1 -
 4 files changed, 11 insertions(+), 12 deletions(-)

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
index 074ff4b..cc52dac 100644
--- 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
@@ -19,7 +19,7 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
-import org.apache.helix.model.IdealState;
+import org.apache.helix.model.ResourceAssignment;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -28,23 +28,23 @@ 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<>();
+  private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
+  private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
 
-  public Map<String, IdealState> getBaseline() {
+  public Map<String, ResourceAssignment> getBaseline() {
     return _persistGlobalBaseline;
   }
 
-  public void persistBaseline(Map<String, IdealState> globalBaseline) {
+  public void persistBaseline(Map<String, ResourceAssignment> globalBaseline) {
     // TODO clean up invalid items
     _persistGlobalBaseline = globalBaseline;
   }
 
-  public Map<String, IdealState> getBestPossibleAssignment() {
+  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
     return _persistBestPossibleAssignment;
   }
 
-  public void persistBestPossibleAssignment(Map<String, IdealState> bestPossibleAssignment) {
+  public void persistBestPossibleAssignment(Map<String, ResourceAssignment> bestPossibleAssignment) {
     // TODO clean up invalid items
     _persistBestPossibleAssignment.putAll(bestPossibleAssignment);
   }
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
index feae1dc..387666c 100644
--- 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
@@ -21,7 +21,7 @@ package org.apache.helix.controller.rebalancer.waged;
 
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
-import org.apache.helix.model.IdealState;
+import org.apache.helix.model.ResourceAssignment;
 
 import java.util.Map;
 import java.util.Set;
@@ -46,7 +46,8 @@ public class ClusterDataProvider {
    */
   protected static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
       Set<String> activeInstances, Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
-      Map<String, IdealState> baselineAssignment, Map<String, IdealState> bestPossibleAssignment) {
+      Map<String, ResourceAssignment> baselineAssignment,
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
     // TODO finish the implementation.
     return null;
   }
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
index ae258ca..d4221a1 100644
--- 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
@@ -21,7 +21,6 @@ package org.apache.helix.controller.rebalancer.waged;
 
 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 org.apache.helix.model.ResourceAssignment;
 
 import java.util.Map;
@@ -39,7 +38,7 @@ public interface RebalanceAlgorithm {
    * @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.
+   * @return A map of <ResourceName, ResourceAssignment>.
    */
   Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
       Map<String, Map<HardConstraint.FailureReason, Integer>> failureReasons);
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
index a75854a..286fd07 100644
--- 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
@@ -21,7 +21,6 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
 
 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.apache.helix.model.ResourceAssignment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;


[helix] 31/37: Integrate the WAGED rebalancer with all the related components. (#466)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5c6f1e9d5ca32d943a50878630c77421f4c066a8
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Wed Sep 18 13:42:43 2019 -0700

    Integrate the WAGED rebalancer with all the related components. (#466)
    
    1. Integrate with the algorithm, assignment metadata store, etc. Fix several conflicting interfaces and logics so as to all the rebalancer run correctly.
    2. Complete OptimalAssignment.
    3. Add integration tests to ensure the correctness of rebalancing logic.
---
 .../org/apache/helix/HelixRebalanceException.java  |   3 +
 .../rebalancer/waged/AssignmentMetadataStore.java  |  54 ++-
 .../rebalancer/waged/WagedRebalancer.java          | 248 ++++++++---
 .../constraints/ConstraintBasedAlgorithm.java      |  92 +++-
 .../NodeMaxPartitionLimitConstraint.java           |   9 +-
 .../rebalancer/waged/model/AssignableNode.java     |  38 +-
 .../rebalancer/waged/model/AssignableReplica.java  |  12 +-
 .../waged/model/ClusterModelProvider.java          |  30 ++
 .../rebalancer/waged/model/OptimalAssignment.java  |  52 ++-
 .../stages/BestPossibleStateCalcStage.java         | 140 +++---
 .../helix/manager/zk/ZkBucketDataAccessor.java     |   3 +-
 .../java/org/apache/helix/common/ZkTestBase.java   |  18 +-
 .../waged/MockAssignmentMetadataStore.java         |   9 +-
 .../waged/TestAssignmentMetadataStore.java         |  20 +-
 .../rebalancer/waged/TestWagedRebalancer.java      |  33 +-
 .../waged/constraints/MockRebalanceAlgorithm.java  |   2 +-
 .../waged/model/AbstractTestClusterModel.java      |   2 +-
 .../waged/model/TestOptimalAssignment.java         |  91 ++++
 .../WagedRebalancer/TestWagedRebalance.java        | 477 +++++++++++++++++++++
 .../TestWagedRebalanceFaultZone.java               | 374 ++++++++++++++++
 .../TestWagedRebalanceTopologyAware.java           | 114 +++++
 21 files changed, 1598 insertions(+), 223 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
index a8b5055..d54853f 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
@@ -23,10 +23,13 @@ package org.apache.helix;
  * Exception thrown by Helix due to rebalance failures.
  */
 public class HelixRebalanceException extends Exception {
+  // TODO: Adding static description or other necessary fields into the enum instances for
+  // TODO: supporting the rebalance monitor to understand the exception.
   public enum Type {
     INVALID_CLUSTER_STATUS,
     INVALID_REBALANCER_STATUS,
     FAILED_TO_CALCULATE,
+    INVALID_INPUT,
     UNKNOWN_FAILURE
   }
 
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
index fd655d1..a540ffb 100644
--- 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
@@ -20,13 +20,15 @@ package org.apache.helix.controller.rebalancer.waged;
  */
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.BucketDataAccessor;
 import org.apache.helix.HelixException;
-import org.apache.helix.HelixManager;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZNRecordJacksonSerializer;
@@ -50,23 +52,27 @@ public class AssignmentMetadataStore {
   private Map<String, ResourceAssignment> _globalBaseline;
   private Map<String, ResourceAssignment> _bestPossibleAssignment;
 
+  AssignmentMetadataStore(String metadataStoreAddrs, String clusterName) {
+    this(new ZkBucketDataAccessor(metadataStoreAddrs), clusterName);
+  }
+
   AssignmentMetadataStore(BucketDataAccessor bucketDataAccessor, String clusterName) {
     _dataAccessor = bucketDataAccessor;
     _baselinePath = String.format(BASELINE_TEMPLATE, clusterName, ASSIGNMENT_METADATA_KEY);
     _bestPossiblePath = String.format(BEST_POSSIBLE_TEMPLATE, clusterName, ASSIGNMENT_METADATA_KEY);
   }
 
-  AssignmentMetadataStore(HelixManager helixManager) {
-    this(new ZkBucketDataAccessor(helixManager.getMetadataStoreConnectionString()),
-        helixManager.getClusterName());
-  }
-
   public Map<String, ResourceAssignment> getBaseline() {
     // Return the in-memory baseline. If null, read from ZK. This is to minimize reads from ZK
     if (_globalBaseline == null) {
-      HelixProperty baseline =
-          _dataAccessor.compressedBucketRead(_baselinePath, HelixProperty.class);
-      _globalBaseline = splitAssignments(baseline);
+      try {
+        HelixProperty baseline =
+            _dataAccessor.compressedBucketRead(_baselinePath, HelixProperty.class);
+        _globalBaseline = splitAssignments(baseline);
+      } catch (ZkNoNodeException ex) {
+        // Metadata does not exist, so return an empty map
+        _globalBaseline = Collections.emptyMap();
+      }
     }
     return _globalBaseline;
   }
@@ -74,9 +80,14 @@ public class AssignmentMetadataStore {
   public Map<String, ResourceAssignment> getBestPossibleAssignment() {
     // Return the in-memory baseline. If null, read from ZK. This is to minimize reads from ZK
     if (_bestPossibleAssignment == null) {
-      HelixProperty baseline =
-          _dataAccessor.compressedBucketRead(_bestPossiblePath, HelixProperty.class);
-      _bestPossibleAssignment = splitAssignments(baseline);
+      try {
+        HelixProperty baseline =
+            _dataAccessor.compressedBucketRead(_bestPossiblePath, HelixProperty.class);
+        _bestPossibleAssignment = splitAssignments(baseline);
+      } catch (ZkNoNodeException ex) {
+        // Metadata does not exist, so return an empty map
+        _bestPossibleAssignment = Collections.emptyMap();
+      }
     }
     return _bestPossibleAssignment;
   }
@@ -113,6 +124,16 @@ public class AssignmentMetadataStore {
     _bestPossibleAssignment = bestPossibleAssignment;
   }
 
+  protected void finalize() {
+    // To ensure all resources are released.
+    close();
+  }
+
+  // Close to release all the resources.
+  public void close() {
+    _dataAccessor.disconnect();
+  }
+
   /**
    * Produces one HelixProperty that contains all assignment data.
    * @param name
@@ -123,8 +144,9 @@ public class AssignmentMetadataStore {
       Map<String, ResourceAssignment> assignmentMap) {
     HelixProperty property = new HelixProperty(name);
     // Add each resource's assignment as a simple field in one ZNRecord
+    // Node that don't use Arrays.toString() for the record converting. The deserialize will fail.
     assignmentMap.forEach((resource, assignment) -> property.getRecord().setSimpleField(resource,
-        Arrays.toString(SERIALIZER.serialize(assignment.getRecord()))));
+        new String(SERIALIZER.serialize(assignment.getRecord()))));
     return property;
   }
 
@@ -138,8 +160,8 @@ public class AssignmentMetadataStore {
     // Convert each resource's assignment String into a ResourceAssignment object and put it in a
     // map
     property.getRecord().getSimpleFields()
-        .forEach((resource, assignment) -> assignmentMap.put(resource,
-            new ResourceAssignment((ZNRecord) SERIALIZER.deserialize(assignment.getBytes()))));
+        .forEach((resource, assignmentStr) -> assignmentMap.put(resource,
+            new ResourceAssignment((ZNRecord) SERIALIZER.deserialize(assignmentStr.getBytes()))));
     return assignmentMap;
   }
 }
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
index 551239d..1861e10 100644
--- 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
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.changedetector.ResourceChangeDetector;
@@ -64,27 +65,34 @@ public class WagedRebalancer {
   // When any of the following change happens, the rebalancer needs to do a global rebalance which
   // contains 1. baseline recalculate, 2. partial rebalance that is based on the new baseline.
   private static final Set<HelixConstants.ChangeType> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES =
-      ImmutableSet.of(HelixConstants.ChangeType.RESOURCE_CONFIG,
-          HelixConstants.ChangeType.CLUSTER_CONFIG, HelixConstants.ChangeType.INSTANCE_CONFIG);
+      ImmutableSet.of(
+          HelixConstants.ChangeType.RESOURCE_CONFIG,
+          HelixConstants.ChangeType.CLUSTER_CONFIG,
+          HelixConstants.ChangeType.INSTANCE_CONFIG);
   // The cluster change detector is a stateful object.
   // Make it static to avoid unnecessary reinitialization.
   private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
       new ThreadLocal<>();
   private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
-
-  // --------- The following fields are placeholders and need replacement. -----------//
-  // TODO Shall we make the metadata store a static threadlocal object as well to avoid
-  // reinitialization?
   private final AssignmentMetadataStore _assignmentMetadataStore;
   private final RebalanceAlgorithm _rebalanceAlgorithm;
-  // ------------------------------------------------------------------------------------//
+
+  private static AssignmentMetadataStore constructAssignmentStore(HelixManager helixManager) {
+    AssignmentMetadataStore assignmentMetadataStore = null;
+    if (helixManager != null) {
+      String metadataStoreAddrs = helixManager.getMetadataStoreConnectionString();
+      String clusterName = helixManager.getClusterName();
+      if (metadataStoreAddrs != null && clusterName != null) {
+        assignmentMetadataStore = new AssignmentMetadataStore(metadataStoreAddrs, clusterName);
+      }
+    }
+    return assignmentMetadataStore;
+  }
 
   public WagedRebalancer(HelixManager helixManager,
       Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences) {
-    this(
-        // TODO init the metadata store according to their requirement when integrate,
-        // or change to final static method if possible.
-        new AssignmentMetadataStore(helixManager), ConstraintBasedAlgorithmFactory.getInstance(preferences),
+    this(constructAssignmentStore(helixManager),
+        ConstraintBasedAlgorithmFactory.getInstance(preferences),
         // Use DelayedAutoRebalancer as the mapping calculator for the final assignment output.
         // Mapping calculator will translate the best possible assignment into the applicable state
         // mapping based on the current states.
@@ -94,6 +102,10 @@ public class WagedRebalancer {
 
   private WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
       RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator) {
+    if (assignmentMetadataStore == null) {
+      LOG.warn("Assignment Metadata Store is not configured properly."
+          + " The rebalancer will not access the assignment store during the rebalance.");
+    }
     _assignmentMetadataStore = assignmentMetadataStore;
     _rebalanceAlgorithm = algorithm;
     _mappingCalculator = mappingCalculator;
@@ -103,7 +115,13 @@ public class WagedRebalancer {
   protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
       RebalanceAlgorithm algorithm) {
     this(assignmentMetadataStore, algorithm, new DelayedAutoRebalancer());
+  }
 
+  // Release all the resources.
+  public void close() {
+    if (_assignmentMetadataStore != null) {
+      _assignmentMetadataStore.close();
+    }
   }
 
   /**
@@ -117,27 +135,18 @@ public class WagedRebalancer {
   public Map<String, IdealState> computeNewIdealStates(ResourceControllerDataProvider clusterData,
       Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
       throws HelixRebalanceException {
-    LOG.info("Start computing new ideal states for resources: {}", resourceMap.keySet().toString());
-
-    // Find the compatible resources: 1. FULL_AUTO 2. Configured to use the WAGED rebalancer
-    resourceMap = resourceMap.entrySet().stream().filter(resourceEntry -> {
-      IdealState is = clusterData.getIdealState(resourceEntry.getKey());
-      return is != null && is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
-          && getClass().getName().equals(is.getRebalancerClassName());
-    }).collect(Collectors.toMap(resourceEntry -> resourceEntry.getKey(),
-        resourceEntry -> resourceEntry.getValue()));
-
     if (resourceMap.isEmpty()) {
-      LOG.warn("There is no valid resource to be rebalanced by {}",
+      LOG.warn("There is no resource to be rebalanced by {}",
           this.getClass().getSimpleName());
       return Collections.emptyMap();
-    } else {
-      LOG.info("Valid resources that will be rebalanced by {}: {}", this.getClass().getSimpleName(),
-          resourceMap.keySet().toString());
     }
 
+    LOG.info("Start computing new ideal states for resources: {}", resourceMap.keySet().toString());
+    validateInput(clusterData, resourceMap);
+
     // Calculate the target assignment based on the current cluster status.
-    Map<String, IdealState> newIdealStates = computeBestPossibleStates(clusterData, resourceMap);
+    Map<String, IdealState> newIdealStates =
+        computeBestPossibleStates(clusterData, resourceMap, currentStateOutput);
 
     // Construct the new best possible states according to the current state and target assignment.
     // Note that the new ideal state might be an intermediate state between the current state and
@@ -166,28 +175,29 @@ public class WagedRebalancer {
 
   // Coordinate baseline recalculation and partial rebalance according to the cluster changes.
   private Map<String, IdealState> computeBestPossibleStates(
-      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
-      throws HelixRebalanceException {
+      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap,
+      final CurrentStateOutput currentStateOutput) throws HelixRebalanceException {
     getChangeDetector().updateSnapshots(clusterData);
-    // Get all the modified and new items' information
+    // Get all the changed items' information
     Map<HelixConstants.ChangeType, Set<String>> clusterChanges =
         getChangeDetector().getChangeTypes().stream()
             .collect(Collectors.toMap(changeType -> changeType, changeType -> {
               Set<String> itemKeys = new HashSet<>();
               itemKeys.addAll(getChangeDetector().getAdditionsByType(changeType));
               itemKeys.addAll(getChangeDetector().getChangesByType(changeType));
+              itemKeys.addAll(getChangeDetector().getRemovalsByType(changeType));
               return itemKeys;
             }));
 
     if (clusterChanges.keySet().stream()
         .anyMatch(changeType -> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES.contains(changeType))) {
-      refreshBaseline(clusterData, clusterChanges, resourceMap);
+      refreshBaseline(clusterData, clusterChanges, resourceMap, currentStateOutput);
       // Inject a cluster config change for large scale partial rebalance once the baseline changed.
       clusterChanges.putIfAbsent(HelixConstants.ChangeType.CLUSTER_CONFIG, Collections.emptySet());
     }
 
     Map<String, ResourceAssignment> newAssignment =
-        partialRebalance(clusterData, clusterChanges, resourceMap);
+        partialRebalance(clusterData, clusterChanges, resourceMap, currentStateOutput);
 
     // Convert the assignments into IdealState for the following state mapping calculation.
     Map<String, IdealState> finalIdealState = new HashMap<>();
@@ -213,56 +223,60 @@ public class WagedRebalancer {
 
   // TODO make the Baseline calculation async if complicated algorithm is used for the Baseline
   private void refreshBaseline(ResourceControllerDataProvider clusterData,
-      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap)
-      throws HelixRebalanceException {
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap,
+      final CurrentStateOutput currentStateOutput) throws HelixRebalanceException {
+    LOG.info("Start calculating the new baseline.");
+    Map<String, ResourceAssignment> currentBaseline =
+        getBaselineAssignment(_assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
     // For baseline calculation
     // 1. Ignore node status (disable/offline).
     // 2. Use the baseline as the previous best possible assignment since there is no "baseline" for
     // the baseline.
-    LOG.info("Start calculating the new baseline.");
-    Map<String, ResourceAssignment> currentBaseline;
-    try {
-      currentBaseline = _assignmentMetadataStore.getBaseline();
-    } catch (Exception ex) {
-      throw new HelixRebalanceException("Failed to get the current baseline assignment.",
-          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
-    }
-    Map<String, ResourceAssignment> baseline = calculateAssignment(clusterData, clusterChanges,
-        resourceMap, clusterData.getAllInstances(), Collections.emptyMap(), currentBaseline);
-    try {
-      _assignmentMetadataStore.persistBaseline(baseline);
-    } catch (Exception ex) {
-      throw new HelixRebalanceException("Failed to persist the new baseline assignment.",
-          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    Map<String, ResourceAssignment> newBaseline =
+        calculateAssignment(clusterData, clusterChanges, resourceMap, clusterData.getAllInstances(),
+            Collections.emptyMap(), currentBaseline);
+
+    if (_assignmentMetadataStore != null) {
+      try {
+        _assignmentMetadataStore.persistBaseline(newBaseline);
+      } catch (Exception ex) {
+        throw new HelixRebalanceException("Failed to persist the new baseline assignment.",
+            HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+      }
+    } else {
+      LOG.debug("Assignment Metadata Store is empty. Skip persist the baseline assignment.");
     }
+
     LOG.info("Finish calculating the new baseline.");
   }
 
   private Map<String, ResourceAssignment> partialRebalance(
       ResourceControllerDataProvider clusterData,
-      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap)
-      throws HelixRebalanceException {
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap,
+      final CurrentStateOutput currentStateOutput) throws HelixRebalanceException {
     LOG.info("Start calculating the new best possible assignment.");
-    Set<String> activeInstances = clusterData.getEnabledLiveInstances();
-    Map<String, ResourceAssignment> baseline;
-    Map<String, ResourceAssignment> prevBestPossibleAssignment;
-    try {
-      baseline = _assignmentMetadataStore.getBaseline();
-      prevBestPossibleAssignment = _assignmentMetadataStore.getBestPossibleAssignment();
-    } catch (Exception ex) {
-      throw new HelixRebalanceException("Failed to get the persisted assignment records.",
-          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
-    }
-    Map<String, ResourceAssignment> newAssignment = calculateAssignment(clusterData, clusterChanges,
-        resourceMap, activeInstances, baseline, prevBestPossibleAssignment);
-    try {
-      // TODO Test to confirm if persisting the final assignment (with final partition states)
-      // would be a better option.
-      _assignmentMetadataStore.persistBestPossibleAssignment(newAssignment);
-    } catch (Exception ex) {
-      throw new HelixRebalanceException("Failed to persist the new best possible assignment.",
-          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    Map<String, ResourceAssignment> currentBaseline =
+        getBaselineAssignment(_assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
+    Map<String, ResourceAssignment> currentBestPossibleAssignment =
+        getBestPossibleAssignment(_assignmentMetadataStore, currentStateOutput,
+            resourceMap.keySet());
+    Map<String, ResourceAssignment> newAssignment =
+        calculateAssignment(clusterData, clusterChanges, resourceMap,
+            clusterData.getEnabledLiveInstances(), currentBaseline, currentBestPossibleAssignment);
+
+    if (_assignmentMetadataStore != null) {
+      try {
+        // TODO Test to confirm if persisting the final assignment (with final partition states)
+        // would be a better option.
+        _assignmentMetadataStore.persistBestPossibleAssignment(newAssignment);
+      } catch (Exception ex) {
+        throw new HelixRebalanceException("Failed to persist the new best possible assignment.",
+            HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+      }
+    } else {
+      LOG.debug("Assignment Metadata Store is empty. Skip persist the baseline assignment.");
     }
+
     LOG.info("Finish calculating the new best possible assignment.");
     return newAssignment;
   }
@@ -348,4 +362,100 @@ public class WagedRebalancer {
     }
     return preferenceList;
   }
+
+  private void validateInput(ResourceControllerDataProvider clusterData,
+      Map<String, Resource> resourceMap) throws HelixRebalanceException {
+    Set<String> nonCompatibleResources = resourceMap.entrySet().stream().filter(resourceEntry -> {
+      IdealState is = clusterData.getIdealState(resourceEntry.getKey());
+      return is == null || !is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
+          || !getClass().getName().equals(is.getRebalancerClassName());
+    }).map(Map.Entry::getKey).collect(Collectors.toSet());
+    if (!nonCompatibleResources.isEmpty()) {
+      throw new HelixRebalanceException(String.format(
+          "Input contains invalid resource(s) that cannot be rebalanced by the WAGED rebalancer. %s",
+          nonCompatibleResources.toString()), HelixRebalanceException.Type.INVALID_INPUT);
+    }
+  }
+
+  /**
+   * @param assignmentMetadataStore
+   * @param currentStateOutput
+   * @param resources
+   * @return The current baseline assignment. If record does not exist in the
+   * assignmentMetadataStore, return the current state assignment.
+   * @throws HelixRebalanceException
+   */
+  private Map<String, ResourceAssignment> getBaselineAssignment(
+      AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput currentStateOutput,
+      Set<String> resources) throws HelixRebalanceException {
+    Map<String, ResourceAssignment> currentBaseline = Collections.emptyMap();
+    if (assignmentMetadataStore != null) {
+      try {
+        currentBaseline = assignmentMetadataStore.getBaseline();
+      } catch (HelixException ex) {
+        // Report error. and use empty mapping instead.
+        LOG.error("Failed to get the current baseline assignment.", ex);
+      } catch (Exception ex) {
+        throw new HelixRebalanceException(
+            "Failed to get the current baseline assignment because of unexpected error.",
+            HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+      }
+    }
+    if (currentBaseline.isEmpty()) {
+      LOG.warn(
+          "The current baseline assignment record is empty. Use the current states instead.");
+      currentBaseline = getCurrentStateAssingment(currentStateOutput, resources);
+    }
+    return currentBaseline;
+  }
+
+  /**
+   * @param assignmentMetadataStore
+   * @param currentStateOutput
+   * @param resources
+   * @return The current best possible assignment. If record does not exist in the
+   * assignmentMetadataStore, return the current state assignment.
+   * @throws HelixRebalanceException
+   */
+  private Map<String, ResourceAssignment> getBestPossibleAssignment(
+      AssignmentMetadataStore assignmentMetadataStore, CurrentStateOutput currentStateOutput,
+      Set<String> resources) throws HelixRebalanceException {
+    Map<String, ResourceAssignment> currentBestAssignment = Collections.emptyMap();
+    if (assignmentMetadataStore != null) {
+      try {
+        currentBestAssignment = assignmentMetadataStore.getBestPossibleAssignment();
+      } catch (HelixException ex) {
+        // Report error. and use empty mapping instead.
+        LOG.error("Failed to get the current best possible assignment.", ex);
+      } catch (Exception ex) {
+        throw new HelixRebalanceException(
+            "Failed to get the current best possible assignment because of unexpected error.",
+            HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+      }
+    }
+    if (currentBestAssignment.isEmpty()) {
+      LOG.warn(
+          "The current best possible assignment record is empty. Use the current states instead.");
+      currentBestAssignment = getCurrentStateAssingment(currentStateOutput, resources);
+    }
+    return currentBestAssignment;
+  }
+
+  private Map<String, ResourceAssignment> getCurrentStateAssingment(
+      CurrentStateOutput currentStateOutput, Set<String> resourceSet) {
+    Map<String, ResourceAssignment> currentStateAssignment = new HashMap<>();
+    for (String resourceName : resourceSet) {
+      Map<Partition, Map<String, String>> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName);
+      if (!currentStateMap.isEmpty()) {
+        ResourceAssignment newResourceAssignment = new ResourceAssignment(resourceName);
+        currentStateMap.entrySet().stream().forEach(currentStateEntry -> {
+          newResourceAssignment
+              .addReplicaMap(currentStateEntry.getKey(), currentStateEntry.getValue());
+        });
+        currentStateAssignment.put(resourceName, newResourceAssignment);
+      }
+    }
+    return currentStateAssignment;
+  }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
index 89a3f29..1a41aef 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
@@ -29,6 +29,7 @@ import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
+import com.google.common.collect.Maps;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
 import org.apache.helix.controller.rebalancer.waged.model.AssignableNode;
@@ -36,11 +37,10 @@ import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
 import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
+import org.apache.helix.model.ResourceAssignment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
-
 /**
  * The algorithm is based on a given set of constraints
  * - HardConstraint: Approve or deny the assignment given its condition, any assignment cannot
@@ -64,29 +64,26 @@ class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
   @Override
   public OptimalAssignment calculate(ClusterModel clusterModel) throws HelixRebalanceException {
     OptimalAssignment optimalAssignment = new OptimalAssignment();
-    Map<String, Set<AssignableReplica>> replicasByResource = clusterModel.getAssignableReplicaMap();
     List<AssignableNode> nodes = new ArrayList<>(clusterModel.getAssignableNodes().values());
-
-    // TODO: different orders of resource/replica could lead to different greedy assignments, will
-    // revisit and improve the performance
-    for (String resource : replicasByResource.keySet()) {
-      for (AssignableReplica replica : replicasByResource.get(resource)) {
-        Optional<AssignableNode> maybeBestNode =
-            getNodeWithHighestPoints(replica, nodes, clusterModel.getContext(), optimalAssignment);
-        // stop immediately if any replica cannot find best assignable node
-        if (optimalAssignment.hasAnyFailure()) {
-          String errorMessage = String.format(
-              "Unable to find any available candidate node for partition %s; Fail reasons: %s",
-              replica.getPartitionName(), optimalAssignment.getFailures());
-          throw new HelixRebalanceException(errorMessage,
-              HelixRebalanceException.Type.FAILED_TO_CALCULATE);
-        }
-        maybeBestNode.ifPresent(node -> clusterModel.assign(replica.getResourceName(),
-            replica.getPartitionName(), replica.getReplicaState(), node.getInstanceName()));
+    // Sort the replicas so the input is stable for the greedy algorithm.
+    // For the other algorithm implementation, this sorting could be unnecessary.
+    for (AssignableReplica replica : getOrderedAssignableReplica(clusterModel)) {
+      Optional<AssignableNode> maybeBestNode =
+          getNodeWithHighestPoints(replica, nodes, clusterModel.getContext(), optimalAssignment);
+      // stop immediately if any replica cannot find best assignable node
+      if (optimalAssignment.hasAnyFailure()) {
+        String errorMessage = String.format(
+            "Unable to find any available candidate node for partition %s; Fail reasons: %s",
+            replica.getPartitionName(), optimalAssignment.getFailures());
+        throw new HelixRebalanceException(errorMessage,
+            HelixRebalanceException.Type.FAILED_TO_CALCULATE);
       }
+      maybeBestNode.ifPresent(node -> clusterModel
+          .assign(replica.getResourceName(), replica.getPartitionName(), replica.getReplicaState(),
+              node.getInstanceName()));
     }
-
-    return optimalAssignment.convertFrom(clusterModel);
+    optimalAssignment.updateAssignments(clusterModel);
+    return optimalAssignment;
   }
 
   private Optional<AssignableNode> getNodeWithHighestPoints(AssignableReplica replica,
@@ -133,4 +130,55 @@ class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
     return hardConstraints.stream().map(HardConstraint::getDescription)
         .collect(Collectors.toList());
   }
+
+  // TODO investigate better ways to sort replicas. One option is sorting based on the creation time.
+  private List<AssignableReplica> getOrderedAssignableReplica(ClusterModel clusterModel) {
+    Map<String, Set<AssignableReplica>> replicasByResource = clusterModel.getAssignableReplicaMap();
+    List<AssignableReplica> orderedAssignableReplicas =
+        replicasByResource.values().stream().flatMap(replicas -> replicas.stream())
+            .collect(Collectors.toList());
+
+    Map<String, ResourceAssignment> bestPossibleAssignment =
+        clusterModel.getContext().getBestPossibleAssignment();
+    Map<String, ResourceAssignment> baselineAssignment =
+        clusterModel.getContext().getBaselineAssignment();
+
+    // 1. Sort according if the assignment exists in the best possible and/or baseline assignment
+    // 2. Sort according to the state priority. Note that prioritizing the top state is required.
+    // Or the greedy algorithm will unnecessarily shuffle the states between replicas.
+    // 3. Sort according to the resource/partition name.
+    orderedAssignableReplicas.sort((replica1, replica2) -> {
+      String resourceName1 = replica1.getResourceName();
+      String resourceName2 = replica2.getResourceName();
+      if (bestPossibleAssignment.containsKey(resourceName1) == bestPossibleAssignment
+          .containsKey(resourceName2)) {
+        if (baselineAssignment.containsKey(resourceName1) == baselineAssignment
+            .containsKey(resourceName2)) {
+          // If both assignment states have/not have the resource assignment the same,
+          // compare for additional dimensions.
+          int statePriority1 = replica1.getStatePriority();
+          int statePriority2 = replica2.getStatePriority();
+          if (statePriority1 == statePriority2) {
+            // If state prioritizes are the same, compare the names.
+            if (resourceName1.equals(resourceName2)) {
+              return replica1.getPartitionName().compareTo(replica2.getPartitionName());
+            } else {
+              return resourceName1.compareTo(resourceName2);
+            }
+          } else {
+            // Note we shall prioritize the replica with a higher state priority,
+            // the smaller priority number means higher priority.
+            return statePriority1 - statePriority2;
+          }
+        } else {
+          // If the baseline assignment contains the assignment, prioritize the replica.
+          return baselineAssignment.containsKey(resourceName1) ? -1 : 1;
+        }
+      } else {
+        // If the best possible assignment contains the assignment, prioritize the replica.
+        return bestPossibleAssignment.containsKey(resourceName1) ? -1 : 1;
+      }
+    });
+    return orderedAssignableReplicas;
+  }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
index 9d0752b..cda5329 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeMaxPartitionLimitConstraint.java
@@ -28,9 +28,12 @@ class NodeMaxPartitionLimitConstraint extends HardConstraint {
   @Override
   boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext) {
-    return node.getAssignedReplicaCount() < node.getMaxPartition()
-        && node.getAssignedPartitionsByResource(replica.getResourceName()).size() < replica
-            .getResourceMaxPartitionsPerInstance();
+    boolean exceedMaxPartitionLimit =
+        node.getMaxPartition() < 0 || node.getAssignedReplicaCount() < node.getMaxPartition();
+    boolean exceedResourceMaxPartitionLimit = replica.getResourceMaxPartitionsPerInstance() < 0
+        || node.getAssignedPartitionsByResource(replica.getResourceName()).size() < replica
+        .getResourceMaxPartitionsPerInstance();
+    return exceedMaxPartitionLimit && exceedResourceMaxPartitionLimit;
   }
 
   @Override
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
index a3460fb..20de6da 100644
--- 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
@@ -19,8 +19,6 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import static java.lang.Math.max;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -287,16 +285,23 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * Any missing field will cause an invalid topology config exception.
    */
   private String computeFaultZone(ClusterConfig clusterConfig, InstanceConfig instanceConfig) {
-    if (clusterConfig.isTopologyAwareEnabled()) {
-      String topologyStr = clusterConfig.getTopology();
-      String faultZoneType = clusterConfig.getFaultZoneType();
-      if (topologyStr == null || faultZoneType == null) {
-        throw new HelixException("Fault zone or cluster topology information is not configured.");
-      }
-
+    if (!clusterConfig.isTopologyAwareEnabled()) {
+      // Instance name is the default fault zone if topology awareness is false.
+      return instanceConfig.getInstanceName();
+    }
+    String topologyStr = clusterConfig.getTopology();
+    String faultZoneType = clusterConfig.getFaultZoneType();
+    if (topologyStr == null || faultZoneType == null) {
+      LOG.debug("Topology configuration is not complete. Topology define: {}, Fault Zone Type: {}",
+          topologyStr, faultZoneType);
+      // Use the instance name, or the deprecated ZoneId field (if exists) as the default fault zone.
+      String zoneId = instanceConfig.getZoneId();
+      return zoneId == null ? instanceConfig.getInstanceName() : zoneId;
+    } else {
+      // Get the fault zone information from the complete topology definition.
       String[] topologyDef = topologyStr.trim().split("/");
-      if (topologyDef.length == 0
-          || Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
+      if (topologyDef.length == 0 ||
+          Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
         throw new HelixException(
             "The configured topology definition is empty or does not contain the fault zone type.");
       }
@@ -324,10 +329,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
         }
         return faultZoneStringBuilder.toString();
       }
-    } else {
-      // For backward compatibility
-      String zoneId = instanceConfig.getZoneId();
-      return zoneId == null ? instanceConfig.getInstanceName() : zoneId;
     }
   }
 
@@ -356,7 +357,7 @@ public class AssignableNode implements Comparable<AssignableNode> {
       // For the purpose of constraint calculation, the max utilization cannot be larger than 100%.
       float utilization = Math.min(
           (float) (_maxCapacity.get(capacityKey) - newCapacity) / _maxCapacity.get(capacityKey), 1);
-      _highestCapacityUtilization = max(_highestCapacityUtilization, utilization);
+      _highestCapacityUtilization = Math.max(_highestCapacityUtilization, utilization);
     }
     // else if the capacityKey does not exist in the capacity map, this method essentially becomes
     // a NOP; in other words, this node will be treated as if it has unlimited capacity.
@@ -394,4 +395,9 @@ public class AssignableNode implements Comparable<AssignableNode> {
   public int compareTo(AssignableNode o) {
     return _instanceName.compareTo(o.getInstanceName());
   }
+
+  @Override
+  public String toString() {
+    return _instanceName;
+  }
 }
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
index 66bd7b7..a651e19 100644
--- 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
@@ -20,17 +20,22 @@ package org.apache.helix.controller.rebalancer.waged.model;
  */
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class represents a partition replication that needs to be allocated.
  */
 public class AssignableReplica implements Comparable<AssignableReplica> {
+  private static final Logger LOG = LoggerFactory.getLogger(AssignableReplica.class);
+
   private final String _partitionName;
   private final String _resourceName;
   private final String _resourceInstanceGroupTag;
@@ -149,9 +154,10 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
       partitionCapacity = capacityMap.get(ResourceConfig.DEFAULT_PARTITION_KEY);
     }
     if (partitionCapacity == null) {
-      throw new IllegalArgumentException(String.format(
-          "The capacity usage of the specified partition %s is not configured in the Resource Config %s. No default partition capacity is configured neither.",
-          partitionName, resourceConfig.getResourceName()));
+      LOG.warn("The capacity usage of the specified partition {} is not configured in the Resource"
+          + " Config {}. No default partition capacity is configured either. Will proceed with"
+          + " empty capacity configuration.", partitionName, resourceConfig.getResourceName());
+      partitionCapacity = new HashMap<>();
     }
 
     List<String> requiredCapacityKeys = clusterConfig.getInstanceCapacityKeys();
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index af1a8d8..2b53422 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -200,6 +200,9 @@ public class ClusterModelProvider {
 
     for (String resourceName : resourceMap.keySet()) {
       ResourceConfig resourceConfig = dataProvider.getResourceConfig(resourceName);
+      if (resourceConfig == null) {
+        resourceConfig = new ResourceConfig(resourceName);
+      }
       IdealState is = dataProvider.getIdealState(resourceName);
       if (is == null) {
         throw new HelixException(
@@ -223,6 +226,7 @@ public class ClusterModelProvider {
         for (Map.Entry<String, Integer> entry : stateCountMap.entrySet()) {
           String state = entry.getKey();
           for (int i = 0; i < entry.getValue(); i++) {
+            mergeIdealStateWithResourceConfig(resourceConfig, is);
             totalReplicaMap.computeIfAbsent(resourceName, key -> new HashSet<>()).add(
                 new AssignableReplica(clusterConfig, resourceConfig, partition, state,
                     def.getStatePriorityMap().get(state)));
@@ -234,6 +238,32 @@ public class ClusterModelProvider {
   }
 
   /**
+   * For backward compatibility, propagate the critical simple fields from the IdealState to
+   * the Resource Config.
+   * Eventually, Resource Config should be the only metadata node that contains the required information.
+   */
+  private static void mergeIdealStateWithResourceConfig(ResourceConfig resourceConfig,
+      final IdealState idealState) {
+    // Note that the config fields get updated in this method shall be fully compatible with ones in the IdealState.
+    // 1. The fields shall have exactly the same meaning.
+    // 2. The value shall be exactly compatible, no additional calculation involved.
+    // 3. Resource Config items have a high priority.
+    // This is to ensure the resource config is not polluted after the merge.
+    if (null == resourceConfig.getRecord()
+        .getSimpleField(ResourceConfig.ResourceConfigProperty.INSTANCE_GROUP_TAG.name())) {
+      resourceConfig.getRecord()
+          .setSimpleField(ResourceConfig.ResourceConfigProperty.INSTANCE_GROUP_TAG.name(),
+              idealState.getInstanceGroupTag());
+    }
+    if (null == resourceConfig.getRecord()
+        .getSimpleField(ResourceConfig.ResourceConfigProperty.MAX_PARTITIONS_PER_INSTANCE.name())) {
+      resourceConfig.getRecord()
+          .setIntField(ResourceConfig.ResourceConfigProperty.MAX_PARTITIONS_PER_INSTANCE.name(),
+              idealState.getMaxPartitionsPerInstance());
+    }
+  }
+
+  /**
    * @return A map contains the assignments for each fault zone. <fault zone, <resource, set of partitions>>
    */
   private static Map<String, Map<String, Set<String>>> mapAssignmentToFaultZone(
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java
index 31cb181..138f30c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java
@@ -19,38 +19,64 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.helix.HelixException;
+import org.apache.helix.model.Partition;
 import org.apache.helix.model.ResourceAssignment;
 
 /**
  * The data model represents the optimal assignment of N replicas assigned to M instances;
  * It's mostly used as the return parameter of an assignment calculation algorithm; If the algorithm
- * failed to find optimal assignment given the endeavor, the user could check the failure reasons
+ * failed to find optimal assignment given the endeavor, the user could check the failure reasons.
+ * Note that this class is not thread safe.
  */
 public class OptimalAssignment {
   private Map<AssignableNode, List<AssignableReplica>> _optimalAssignment = new HashMap<>();
   private Map<AssignableReplica, Map<AssignableNode, List<String>>> _failedAssignments =
       new HashMap<>();
 
-  public OptimalAssignment() {
-
-  }
-
+  /**
+   * Update the OptimalAssignment instance with the existing assignment recorded in the input cluster model.
+   *
+   * @param clusterModel
+   */
   public void updateAssignments(ClusterModel clusterModel) {
-
+    _optimalAssignment.clear();
+    clusterModel.getAssignableNodes().values().stream()
+        .forEach(node -> _optimalAssignment.put(node, new ArrayList<>(node.getAssignedReplicas())));
   }
 
-  // TODO: determine the output of final assignment format
+  /**
+   * @return The optimal assignment in the form of a <Resource Name, ResourceAssignment> map.
+   */
   public Map<String, ResourceAssignment> getOptimalResourceAssignment() {
-    throw new UnsupportedOperationException("Not implemented yet");
-  }
-
-  // TODO: the convert method is not the best choice so far, will revisit the data model
-  public OptimalAssignment convertFrom(ClusterModel clusterModel) {
-    return this;
+    if (hasAnyFailure()) {
+      throw new HelixException(
+          "Cannot get the optimal resource assignment since a calculation failure is recorded. "
+              + getFailures());
+    }
+    Map<String, ResourceAssignment> assignmentMap = new HashMap<>();
+    for (AssignableNode node : _optimalAssignment.keySet()) {
+      for (AssignableReplica replica : _optimalAssignment.get(node)) {
+        String resourceName = replica.getResourceName();
+        Partition partition = new Partition(replica.getPartitionName());
+        ResourceAssignment resourceAssignment = assignmentMap
+            .computeIfAbsent(resourceName, key -> new ResourceAssignment(resourceName));
+        Map<String, String> partitionStateMap = resourceAssignment.getReplicaMap(partition);
+        if (partitionStateMap.isEmpty()) {
+          // ResourceAssignment returns immutable empty map while no such assignment recorded yet.
+          // So if the returned map is empty, create a new map.
+          partitionStateMap = new HashMap<>();
+        }
+        partitionStateMap.put(node.getInstanceName(), replica.getReplicaState());
+        resourceAssignment.addReplicaMap(partition, partitionStateMap);
+      }
+    }
+    return assignmentMap;
   }
 
   public void recordAssignmentFailure(AssignableReplica replica,
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 806ef85..cd7ab59 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
+import java.util.stream.Collectors;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
@@ -114,67 +115,46 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
 
     // Check whether the offline/disabled instance count in the cluster reaches the set limit,
     // if yes, pause the rebalancer.
-    boolean isValid = validateOfflineInstancesLimit(cache,
-        (HelixManager) event.getAttribute(AttributeName.helixmanager.name()));
-
-    // 1. Rebalance with the WAGED rebalancer
-    // The rebalancer only calculates the new ideal assignment for all the resources that are
-    // configured to use the WAGED rebalancer.
-    // For the other resources, the legacy rebalancers will be triggered in the next step.
-    Map<String, IdealState> newIdealStates = new HashMap<>();
-    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences = cache.getClusterConfig()
-        .getGlobalRebalancePreference();
-    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager, preferences);
-    try {
-      newIdealStates
-          .putAll(wagedRebalancer.computeNewIdealStates(cache, resourceMap, currentStateOutput));
-    } catch (HelixRebalanceException ex) {
-      // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
-      // Since it calculates for all the eligible resources globally, a partial result is invalid.
-      // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
-      LogUtil.logError(logger, _eventId, String
-          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
-              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
-    }
+    boolean isValid =
+        validateOfflineInstancesLimit(cache, event.getAttribute(AttributeName.helixmanager.name()));
 
     final List<String> failureResources = new ArrayList<>();
-    Iterator<Resource> itr = resourceMap.values().iterator();
+
+    Map<String, Resource> calculatedResourceMap =
+        computeResourceBestPossibleStateWithWagedRebalancer(cache, currentStateOutput, helixManager,
+            resourceMap, output, failureResources);
+
+    Map<String, Resource> remainingResourceMap = new HashMap<>(resourceMap);
+    remainingResourceMap.keySet().removeAll(calculatedResourceMap.keySet());
+
+    // Fallback to the original single resource rebalancer calculation.
+    // This is required because we support mixed cluster that uses both WAGED rebalancer and the
+    // older rebalancers.
+    Iterator<Resource> itr = remainingResourceMap.values().iterator();
     while (itr.hasNext()) {
       Resource resource = itr.next();
       boolean result = false;
-      IdealState is = newIdealStates.get(resource.getResourceName());
-      if (is != null) {
-        // 2. Check if the WAGED rebalancer has calculated for this resource or not.
-        result = checkBestPossibleStateCalculation(is);
-        if (result) {
-          // The WAGED rebalancer calculates a valid result, record in the output
-          updateBestPossibleStateOutput(output, resource, is);
-        }
-      } else {
-        // 3. The WAGED rebalancer skips calculating the resource assignment, fallback to use a
-        // legacy resource rebalancer if applicable.
-        // If this calculation fails, the resource will be reported in the failureResources list.
-        try {
-          result =
-              computeSingleResourceBestPossibleState(event, cache, currentStateOutput, resource,
-                  output);
-        } catch (HelixException ex) {
-          LogUtil.logError(logger, _eventId,
-              "Exception when calculating best possible states for " + resource.getResourceName(),
-              ex);
-        }
+      try {
+        result = computeSingleResourceBestPossibleState(event, cache, currentStateOutput, resource,
+            output);
+      } catch (HelixException ex) {
+        LogUtil.logError(logger, _eventId, String
+            .format("Exception when calculating best possible states for %s",
+                resource.getResourceName()), ex);
+
       }
       if (!result) {
         failureResources.add(resource.getResourceName());
-        LogUtil.logWarn(logger, _eventId,
-            "Failed to calculate best possible states for " + resource.getResourceName());
+        LogUtil.logWarn(logger, _eventId, String
+            .format("Failed to calculate best possible states for %s", resource.getResourceName()));
       }
     }
 
     // Check and report if resource rebalance has failure
     updateRebalanceStatus(!isValid || !failureResources.isEmpty(), failureResources, helixManager,
-        cache, clusterStatusMonitor,
-        "Failed to calculate best possible states for " + failureResources.size() + " resources.");
+        cache, clusterStatusMonitor, String
+            .format("Failed to calculate best possible states for %d resources.",
+                failureResources.size()));
 
     return output;
   }
@@ -238,6 +218,70 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
     return true;
   }
 
+  /**
+   * Rebalance with the WAGED rebalancer
+   * The rebalancer only calculates the new ideal assignment for all the resources that are
+   * configured to use the WAGED rebalancer.
+   *
+   * @param cache              Cluster data cache.
+   * @param currentStateOutput The current state information.
+   * @param helixManager
+   * @param resourceMap        The complete resource map. The method will filter the map for the compatible resources.
+   * @param output             The best possible state output.
+   * @param failureResources   The failure records that will be updated if any resource cannot be computed.
+   * @return The map of all the calculated resources.
+   */
+  private Map<String, Resource> computeResourceBestPossibleStateWithWagedRebalancer(
+      ResourceControllerDataProvider cache, CurrentStateOutput currentStateOutput,
+      HelixManager helixManager, Map<String, Resource> resourceMap, BestPossibleStateOutput output,
+      List<String> failureResources) {
+    // Find the compatible resources: 1. FULL_AUTO 2. Configured to use the WAGED rebalancer
+    Map<String, Resource> wagedRebalancedResourceMap =
+        resourceMap.entrySet().stream().filter(resourceEntry -> {
+          IdealState is = cache.getIdealState(resourceEntry.getKey());
+          return is != null && is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
+              && WagedRebalancer.class.getName().equals(is.getRebalancerClassName());
+        }).collect(Collectors.toMap(resourceEntry -> resourceEntry.getKey(),
+            resourceEntry -> resourceEntry.getValue()));
+
+    Map<String, IdealState> newIdealStates = new HashMap<>();
+
+    // Init rebalancer with the rebalance preferences.
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preferences = cache.getClusterConfig()
+        .getGlobalRebalancePreference();
+    // TODO avoid creating the rebalancer on every rebalance call for performance enhancement
+    WagedRebalancer wagedRebalancer = new WagedRebalancer(helixManager, preferences);
+    try {
+      newIdealStates.putAll(wagedRebalancer
+          .computeNewIdealStates(cache, wagedRebalancedResourceMap, currentStateOutput));
+    } catch (HelixRebalanceException ex) {
+      // Note that unlike the legacy rebalancer, the WAGED rebalance won't return partial result.
+      // Since it calculates for all the eligible resources globally, a partial result is invalid.
+      // TODO propagate the rebalancer failure information to updateRebalanceStatus for monitoring.
+      LogUtil.logError(logger, _eventId, String
+          .format("Failed to calculate the new Ideal States using the rebalancer %s due to %s",
+              wagedRebalancer.getClass().getSimpleName(), ex.getFailureType()), ex);
+    } finally {
+      wagedRebalancer.close();
+    }
+    Iterator<Resource> itr = wagedRebalancedResourceMap.values().iterator();
+    while (itr.hasNext()) {
+      Resource resource = itr.next();
+      IdealState is = newIdealStates.get(resource.getResourceName());
+      // Check if the WAGED rebalancer has calculated the result for this resource or not.
+      if (is != null && checkBestPossibleStateCalculation(is)) {
+        // The WAGED rebalancer calculates a valid result, record in the output
+        updateBestPossibleStateOutput(output, resource, is);
+      } else {
+        failureResources.add(resource.getResourceName());
+        LogUtil.logWarn(logger, _eventId, String
+            .format("Failed to calculate best possible states for %s.",
+                resource.getResourceName()));
+      }
+    }
+    return wagedRebalancedResourceMap;
+  }
+
   private void updateBestPossibleStateOutput(BestPossibleStateOutput output, Resource resource,
       IdealState computedIdealState) {
     output.setPreferenceLists(resource.getResourceName(), computedIdealState.getPreferenceLists());
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java
index 24c7c8e..a11da29 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import org.I0Itec.zkclient.exception.ZkMarshallingError;
+import org.I0Itec.zkclient.exception.ZkNoNodeException;
 import org.I0Itec.zkclient.serialize.ZkSerializer;
 import org.apache.helix.AccessOption;
 import org.apache.helix.BaseDataAccessor;
@@ -215,7 +216,7 @@ public class ZkBucketDataAccessor implements BucketDataAccessor, AutoCloseable {
         ZNRecord metadataRecord =
             _znRecordBaseDataAccessor.get(path, null, AccessOption.PERSISTENT);
         if (metadataRecord == null) {
-          throw new HelixException(
+          throw new ZkNoNodeException(
               String.format("Metadata ZNRecord does not exist for path: %s", path));
         }
 
diff --git a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
index 03338b4..b9284b9 100644
--- a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
@@ -30,9 +30,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.logging.Level;
-
 import javax.management.MBeanServerConnection;
 import javax.management.ObjectName;
+
 import org.I0Itec.zkclient.IZkStateListener;
 import org.I0Itec.zkclient.ZkServer;
 import org.apache.helix.BaseDataAccessor;
@@ -54,6 +54,7 @@ import org.apache.helix.controller.pipeline.Stage;
 import org.apache.helix.controller.pipeline.StageContext;
 import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
 import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy;
+import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
 import org.apache.helix.controller.stages.AttributeName;
 import org.apache.helix.controller.stages.ClusterEvent;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
@@ -347,6 +348,19 @@ public class ZkTestBase {
   protected IdealState createResourceWithDelayedRebalance(String clusterName, String db,
       String stateModel, int numPartition, int replica, int minActiveReplica, long delay,
       String rebalanceStrategy) {
+    return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica,
+        delay, DelayedAutoRebalancer.class.getName(), rebalanceStrategy);
+  }
+
+  protected IdealState createResourceWithWagedRebalance(String clusterName, String db,
+      String stateModel, int numPartition, int replica, int minActiveReplica, long delay) {
+    return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica,
+        delay, WagedRebalancer.class.getName(), null);
+  }
+
+  private IdealState createResource(String clusterName, String db, String stateModel,
+      int numPartition, int replica, int minActiveReplica, long delay, String rebalancerClassName,
+      String rebalanceStrategy) {
     IdealState idealState =
         _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
     if (idealState == null) {
@@ -362,7 +376,7 @@ public class ZkTestBase {
     if (delay > 0) {
       idealState.setRebalanceDelay(delay);
     }
-    idealState.setRebalancerClassName(DelayedAutoRebalancer.class.getName());
+    idealState.setRebalancerClassName(rebalancerClassName);
     _gSetupTool.getClusterManagementTool().setResourceIdealState(clusterName, db, idealState);
     _gSetupTool.rebalanceStorageCluster(clusterName, db, replica);
     idealState = _gSetupTool.getClusterManagementTool().getResourceIdealState(clusterName, db);
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
index 3371c8b..7d05416 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.helix.BucketDataAccessor;
 import org.apache.helix.model.ResourceAssignment;
+import org.mockito.Mockito;
 
 /**
  * A mock up metadata store for unit test.
@@ -32,8 +33,8 @@ public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
   private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
   private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
 
-  MockAssignmentMetadataStore(BucketDataAccessor bucketDataAccessor, String clusterName) {
-    super(bucketDataAccessor, clusterName);
+  MockAssignmentMetadataStore() {
+    super(Mockito.mock(BucketDataAccessor.class), "");
   }
 
   public Map<String, ResourceAssignment> getBaseline() {
@@ -53,6 +54,10 @@ public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
     _persistBestPossibleAssignment = bestPossibleAssignment;
   }
 
+  public void close() {
+    // do nothing
+  }
+
   public void clearMetadataStore() {
     _persistBestPossibleAssignment.clear();
     _persistGlobalBaseline.clear();
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
index 922915f..ecd2af3 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
@@ -20,6 +20,7 @@ package org.apache.helix.controller.rebalancer.waged;
  */
 
 import java.util.Map;
+
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
 import org.apache.helix.InstanceType;
@@ -28,6 +29,7 @@ import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ResourceAssignment;
 import org.testng.Assert;
+import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
@@ -79,7 +81,15 @@ public class TestAssignmentMetadataStore extends ZkTestBase {
     _manager.connect();
 
     // create AssignmentMetadataStore
-    _store = new AssignmentMetadataStore(_manager);
+    _store = new AssignmentMetadataStore(_manager.getMetadataStoreConnectionString(),
+        _manager.getClusterName());
+  }
+
+  @AfterClass
+  public void afterClass() {
+    if (_store != null) {
+      _store.close();
+    }
   }
 
   /**
@@ -91,11 +101,7 @@ public class TestAssignmentMetadataStore extends ZkTestBase {
    */
   @Test
   public void testReadEmptyBaseline() {
-    try {
-      Map<String, ResourceAssignment> baseline = _store.getBaseline();
-      Assert.fail("Should fail because there shouldn't be any data.");
-    } catch (Exception e) {
-      // OK
-    }
+    Map<String, ResourceAssignment> baseline = _store.getBaseline();
+    Assert.assertTrue(baseline.isEmpty());
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
index d6fd99b..e7368be 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
@@ -26,7 +26,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-import org.apache.helix.BucketDataAccessor;
+
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
@@ -47,8 +47,9 @@ import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.mockito.Matchers.*;
-import static org.mockito.Mockito.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
 
 public class TestWagedRebalancer extends AbstractTestClusterModel {
   private Set<String> _instances;
@@ -63,9 +64,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     _algorithm = new MockRebalanceAlgorithm();
 
     // Initialize a mock assignment metadata store
-    BucketDataAccessor mockAccessor = Mockito.mock(BucketDataAccessor.class);
-    String clusterName = ""; // an empty string for testing purposes
-    _metadataStore = new MockAssignmentMetadataStore(mockAccessor, clusterName);
+    _metadataStore = new MockAssignmentMetadataStore();
   }
 
   @Override
@@ -181,9 +180,9 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         String resourceName = csEntry.getKey();
         CurrentState cs = csEntry.getValue();
         for (Map.Entry<String, String> partitionStateEntry : cs.getPartitionStateMap().entrySet()) {
-          currentStateOutput.setCurrentState(resourceName,
-              new Partition(partitionStateEntry.getKey()), instanceName,
-              partitionStateEntry.getValue());
+          currentStateOutput
+              .setCurrentState(resourceName, new Partition(partitionStateEntry.getKey()),
+                  instanceName, partitionStateEntry.getValue());
         }
       }
     }
@@ -216,7 +215,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         "DROPPED");
   }
 
-  @Test(dependsOnMethods = "testRebalance")
+  @Test(dependsOnMethods = "testRebalance", expectedExceptions = HelixRebalanceException.class, expectedExceptionsMessageRegExp = "Input contains invalid resource\\(s\\) that cannot be rebalanced by the WAGED rebalancer. \\[Resource1\\] Failure Type: INVALID_INPUT")
   public void testNonCompatibleConfiguration() throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
     WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
@@ -233,12 +232,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
               .forEach(partition -> resource.addPartition(partition));
           return resource;
         }));
-    Map<String, IdealState> newIdealStates =
-        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
-    Map<String, ResourceAssignment> algorithmResult = _algorithm.getRebalanceResult();
-    // The output shall not contains the nonCompatibleResource.
-    resourceMap.remove(nonCompatibleResourceName);
-    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+    rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
   }
 
   // TODO test with invalid capacity configuration which will fail the cluster model constructing.
@@ -283,7 +277,7 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
       Assert.assertEquals(ex.getFailureType(),
           HelixRebalanceException.Type.INVALID_REBALANCER_STATUS);
       Assert.assertEquals(ex.getMessage(),
-          "Failed to get the persisted assignment records. Failure Type: INVALID_REBALANCER_STATUS");
+          "Failed to get the current baseline assignment because of unexpected error. Failure Type: INVALID_REBALANCER_STATUS");
     }
   }
 
@@ -425,8 +419,9 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
       Assert.assertTrue(newIdealStates.containsKey(resourceName));
       IdealState is = newIdealStates.get(resourceName);
       ResourceAssignment assignment = expectedResult.get(resourceName);
-      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(assignment.getMappedPartitions()
-          .stream().map(partition -> partition.getPartitionName()).collect(Collectors.toSet())));
+      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(
+          assignment.getMappedPartitions().stream().map(partition -> partition.getPartitionName())
+              .collect(Collectors.toSet())));
       for (String partitionName : is.getPartitionSet()) {
         Assert.assertEquals(is.getInstanceStateMap(partitionName),
             assignment.getReplicaMap(new Partition(partitionName)));
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java
index 2a39482..759c685 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java
@@ -72,7 +72,7 @@ public class MockRebalanceAlgorithm implements RebalanceAlgorithm {
 
     _resultHistory = result;
 
-    // TODO remove this mockito when OptimalAssignment.getOptimalResourceAssignment is ready.
+    // Mock the return value for supporting test.
     OptimalAssignment optimalAssignment = Mockito.mock(OptimalAssignment.class);
     when(optimalAssignment.getOptimalResourceAssignment()).thenReturn(result);
     return optimalAssignment;
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
index 0f799b3..91db076 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -103,8 +103,8 @@ public abstract class AbstractTestClusterModel {
     ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
     testClusterConfig.setMaxPartitionsPerInstance(5);
     testClusterConfig.setDisabledInstances(Collections.emptyMap());
-    testClusterConfig.setTopologyAwareEnabled(false);
     testClusterConfig.setInstanceCapacityKeys(new ArrayList<>(_capacityDataMap.keySet()));
+    testClusterConfig.setTopologyAwareEnabled(true);
     when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
 
     // 3. Mock the live instance node for the default instance.
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestOptimalAssignment.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestOptimalAssignment.java
new file mode 100644
index 0000000..bd820a9
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestOptimalAssignment.java
@@ -0,0 +1,91 @@
+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.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.helix.HelixException;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.ResourceAssignment;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestOptimalAssignment extends ClusterModelTestHelper {
+
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+  }
+
+  @Test
+  public void testUpdateAssignment() throws IOException {
+    OptimalAssignment assignment = new OptimalAssignment();
+
+    // update with empty cluster model
+    assignment.updateAssignments(getDefaultClusterModel());
+    Map<String, ResourceAssignment> optimalAssignmentMap =
+        assignment.getOptimalResourceAssignment();
+    Assert.assertEquals(optimalAssignmentMap, Collections.emptyMap());
+
+    // update with valid assignment
+    ClusterModel model = getDefaultClusterModel();
+    model.assign(_resourceNames.get(0), _partitionNames.get(1), "SLAVE", _testInstanceId);
+    model.assign(_resourceNames.get(0), _partitionNames.get(0), "MASTER", _testInstanceId);
+    assignment.updateAssignments(model);
+    optimalAssignmentMap = assignment.getOptimalResourceAssignment();
+    Assert.assertEquals(optimalAssignmentMap.get(_resourceNames.get(0)).getMappedPartitions(),
+        Arrays
+            .asList(new Partition(_partitionNames.get(0)), new Partition(_partitionNames.get(1))));
+    Assert.assertEquals(optimalAssignmentMap.get(_resourceNames.get(0))
+            .getReplicaMap(new Partition(_partitionNames.get(1))),
+        Collections.singletonMap(_testInstanceId, "SLAVE"));
+    Assert.assertEquals(optimalAssignmentMap.get(_resourceNames.get(0))
+            .getReplicaMap(new Partition(_partitionNames.get(0))),
+        Collections.singletonMap(_testInstanceId, "MASTER"));
+  }
+
+  @Test(dependsOnMethods = "testUpdateAssignment")
+  public void TestAssignmentFailure() throws IOException {
+    OptimalAssignment assignment = new OptimalAssignment();
+    ClusterModel model = getDefaultClusterModel();
+
+    // record failure
+    AssignableReplica targetFailureReplica =
+        model.getAssignableReplicaMap().get(_resourceNames.get(0)).iterator().next();
+    AssignableNode targetFailureNode = model.getAssignableNodes().get(_testInstanceId);
+    assignment.recordAssignmentFailure(targetFailureReplica, Collections
+        .singletonMap(targetFailureNode, Collections.singletonList("Assignment Failure!")));
+
+    Assert.assertTrue(assignment.hasAnyFailure());
+
+    assignment.updateAssignments(getDefaultClusterModel());
+    try {
+      assignment.getOptimalResourceAssignment();
+      Assert.fail("Get optimal assignment shall fail because of the failure record.");
+    } catch (HelixException ex) {
+      Assert.assertTrue(ex.getMessage().startsWith(
+          "Cannot get the optimal resource assignment since a calculation failure is recorded."));
+    }
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java
new file mode 100644
index 0000000..fb5375c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java
@@ -0,0 +1,477 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.TestHelper;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
+import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.tools.ClusterVerifiers.HelixClusterVerifier;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestWagedRebalance extends ZkTestBase {
+  protected final int NUM_NODE = 6;
+  protected static final int START_PORT = 12918;
+  protected static final int PARTITIONS = 20;
+  protected static final int TAGS = 2;
+
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  protected ClusterControllerManager _controller;
+
+  List<MockParticipantManager> _participants = new ArrayList<>();
+  Map<String, String> _nodeToTagMap = new HashMap<>();
+  List<String> _nodes = new ArrayList<>();
+  private Set<String> _allDBs = new HashSet<>();
+  private int _replica = 3;
+
+  private static String[] _testModels = {
+      BuiltInStateModelDefinitions.OnlineOffline.name(),
+      BuiltInStateModelDefinitions.MasterSlave.name(),
+      BuiltInStateModelDefinitions.LeaderStandby.name()
+  };
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      addInstanceConfig(storageNodeName, i, TAGS);
+    }
+
+    // start dummy participants
+    for (String node : _nodes) {
+      MockParticipantManager participant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, node);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+  }
+
+  protected void addInstanceConfig(String storageNodeName, int seqNo, int tagCount) {
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    String tag = "tag-" + seqNo % tagCount;
+    _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
+    _nodeToTagMap.put(storageNodeName, tag);
+    _nodes.add(storageNodeName);
+  }
+
+  @Test
+  public void test() throws Exception {
+    int i = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
+          -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+
+    validate(_replica);
+
+    // Adding 3 more resources
+    i = 0;
+    for (String stateModel : _testModels) {
+      String moreDB = "More-Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, moreDB, stateModel, PARTITIONS, _replica,
+          _replica, -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, moreDB, _replica);
+      _allDBs.add(moreDB);
+
+      Thread.sleep(300);
+
+      validate(_replica);
+    }
+
+    // Drop the 3 additional resources
+    for (int j = 0; j < 3; j++) {
+      String moreDB = "More-Test-DB-" + j++;
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, moreDB);
+      _allDBs.remove(moreDB);
+
+      Thread.sleep(300);
+
+      validate(_replica);
+    }
+  }
+
+  @Test(dependsOnMethods = "test")
+  public void testWithInstanceTag() throws Exception {
+    Set<String> tags = new HashSet<String>(_nodeToTagMap.values());
+    int i = 3;
+    for (String tag : tags) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db,
+          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+      IdealState is =
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      is.setInstanceGroupTag(tag);
+      _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = "test")
+  public void testChangeIdealState() throws InterruptedException {
+    String dbName = "Test-DB";
+    createResourceWithWagedRebalance(CLUSTER_NAME, dbName,
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
+    _allDBs.add(dbName);
+    Thread.sleep(300);
+
+    validate(_replica);
+
+    // Adjust the replica count
+    IdealState is =
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+    int newReplicaFactor = _replica - 1;
+    is.setReplicas("" + newReplicaFactor);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
+    Thread.sleep(300);
+
+    validate(newReplicaFactor);
+
+    // Adjust the partition list
+    is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, dbName);
+    is.setNumPartitions(PARTITIONS + 1);
+    _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, dbName, is);
+    _gSetupTool.getClusterManagementTool().rebalance(CLUSTER_NAME, dbName, newReplicaFactor);
+    Thread.sleep(300);
+
+    validate(newReplicaFactor);
+    ExternalView ev =
+        _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, dbName);
+    Assert.assertEquals(ev.getPartitionSet().size(), PARTITIONS + 1);
+  }
+
+  @Test(dependsOnMethods = "test")
+  public void testDisableInstance() throws InterruptedException {
+    String dbName = "Test-DB";
+    createResourceWithWagedRebalance(CLUSTER_NAME, dbName,
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
+    _allDBs.add(dbName);
+    Thread.sleep(300);
+
+    validate(_replica);
+
+    // Disable participants, keep only three left
+    Set<String> disableParticipants = new HashSet<>();
+
+    try {
+      for (int i = 3; i < _participants.size(); i++) {
+        MockParticipantManager p = _participants.get(i);
+        disableParticipants.add(p.getInstanceName());
+        InstanceConfig config = _gSetupTool.getClusterManagementTool()
+            .getInstanceConfig(CLUSTER_NAME, p.getInstanceName());
+        config.setInstanceEnabled(false);
+        _gSetupTool.getClusterManagementTool()
+            .setInstanceConfig(CLUSTER_NAME, p.getInstanceName(), config);
+      }
+      Thread.sleep(300);
+
+      validate(_replica);
+
+      // Verify there is no assignment on the disabled participants.
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, dbName);
+      for (String partition : ev.getPartitionSet()) {
+        Map<String, String> replicaStateMap = ev.getStateMap(partition);
+        for (String instance : replicaStateMap.keySet()) {
+          Assert.assertFalse(disableParticipants.contains(instance));
+        }
+      }
+    } finally {
+      // recover the config
+      for (String instanceName : disableParticipants) {
+        InstanceConfig config =
+            _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instanceName);
+        config.setInstanceEnabled(true);
+        _gSetupTool.getClusterManagementTool()
+            .setInstanceConfig(CLUSTER_NAME, instanceName, config);
+      }
+    }
+  }
+
+  @Test(dependsOnMethods = "testDisableInstance")
+  public void testLackEnoughLiveInstances() throws Exception {
+    // shutdown participants, keep only two left
+    for (int i = 2; i < _participants.size(); i++) {
+      _participants.get(i).syncStop();
+    }
+
+    int j = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + j++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
+          -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(2);
+
+    // restart the participants within the zone
+    for (int i = 2; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      MockParticipantManager newNode =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, p.getInstanceName());
+      _participants.set(i, newNode);
+      newNode.syncStart();
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = "testDisableInstance")
+  public void testLackEnoughInstances() throws Exception {
+    // shutdown participants, keep only two left
+    for (int i = 2; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      p.syncStop();
+      _gSetupTool.getClusterManagementTool()
+          .enableInstance(CLUSTER_NAME, p.getInstanceName(), false);
+      _gSetupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
+
+    }
+
+    int j = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + j++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
+          -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(2);
+
+    // Create new participants within the zone
+    for (int i = 2; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      String replaceNodeName = p.getInstanceName() + "-replacement_" + START_PORT;
+      addInstanceConfig(replaceNodeName, i, TAGS);
+      MockParticipantManager newNode =
+          new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, replaceNodeName);
+      _participants.set(i, newNode);
+      newNode.syncStart();
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = "test")
+  public void testMixedRebalancerUsage() throws InterruptedException {
+    int i = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + i++;
+      if (i == 0) {
+        _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, PARTITIONS, stateModel,
+            IdealState.RebalanceMode.FULL_AUTO + "", CrushRebalanceStrategy.class.getName());
+      } else if (i == 1) {
+        _gSetupTool.addResourceToCluster(CLUSTER_NAME, db, PARTITIONS, stateModel,
+            IdealState.RebalanceMode.FULL_AUTO + "", CrushEdRebalanceStrategy.class.getName());
+      } else {
+        createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+            _replica, -1);
+      }
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = "test")
+  public void testMaxPartitionLimitation() throws Exception {
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+    // Change the cluster level config so no assignment can be done
+    clusterConfig.setMaxPartitionsPerInstance(1);
+    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+    try {
+      String limitedResourceName = null;
+      int i = 0;
+      for (String stateModel : _testModels) {
+        String db = "Test-DB-" + i++;
+        createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+            _replica, -1);
+        if (i == 1) {
+          // The limited resource has additional limitation, so even the other resources can be assigned
+          // later, this resource will still be blocked by the max partition limitation.
+          limitedResourceName = db;
+          IdealState idealState =
+              _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+          idealState.setMaxPartitionsPerInstance(1);
+          _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, idealState);
+        }
+        _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+        _allDBs.add(db);
+      }
+      Thread.sleep(300);
+
+      // Since the WAGED rebalancer does not do partial rebalance, the initial assignment won't show.
+      Assert.assertFalse(TestHelper.verify(() -> _allDBs.stream().allMatch(db -> {
+        ExternalView ev =
+            _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+        return ev != null && !ev.getPartitionSet().isEmpty();
+      }), 2000));
+
+      // Remove the cluster level limitation
+      clusterConfig.setMaxPartitionsPerInstance(-1);
+      configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+      Thread.sleep(300);
+
+      // wait until any of the resources is rebalanced
+      TestHelper.verify(() -> {
+        for (String db : _allDBs) {
+          ExternalView ev =
+              _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+          if (ev != null && !ev.getPartitionSet().isEmpty()) {
+            return true;
+          }
+        }
+        return false;
+      }, 3000);
+      ExternalView ev = _gSetupTool.getClusterManagementTool()
+          .getResourceExternalView(CLUSTER_NAME, limitedResourceName);
+      Assert.assertFalse(ev != null && !ev.getPartitionSet().isEmpty());
+
+      // Remove the resource level limitation
+      IdealState idealState = _gSetupTool.getClusterManagementTool()
+          .getResourceIdealState(CLUSTER_NAME, limitedResourceName);
+      idealState.setMaxPartitionsPerInstance(Integer.MAX_VALUE);
+      _gSetupTool.getClusterManagementTool()
+          .setResourceIdealState(CLUSTER_NAME, limitedResourceName, idealState);
+
+      validate(_replica);
+    } finally {
+      // recover the config change
+      clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+      clusterConfig.setMaxPartitionsPerInstance(-1);
+      configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+    }
+  }
+
+  private void validate(int expectedReplica) {
+    HelixClusterVerifier _clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+            .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verify(5000));
+    for (String db : _allDBs) {
+      IdealState is =
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      validateIsolation(is, ev, expectedReplica);
+    }
+  }
+
+  /**
+   * Validate each partition is different instances and with necessary tagged instances.
+   */
+  private void validateIsolation(IdealState is, ExternalView ev, int expectedReplica) {
+    String tag = is.getInstanceGroupTag();
+    for (String partition : is.getPartitionSet()) {
+      Map<String, String> assignmentMap = ev.getRecord().getMapField(partition);
+      Set<String> instancesInEV = assignmentMap.keySet();
+      Assert.assertEquals(instancesInEV.size(), expectedReplica);
+      for (String instance : instancesInEV) {
+        if (tag != null) {
+          InstanceConfig config =
+              _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
+          Assert.assertTrue(config.containsTag(tag));
+        }
+      }
+    }
+  }
+
+  @AfterMethod
+  public void afterMethod() throws Exception {
+    for (String db : _allDBs) {
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+    }
+    _allDBs.clear();
+    // waiting for all DB be dropped.
+    Thread.sleep(100);
+    ZkHelixClusterVerifier _clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+            .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
+    for (MockParticipantManager p : _participants) {
+      if (p != null && p.isConnected()) {
+        p.syncStop();
+      }
+    }
+    deleteCluster(CLUSTER_NAME);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java
new file mode 100644
index 0000000..0b020db
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java
@@ -0,0 +1,374 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestWagedRebalanceFaultZone extends ZkTestBase {
+  protected final int NUM_NODE = 6;
+  protected static final int START_PORT = 12918;
+  protected static final int PARTITIONS = 20;
+  protected static final int ZONES = 3;
+  protected static final int TAGS = 2;
+
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+  protected ClusterControllerManager _controller;
+
+  List<MockParticipantManager> _participants = new ArrayList<>();
+  Map<String, String> _nodeToZoneMap = new HashMap<>();
+  Map<String, String> _nodeToTagMap = new HashMap<>();
+  List<String> _nodes = new ArrayList<>();
+  Set<String> _allDBs = new HashSet<>();
+  int _replica = 3;
+
+  String[] _testModels = {
+      BuiltInStateModelDefinitions.OnlineOffline.name(),
+      BuiltInStateModelDefinitions.MasterSlave.name(),
+      BuiltInStateModelDefinitions.LeaderStandby.name()
+  };
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    for (int i = 0; i < NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      addInstanceConfig(storageNodeName, i, ZONES, TAGS);
+    }
+
+    // start dummy participants
+    for (String node : _nodes) {
+      MockParticipantManager participant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, node);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+    enableTopologyAwareRebalance(_gZkClient, CLUSTER_NAME, true);
+  }
+
+  protected void addInstanceConfig(String storageNodeName, int seqNo, int zoneCount, int tagCount) {
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    String zone = "zone-" + seqNo % zoneCount;
+    String tag = "tag-" + seqNo % tagCount;
+    _gSetupTool.getClusterManagementTool().setInstanceZoneId(CLUSTER_NAME, storageNodeName, zone);
+    _gSetupTool.getClusterManagementTool().addInstanceTag(CLUSTER_NAME, storageNodeName, tag);
+    _nodeToZoneMap.put(storageNodeName, zone);
+    _nodeToTagMap.put(storageNodeName, tag);
+    _nodes.add(storageNodeName);
+  }
+
+  @Test
+  public void testZoneIsolation() throws Exception {
+    int i = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica, -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+
+    validate(_replica);
+  }
+
+  @Test
+  public void testZoneIsolationWithInstanceTag() throws Exception {
+    Set<String> tags = new HashSet<String>(_nodeToTagMap.values());
+    int i = 0;
+    for (String tag : tags) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db,
+          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+      IdealState is =
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      is.setInstanceGroupTag(tag);
+      _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, db, is);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = { "testZoneIsolation", "testZoneIsolationWithInstanceTag" })
+  public void testLackEnoughLiveRacks() throws Exception {
+    // shutdown participants within one zone
+    String zone = _nodeToZoneMap.values().iterator().next();
+    for (int i = 0; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      if (_nodeToZoneMap.get(p.getInstanceName()).equals(zone)) {
+        p.syncStop();
+      }
+    }
+
+    int j = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + j++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica, -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+    validate(2);
+
+    // restart the participants within the zone
+    for (int i = 0; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      if (_nodeToZoneMap.get(p.getInstanceName()).equals(zone)) {
+        MockParticipantManager newNode =
+            new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, p.getInstanceName());
+        _participants.set(i, newNode);
+        newNode.syncStart();
+      }
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = { "testLackEnoughLiveRacks" })
+  public void testLackEnoughRacks() throws Exception {
+    // shutdown participants within one zone
+    String zone = _nodeToZoneMap.values().iterator().next();
+    for (int i = 0; i < _participants.size(); i++) {
+      MockParticipantManager p = _participants.get(i);
+      if (_nodeToZoneMap.get(p.getInstanceName()).equals(zone)) {
+        p.syncStop();
+        _gSetupTool.getClusterManagementTool()
+            .enableInstance(CLUSTER_NAME, p.getInstanceName(), false);
+        Thread.sleep(50);
+        _gSetupTool.dropInstanceFromCluster(CLUSTER_NAME, p.getInstanceName());
+      }
+    }
+
+    int j = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + j++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica, -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+    validate(2);
+
+    // Create new participants within the zone
+    int nodeCount = _participants.size();
+    for (int i = 0; i < nodeCount; i++) {
+      MockParticipantManager p = _participants.get(i);
+      if (_nodeToZoneMap.get(p.getInstanceName()).equals(zone)) {
+        String replaceNodeName = p.getInstanceName() + "-replacement_" + START_PORT;
+        addInstanceConfig(replaceNodeName, i, ZONES, TAGS);
+        MockParticipantManager newNode =
+            new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, replaceNodeName);
+        _participants.set(i, newNode);
+        newNode.syncStart();
+      }
+    }
+
+    Thread.sleep(300);
+    // Verify if the partitions get assigned
+    validate(_replica);
+  }
+
+  @Test(dependsOnMethods = { "testZoneIsolation", "testZoneIsolationWithInstanceTag" })
+  public void testAddZone() throws Exception {
+    int i = 0;
+    for (String stateModel : _testModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica, -1);
+      _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
+      _allDBs.add(db);
+    }
+    Thread.sleep(300);
+
+    validate(_replica);
+
+    // Create new participants within the a new zone
+    Set<MockParticipantManager> newNodes = new HashSet<>();
+    Map<String, Integer> newNodeReplicaCount = new HashMap<>();
+
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+
+    try {
+      // Configure the preference so as to allow movements.
+      ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+      preference.put(ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS, 10);
+      preference.put(ClusterConfig.GlobalRebalancePreferenceKey.LESS_MOVEMENT, 0);
+      clusterConfig.setGlobalRebalancePreference(preference);
+      configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+      int nodeCount = 2;
+      for (int j = 0; j < nodeCount; j++) {
+        String newNodeName = "new-zone-node-" + j + "_" + START_PORT;
+        // Add all new node to the new zone
+        addInstanceConfig(newNodeName, j, ZONES + 1, TAGS);
+        MockParticipantManager newNode =
+            new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, newNodeName);
+        newNode.syncStart();
+        newNodes.add(newNode);
+        newNodeReplicaCount.put(newNodeName, 0);
+      }
+      Thread.sleep(300);
+
+      validate(_replica);
+
+      // The new zone nodes shall have some assignments
+      for (String db : _allDBs) {
+        IdealState is =
+            _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+        ExternalView ev =
+            _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+        validateZoneAndTagIsolation(is, ev, _replica);
+        for (String partition : ev.getPartitionSet()) {
+          Map<String, String> stateMap = ev.getStateMap(partition);
+          for (String node : stateMap.keySet()) {
+            if (newNodeReplicaCount.containsKey(node)) {
+              newNodeReplicaCount.computeIfPresent(node, (nodeName, replicaCount) -> replicaCount + 1);
+            }
+          }
+        }
+      }
+      Assert.assertTrue(newNodeReplicaCount.values().stream().allMatch(count -> count > 0));
+    } finally {
+      // Revert the preference
+      ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+      Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+      preference.put(ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS, 1);
+      preference.put(ClusterConfig.GlobalRebalancePreferenceKey.LESS_MOVEMENT, 1);
+      clusterConfig.setGlobalRebalancePreference(preference);
+      configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+      // Stop the new nodes
+      for (MockParticipantManager p : newNodes) {
+        if (p != null && p.isConnected()) {
+          p.syncStop();
+        }
+      }
+    }
+  }
+
+  private void validate(int expectedReplica) {
+    ZkHelixClusterVerifier _clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+            .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+
+    for (String db : _allDBs) {
+      IdealState is =
+          _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      validateZoneAndTagIsolation(is, ev, expectedReplica);
+    }
+  }
+
+  /**
+   * Validate instances for each partition is on different zone and with necessary tagged instances.
+   */
+  private void validateZoneAndTagIsolation(IdealState is, ExternalView ev, int expectedReplica) {
+    String tag = is.getInstanceGroupTag();
+    for (String partition : is.getPartitionSet()) {
+      Set<String> assignedZones = new HashSet<String>();
+
+      Map<String, String> assignmentMap = ev.getRecord().getMapField(partition);
+      Set<String> instancesInEV = assignmentMap.keySet();
+      // TODO: preference List is not persisted in IS.
+      // Assert.assertEquals(instancesInEV, instancesInIs);
+      for (String instance : instancesInEV) {
+        assignedZones.add(_nodeToZoneMap.get(instance));
+        if (tag != null) {
+          InstanceConfig config =
+              _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, instance);
+          Assert.assertTrue(config.containsTag(tag));
+        }
+      }
+      Assert.assertEquals(assignedZones.size(), expectedReplica);
+    }
+  }
+
+  @AfterMethod
+  public void afterMethod() throws Exception {
+    for (String db : _allDBs) {
+      _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
+    }
+    _allDBs.clear();
+    // waiting for all DB be dropped.
+    Thread.sleep(100);
+    ZkHelixClusterVerifier _clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+            .setResources(_allDBs).build();
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+  }
+
+  @AfterClass
+  public void afterClass() throws Exception {
+    /*
+     * shutdown order: 1) disconnect the controller 2) disconnect participants
+     */
+    if (_controller != null && _controller.isConnected()) {
+      _controller.syncStop();
+    }
+    for (MockParticipantManager p : _participants) {
+      if (p != null && p.isConnected()) {
+        p.syncStop();
+      }
+    }
+    deleteCluster(CLUSTER_NAME);
+    System.out.println("END " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceTopologyAware.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceTopologyAware.java
new file mode 100644
index 0000000..412fc8c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceTopologyAware.java
@@ -0,0 +1,114 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.Date;
+
+import org.apache.helix.ConfigAccessor;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestWagedRebalanceTopologyAware extends TestWagedRebalanceFaultZone {
+  private static final String TOLOPOGY_DEF = "/DOMAIN/ZONE/INSTANCE";
+  private static final String DOMAIN_NAME = "Domain";
+  private static final String FAULT_ZONE = "ZONE";
+
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
+
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+
+    ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+    ClusterConfig clusterConfig = configAccessor.getClusterConfig(CLUSTER_NAME);
+    clusterConfig.setTopology(TOLOPOGY_DEF);
+    clusterConfig.setFaultZoneType(FAULT_ZONE);
+    configAccessor.setClusterConfig(CLUSTER_NAME, clusterConfig);
+
+    for (int i = 0; i < NUM_NODE; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      addInstanceConfig(storageNodeName, i, ZONES, TAGS);
+    }
+
+    // start dummy participants
+    for (String node : _nodes) {
+      MockParticipantManager participant = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, node);
+      participant.syncStart();
+      _participants.add(participant);
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+    enableTopologyAwareRebalance(_gZkClient, CLUSTER_NAME, true);
+  }
+
+  protected void addInstanceConfig(String storageNodeName, int seqNo, int zoneCount, int tagCount) {
+    _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    String zone = "zone-" + seqNo % zoneCount;
+    String tag = "tag-" + seqNo % tagCount;
+
+    InstanceConfig config =
+        _gSetupTool.getClusterManagementTool().getInstanceConfig(CLUSTER_NAME, storageNodeName);
+    config.setDomain(
+        String.format("DOMAIN=%s,ZONE=%s,INSTANCE=%s", DOMAIN_NAME, zone, storageNodeName));
+    config.addTag(tag);
+    _gSetupTool.getClusterManagementTool().setInstanceConfig(CLUSTER_NAME, storageNodeName, config);
+
+    _nodeToZoneMap.put(storageNodeName, zone);
+    _nodeToTagMap.put(storageNodeName, tag);
+    _nodes.add(storageNodeName);
+  }
+
+  @Test
+  public void testZoneIsolation() throws Exception {
+    super.testZoneIsolation();
+  }
+
+  @Test
+  public void testZoneIsolationWithInstanceTag() throws Exception {
+    super.testZoneIsolationWithInstanceTag();
+  }
+
+  @Test(dependsOnMethods = { "testZoneIsolation", "testZoneIsolationWithInstanceTag" })
+  public void testLackEnoughLiveRacks() throws Exception {
+    super.testLackEnoughLiveRacks();
+  }
+
+  @Test(dependsOnMethods = { "testLackEnoughLiveRacks" })
+  public void testLackEnoughRacks() throws Exception {
+    super.testLackEnoughRacks();
+  }
+
+  @Test(dependsOnMethods = { "testZoneIsolation", "testZoneIsolationWithInstanceTag" })
+  public void testAddZone() throws Exception {
+    super.testAddZone();
+  }
+}


[helix] 21/37: Implement AssignmentMetadataStore (#453)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3d2da32bddc7798a502ae170873f0fb1abe8cbd5
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Mon Sep 9 16:40:24 2019 -0700

    Implement AssignmentMetadataStore (#453)
    
    Implement AssignmentMetadataStore
    
    AssignmentMetadataStore is a component for the new WAGED Rebalaner. It provides APIs that allows the rebalancer to read and write the baseline and best possible assignments using BucketDataAccessor.
    
    Changelist:
    1. Add AssignmentMetadataStore
    2. Add an integration test: TestAssignmentMetadataStore
---
 .../rebalancer/waged/AssignmentMetadataStore.java  | 112 +++++++++++++++++++--
 .../rebalancer/waged/WagedRebalancer.java          |  64 ++++++------
 .../manager/zk/ZNRecordJacksonSerializer.java      |   4 +-
 .../waged/MockAssignmentMetadataStore.java         |   9 +-
 .../waged/TestAssignmentMetadataStore.java         | 101 +++++++++++++++++++
 5 files changed, 244 insertions(+), 46 deletions(-)

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
index cc52dac..bf9f292 100644
--- 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
@@ -19,6 +19,16 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import java.io.IOException;
+import java.util.Arrays;
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.helix.BucketDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.ZNRecordJacksonSerializer;
+import org.apache.helix.manager.zk.ZkBucketDataAccessor;
 import org.apache.helix.model.ResourceAssignment;
 
 import java.util.HashMap;
@@ -28,24 +38,106 @@ import java.util.Map;
  * A placeholder before we have the real assignment metadata store.
  */
 public class AssignmentMetadataStore {
-  private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
-  private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
+  private static final String ASSIGNMENT_METADATA_KEY = "ASSIGNMENT_METADATA";
+  private static final String BASELINE_TEMPLATE = "/%s/%s/BASELINE";
+  private static final String BEST_POSSIBLE_TEMPLATE = "/%s/%s/BEST_POSSIBLE";
+  private static final String BASELINE_KEY = "BASELINE";
+  private static final String BEST_POSSIBLE_KEY = "BEST_POSSIBLE";
+  private static final ZkSerializer SERIALIZER = new ZNRecordJacksonSerializer();
+
+  private BucketDataAccessor _dataAccessor;
+  private String _baselinePath;
+  private String _bestPossiblePath;
+  private Map<String, ResourceAssignment> _globalBaseline;
+  private Map<String, ResourceAssignment> _bestPossibleAssignment;
+
+  AssignmentMetadataStore(HelixManager helixManager) {
+    _dataAccessor = new ZkBucketDataAccessor(helixManager.getMetadataStoreConnectionString());
+    _baselinePath =
+        String.format(BASELINE_TEMPLATE, helixManager.getClusterName(), ASSIGNMENT_METADATA_KEY);
+    _bestPossiblePath = String.format(BEST_POSSIBLE_TEMPLATE, helixManager.getClusterName(),
+        ASSIGNMENT_METADATA_KEY);
+  }
 
   public Map<String, ResourceAssignment> getBaseline() {
-    return _persistGlobalBaseline;
+    // Return the in-memory baseline. If null, read from ZK. This is to minimize reads from ZK
+    if (_globalBaseline == null) {
+      HelixProperty baseline =
+          _dataAccessor.compressedBucketRead(_baselinePath, HelixProperty.class);
+      _globalBaseline = splitAssignments(baseline);
+    }
+    return _globalBaseline;
+  }
+
+  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
+    // Return the in-memory baseline. If null, read from ZK. This is to minimize reads from ZK
+    if (_bestPossibleAssignment == null) {
+      HelixProperty baseline =
+          _dataAccessor.compressedBucketRead(_bestPossiblePath, HelixProperty.class);
+      _bestPossibleAssignment = splitAssignments(baseline);
+    }
+    return _bestPossibleAssignment;
   }
 
   public void persistBaseline(Map<String, ResourceAssignment> globalBaseline) {
-    // TODO clean up invalid items
-    _persistGlobalBaseline = globalBaseline;
+    // TODO: Make the write async?
+    // Persist to ZK
+    HelixProperty combinedAssignments = combineAssignments(BASELINE_KEY, globalBaseline);
+    try {
+      _dataAccessor.compressedBucketWrite(_baselinePath, combinedAssignments);
+    } catch (IOException e) {
+      // TODO: Improve failure handling
+      throw new HelixException("Failed to persist baseline!", e);
+    }
+
+    // Update the in-memory reference
+    _globalBaseline = globalBaseline;
   }
 
-  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
-    return _persistBestPossibleAssignment;
+  public void persistBestPossibleAssignment(
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
+    // TODO: Make the write async?
+    // Persist to ZK asynchronously
+    HelixProperty combinedAssignments =
+        combineAssignments(BEST_POSSIBLE_KEY, bestPossibleAssignment);
+    try {
+      _dataAccessor.compressedBucketWrite(_bestPossiblePath, combinedAssignments);
+    } catch (IOException e) {
+      // TODO: Improve failure handling
+      throw new HelixException("Failed to persist baseline!", e);
+    }
+
+    // Update the in-memory reference
+    _bestPossibleAssignment = bestPossibleAssignment;
+  }
+
+  /**
+   * Produces one HelixProperty that contains all assignment data.
+   * @param name
+   * @param assignmentMap
+   * @return
+   */
+  private HelixProperty combineAssignments(String name,
+      Map<String, ResourceAssignment> assignmentMap) {
+    HelixProperty property = new HelixProperty(name);
+    // Add each resource's assignment as a simple field in one ZNRecord
+    assignmentMap.forEach((resource, assignment) -> property.getRecord().setSimpleField(resource,
+        Arrays.toString(SERIALIZER.serialize(assignment.getRecord()))));
+    return property;
   }
 
-  public void persistBestPossibleAssignment(Map<String, ResourceAssignment> bestPossibleAssignment) {
-    // TODO clean up invalid items
-    _persistBestPossibleAssignment.putAll(bestPossibleAssignment);
+  /**
+   * Returns a Map of (ResourceName, ResourceAssignment) pairs.
+   * @param property
+   * @return
+   */
+  private Map<String, ResourceAssignment> splitAssignments(HelixProperty property) {
+    Map<String, ResourceAssignment> assignmentMap = new HashMap<>();
+    // Convert each resource's assignment String into a ResourceAssignment object and put it in a
+    // map
+    property.getRecord().getSimpleFields()
+        .forEach((resource, assignment) -> assignmentMap.put(resource,
+            new ResourceAssignment((ZNRecord) SERIALIZER.deserialize(assignment.getBytes()))));
+    return assignmentMap;
   }
 }
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
index 866c7c9..22cac7e 100644
--- 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
@@ -51,10 +51,10 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Weight-Aware Globally-Even Distribute Rebalancer.
- *
- * @see <a href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
- * Design Document
- * </a>
+ * @see <a
+ *      href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
+ *      Design Document
+ *      </a>
  */
 public class WagedRebalancer {
   private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
@@ -62,8 +62,8 @@ public class WagedRebalancer {
   // When any of the following change happens, the rebalancer needs to do a global rebalance which
   // contains 1. baseline recalculate, 2. partial rebalance that is based on the new baseline.
   private static final Set<HelixConstants.ChangeType> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES =
-      Collections.unmodifiableSet(new HashSet<>(Arrays
-          .asList(HelixConstants.ChangeType.RESOURCE_CONFIG,
+      Collections
+          .unmodifiableSet(new HashSet<>(Arrays.asList(HelixConstants.ChangeType.RESOURCE_CONFIG,
               HelixConstants.ChangeType.CLUSTER_CONFIG,
               HelixConstants.ChangeType.INSTANCE_CONFIG)));
   // The cluster change detector is a stateful object.
@@ -73,7 +73,8 @@ public class WagedRebalancer {
   private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
 
   // --------- The following fields are placeholders and need replacement. -----------//
-  // TODO Shall we make the metadata store a static threadlocal object as well to avoid reinitialization?
+  // TODO Shall we make the metadata store a static threadlocal object as well to avoid
+  // reinitialization?
   private final AssignmentMetadataStore _assignmentMetadataStore;
   private final RebalanceAlgorithm _rebalanceAlgorithm;
   // ------------------------------------------------------------------------------------//
@@ -81,8 +82,8 @@ public class WagedRebalancer {
   public WagedRebalancer(HelixManager helixManager) {
     this(
         // TODO init the metadata store according to their requirement when integrate,
-        //  or change to final static method if possible.
-        new AssignmentMetadataStore(),
+        // or change to final static method if possible.
+        new AssignmentMetadataStore(helixManager),
         // TODO parse the cluster setting
         ConstraintBasedAlgorithmFactory.getInstance(),
         // Use DelayedAutoRebalancer as the mapping calculator for the final assignment output.
@@ -103,14 +104,14 @@ public class WagedRebalancer {
   protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
       RebalanceAlgorithm algorithm) {
     this(assignmentMetadataStore, algorithm, new DelayedAutoRebalancer());
+
   }
 
   /**
    * Compute the new IdealStates for all the input resources. The IdealStates include both new
    * partition assignment (in the listFiles) and the new replica state mapping (in the mapFields).
-   *
-   * @param clusterData        The Cluster status data provider.
-   * @param resourceMap        A map containing all the rebalancing resources.
+   * @param clusterData The Cluster status data provider.
+   * @param resourceMap A map containing all the rebalancing resources.
    * @param currentStateOutput The present Current States of the resources.
    * @return A map of the new IdealStates with the resource name as key.
    */
@@ -124,8 +125,8 @@ public class WagedRebalancer {
       IdealState is = clusterData.getIdealState(resourceEntry.getKey());
       return is != null && is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
           && getClass().getName().equals(is.getRebalancerClassName());
-    }).collect(Collectors
-        .toMap(resourceEntry -> resourceEntry.getKey(), resourceEntry -> resourceEntry.getValue()));
+    }).collect(Collectors.toMap(resourceEntry -> resourceEntry.getKey(),
+        resourceEntry -> resourceEntry.getValue()));
 
     if (resourceMap.isEmpty()) {
       LOG.warn("There is no valid resource to be rebalanced by {}",
@@ -140,13 +141,13 @@ public class WagedRebalancer {
     Map<String, IdealState> newIdealStates = computeBestPossibleStates(clusterData, resourceMap);
 
     // Construct the new best possible states according to the current state and target assignment.
-    // Note that the new ideal state might be an intermediate state between the current state and the target assignment.
+    // Note that the new ideal state might be an intermediate state between the current state and
+    // the target assignment.
     for (IdealState is : newIdealStates.values()) {
       String resourceName = is.getResourceName();
       // Adjust the states according to the current state.
-      ResourceAssignment finalAssignment = _mappingCalculator
-          .computeBestPossiblePartitionState(clusterData, is, resourceMap.get(resourceName),
-              currentStateOutput);
+      ResourceAssignment finalAssignment = _mappingCalculator.computeBestPossiblePartitionState(
+          clusterData, is, resourceMap.get(resourceName), currentStateOutput);
 
       // Clean up the state mapping fields. Use the final assignment that is calculated by the
       // mapping calculator to replace them.
@@ -195,10 +196,10 @@ public class WagedRebalancer {
       IdealState newIdeaState;
       try {
         IdealState currentIdealState = clusterData.getIdealState(resourceName);
-        Map<String, Integer> statePriorityMap =
-            clusterData.getStateModelDef(currentIdealState.getStateModelDefRef())
-                .getStatePriorityMap();
-        // Create a new IdealState instance contains the new calculated assignment in the preference list.
+        Map<String, Integer> statePriorityMap = clusterData
+            .getStateModelDef(currentIdealState.getStateModelDefRef()).getStatePriorityMap();
+        // Create a new IdealState instance contains the new calculated assignment in the preference
+        // list.
         newIdeaState = generateIdealStateWithAssignment(resourceName, currentIdealState,
             newAssignment.get(resourceName), statePriorityMap);
       } catch (Exception ex) {
@@ -227,9 +228,8 @@ public class WagedRebalancer {
       throw new HelixRebalanceException("Failed to get the current baseline assignment.",
           HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
     }
-    Map<String, ResourceAssignment> baseline =
-        calculateAssignment(clusterData, clusterChanges, resourceMap, clusterData.getAllInstances(),
-            Collections.emptyMap(), currentBaseline);
+    Map<String, ResourceAssignment> baseline = calculateAssignment(clusterData, clusterChanges,
+        resourceMap, clusterData.getAllInstances(), Collections.emptyMap(), currentBaseline);
     try {
       _assignmentMetadataStore.persistBaseline(baseline);
     } catch (Exception ex) {
@@ -254,9 +254,8 @@ public class WagedRebalancer {
       throw new HelixRebalanceException("Failed to get the persisted assignment records.",
           HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
     }
-    Map<String, ResourceAssignment> newAssignment =
-        calculateAssignment(clusterData, clusterChanges, resourceMap, activeInstances, baseline,
-            prevBestPossibleAssignment);
+    Map<String, ResourceAssignment> newAssignment = calculateAssignment(clusterData, clusterChanges,
+        resourceMap, activeInstances, baseline, prevBestPossibleAssignment);
     try {
       // TODO Test to confirm if persisting the final assignment (with final partition states)
       // would be a better option.
@@ -271,13 +270,13 @@ public class WagedRebalancer {
 
   /**
    * Generate the cluster model based on the input and calculate the optimal assignment.
-   *
    * @param clusterData                the cluster data cache.
    * @param clusterChanges             the detected cluster changes.
    * @param resourceMap                the rebalancing resources.
    * @param activeNodes                the alive and enabled nodes.
    * @param baseline                   the baseline assignment for the algorithm as a reference.
-   * @param prevBestPossibleAssignment the previous best possible assignment for the algorithm as a reference.
+   * @param prevBestPossibleAssignment the previous best possible assignment for the algorithm as a
+   *                                   reference.
    * @return the new optimal assignment for the resources.
    */
   private Map<String, ResourceAssignment> calculateAssignment(
@@ -289,9 +288,8 @@ public class WagedRebalancer {
     LOG.info("Start calculating for an assignment");
     ClusterModel clusterModel;
     try {
-      clusterModel = ClusterModelProvider
-          .generateClusterModel(clusterData, resourceMap, activeNodes, clusterChanges, baseline,
-              prevBestPossibleAssignment);
+      clusterModel = ClusterModelProvider.generateClusterModel(clusterData, resourceMap,
+          activeNodes, clusterChanges, baseline, prevBestPossibleAssignment);
     } catch (Exception ex) {
       throw new HelixRebalanceException("Failed to generate cluster model.",
           HelixRebalanceException.Type.INVALID_CLUSTER_STATUS, ex);
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java
index 989017a..b375e80 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java
@@ -27,8 +27,8 @@ import org.apache.helix.ZNRecord;
 import org.codehaus.jackson.map.ObjectMapper;
 
 /**
- * ZNRecordJacksonSerializer serializes ZNRecord objects into a byte array using MessagePack's
- * serializer. Note that this serializer doesn't check for the size of the resulting binary.
+ * ZNRecordJacksonSerializer serializes ZNRecord objects into a byte array using Jackson. Note that
+ * this serializer doesn't check for the size of the resulting binary.
  */
 public class ZNRecordJacksonSerializer implements ZkSerializer {
   private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
index ea8c164..8b80f2d 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
@@ -19,6 +19,7 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import org.apache.helix.HelixManager;
 import org.apache.helix.model.ResourceAssignment;
 
 import java.util.HashMap;
@@ -32,6 +33,11 @@ public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
   private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
   private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
 
+  public MockAssignmentMetadataStore() {
+    // In-memory mock component, so pass null for HelixManager since it's not needed
+    super(null);
+  }
+
   public Map<String, ResourceAssignment> getBaseline() {
     return _persistGlobalBaseline;
   }
@@ -44,7 +50,8 @@ public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
     return _persistBestPossibleAssignment;
   }
 
-  public void persistBestPossibleAssignment(Map<String, ResourceAssignment> bestPossibleAssignment) {
+  public void persistBestPossibleAssignment(
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
     _persistBestPossibleAssignment = bestPossibleAssignment;
   }
 
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
new file mode 100644
index 0000000..922915f
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestAssignmentMetadataStore.java
@@ -0,0 +1,101 @@
+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 java.util.Map;
+import org.apache.helix.HelixManager;
+import org.apache.helix.HelixManagerFactory;
+import org.apache.helix.InstanceType;
+import org.apache.helix.common.ZkTestBase;
+import org.apache.helix.integration.manager.ClusterControllerManager;
+import org.apache.helix.integration.manager.MockParticipantManager;
+import org.apache.helix.model.ResourceAssignment;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestAssignmentMetadataStore extends ZkTestBase {
+  protected static final int NODE_NR = 5;
+  protected static final int START_PORT = 12918;
+  protected static final String STATE_MODEL = "MasterSlave";
+  protected static final String TEST_DB = "TestDB";
+  protected static final int _PARTITIONS = 20;
+
+  protected HelixManager _manager;
+  protected final String CLASS_NAME = getShortClassName();
+  protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+
+  protected MockParticipantManager[] _participants = new MockParticipantManager[NODE_NR];
+  protected ClusterControllerManager _controller;
+  protected int _replica = 3;
+
+  private AssignmentMetadataStore _store;
+
+  @BeforeClass
+  public void beforeClass() throws Exception {
+    super.beforeClass();
+
+    // setup storage cluster
+    _gSetupTool.addCluster(CLUSTER_NAME, true);
+    _gSetupTool.addResourceToCluster(CLUSTER_NAME, TEST_DB, _PARTITIONS, STATE_MODEL);
+    for (int i = 0; i < NODE_NR; i++) {
+      String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _gSetupTool.addInstanceToCluster(CLUSTER_NAME, storageNodeName);
+    }
+    _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, TEST_DB, _replica);
+
+    // start dummy participants
+    for (int i = 0; i < NODE_NR; i++) {
+      String instanceName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
+      _participants[i] = new MockParticipantManager(ZK_ADDR, CLUSTER_NAME, instanceName);
+      _participants[i].syncStart();
+    }
+
+    // start controller
+    String controllerName = CONTROLLER_PREFIX + "_0";
+    _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
+    _controller.syncStart();
+
+    // create cluster manager
+    _manager = HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, "Admin",
+        InstanceType.ADMINISTRATOR, ZK_ADDR);
+    _manager.connect();
+
+    // create AssignmentMetadataStore
+    _store = new AssignmentMetadataStore(_manager);
+  }
+
+  /**
+   * TODO: Reading baseline will be empty because AssignmentMetadataStore isn't being used yet by
+   * the new rebalancer. Modify this integration test once the WAGED rebalancer
+   * starts using AssignmentMetadataStore's persist APIs.
+   * TODO: WAGED Rebalancer currently does NOT work with ZKClusterVerifier because verifier's
+   * HelixManager is null, and that causes an NPE when instantiating AssignmentMetadataStore.
+   */
+  @Test
+  public void testReadEmptyBaseline() {
+    try {
+      Map<String, ResourceAssignment> baseline = _store.getBaseline();
+      Assert.fail("Should fail because there shouldn't be any data.");
+    } catch (Exception e) {
+      // OK
+    }
+  }
+}


[helix] 23/37: Implement one of the soft constraints (#450)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 06cb6928563da2213ad532e85b5f096e0a387c9f
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Tue Sep 10 15:37:15 2019 -0700

    Implement one of the soft constraints (#450)
    
    Implement Instance Partitions Count soft constraint.
    Evaluate by instance's current partition count versus estimated max partition count.
    Intuitively, Encourage the assignment if the instance's occupancy rate is below average;
    Discourage the assignment if the instance's occupancy rate is above average.
    
    The final normalized score will be within [0, 1].
    The implementation of the class will depend on the cluster current total partitions count as the max score.
---
 .../constraints/ConstraintBasedAlgorithm.java      |  2 +-
 .../InstancePartitionsCountConstraint.java         | 47 +++++++++++++++++
 .../constraints/LeastPartitionCountConstraint.java | 53 -------------------
 .../waged/constraints/SoftConstraint.java          | 40 ++++++++++-----
 .../constraints/SoftConstraintWeightModel.java     | 12 ++---
 .../rebalancer/waged/model/ClusterContext.java     | 11 ++--
 .../constraints/TestConstraintBasedAlgorithm.java  |  4 +-
 .../TestInstancePartitionsCountConstraint.java     | 60 ++++++++++++++++++++++
 .../TestSoftConstraintNormalizeFunction.java       | 47 +++++++++++++++++
 9 files changed, 196 insertions(+), 80 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
index 99d8d2a..479fb78 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
@@ -117,7 +117,7 @@ class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
     Function<AssignableNode, Float> calculatePoints =
         (candidateNode) -> _softConstraintsWeightModel.getSumOfScores(_softConstraints.stream()
             .collect(Collectors.toMap(Function.identity(), softConstraint -> softConstraint
-                .getAssignmentOriginScore(candidateNode, replica, clusterContext))));
+                .getAssignmentNormalizedScore(candidateNode, replica, clusterContext))));
 
     return candidateNodes.stream().max(Comparator.comparing(calculatePoints));
   }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
new file mode 100644
index 0000000..ca05cf8
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/InstancePartitionsCountConstraint.java
@@ -0,0 +1,47 @@
+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 by instance's current partition count versus estimated max partition count
+ * Intuitively, Encourage the assignment if the instance's occupancy rate is below average;
+ * Discourage the assignment if the instance's occupancy rate is above average
+ */
+class InstancePartitionsCountConstraint extends SoftConstraint {
+  private static final float MAX_SCORE = 1f;
+  private static final float MIN_SCORE = 0f;
+
+  InstancePartitionsCountConstraint() {
+    super(MAX_SCORE, MIN_SCORE);
+  }
+
+  @Override
+  protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    float doubleEstimatedMaxPartitionCount = 2 * clusterContext.getEstimatedMaxPartitionCount();
+    float currentPartitionCount = node.getAssignedReplicaCount();
+    return Math.max((doubleEstimatedMaxPartitionCount - currentPartitionCount)
+        / doubleEstimatedMaxPartitionCount, 0);
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java
deleted file mode 100644
index a8d36db..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/LeastPartitionCountConstraint.java
+++ /dev/null
@@ -1,53 +0,0 @@
-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 the proposed assignment according to the instance's partition count.
- */
-class LeastPartitionCountConstraint extends SoftConstraint {
-  static LeastPartitionCountConstraint INSTANCE = new LeastPartitionCountConstraint();
-
-  private LeastPartitionCountConstraint() {
-  }
-
-  /**
-   * Returns a score depending on the number of assignments on this node. The score is scaled evenly
-   * between the minScore and maxScore.
-   * When the node is idle, return with the maxScore.
-   * When the node usage reaches the estimated max partition, return with (minScore + maxScore ) /
-   * 2.
-   * When the node usage reaches 2 * estimated_max or more, return with the minScore.
-   * If the estimated max partition count is not set, it defaults to Integer.MAX_VALUE in
-   * clusterContext.
-   */
-  @Override
-  float getAssignmentOriginScore(AssignableNode node, AssignableReplica replica,
-      ClusterContext clusterContext) {
-      throw new UnsupportedOperationException("The POC implementation has a bug, will fix it as TODO");
-//    float doubleMaxPartitionCount = 2.0f * clusterContext.getEstimatedMaxPartitionCount();
-//    int curPartitionCount = node.getCurrentAssignmentCount();
-//    return Math.max((doubleMaxPartitionCount - curPartitionCount) / doubleMaxPartitionCount, 0);
-  }
-}
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
index db145fe..0f2bdbc 100644
--- 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
@@ -24,14 +24,15 @@ 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.
+ * The "soft" constraint evaluates the optimality of an assignment by giving it a score of a scale of [minScore, maxScore]
+ * The higher the score, the better the assignment; Intuitively, the assignment is encouraged.
+ * The lower score the score, the worse the assignment; Intuitively, the assignment is penalized.
  */
 abstract class SoftConstraint {
   private float _maxScore = 1000f;
   private float _minScore = -1000f;
 
-  interface ScalerFunction {
+  interface NormalizeFunction {
     /**
      * Scale the origin score to a normalized range (0, 1).
      * The purpose is to compare scores between different soft constraints.
@@ -57,23 +58,38 @@ abstract class SoftConstraint {
     _minScore = minScore;
   }
 
+  float getMaxScore() {
+    return _maxScore;
+  }
+
+  float getMinScore() {
+    return _minScore;
+  }
+
   /**
-   * 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.
-   * @return float value representing the score
+   * Evaluate and give a score for an potential assignment partition -> instance
+   * Child class only needs to care about how the score is implemented
+   * @return The score of the assignment in float value
    */
-  abstract float getAssignmentOriginScore(AssignableNode node, AssignableReplica replica,
+  protected abstract float getAssignmentScore(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext);
 
   /**
+   * Evaluate and give a score for an potential assignment partition -> instance
+   * It's the only exposed method to the caller
+   * @return The score is normalized to be within MinScore and MaxScore
+   */
+  float getAssignmentNormalizedScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    return getNormalizeFunction().scale(getAssignmentScore(node, replica, clusterContext));
+  }
+
+  /**
    * The default scaler function that squashes any score within (min_score, max_score) to (0, 1);
    * Child class could override the method and customize the method on its own
    * @return The MinMaxScaler instance by default
    */
-  ScalerFunction getScalerFunction() {
-    return (score) -> (score - _minScore) / (_maxScore - _minScore);
+  NormalizeFunction getNormalizeFunction() {
+    return (score) -> (score - getMinScore()) / (getMaxScore() - getMinScore());
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
index 41e4334..a961936 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
@@ -27,7 +27,7 @@ import com.google.common.collect.ImmutableMap;
  * The class retrieves the offline model that defines the relative importance of soft constraints.
  */
 class SoftConstraintWeightModel {
-  private static Map<? extends SoftConstraint, Float> MODEL;
+  private static Map<Class, Float> MODEL;
 
   // TODO either define the weights in property files or zookeeper node or static human input
   SoftConstraintWeightModel() {
@@ -35,8 +35,9 @@ class SoftConstraintWeightModel {
   }
 
   static {
-    MODEL = ImmutableMap.<SoftConstraint, Float> builder()
-        .put(LeastPartitionCountConstraint.INSTANCE, 1.0f).build();
+    //TODO update the weight
+    MODEL = ImmutableMap.<Class, Float> builder().put(InstancePartitionsCountConstraint.class, 1.0f)
+        .build();
   }
 
   /**
@@ -48,9 +49,8 @@ class SoftConstraintWeightModel {
     float sum = 0;
     for (Map.Entry<SoftConstraint, Float> softConstraintScoreEntry : originScoresMap.entrySet()) {
       SoftConstraint softConstraint = softConstraintScoreEntry.getKey();
-      float score = softConstraint.getScalerFunction().scale(softConstraintScoreEntry.getValue());
-      float weight = MODEL.get(softConstraint);
-      sum += score * weight;
+      float weight = MODEL.get(softConstraint.getClass());
+      sum += softConstraintScoreEntry.getValue() * weight;
     }
 
     return sum;
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
index c163e4c..a0c841a 100644
--- 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
@@ -19,8 +19,6 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
-import org.apache.helix.HelixException;
-
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -29,6 +27,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.helix.HelixException;
+
 /**
  * This class tracks the rebalance-related global cluster status.
  */
@@ -47,8 +47,7 @@ public class ClusterContext {
 
   /**
    * Construct the cluster context based on the current instance status.
-   *
-   * @param replicaSet    All the partition replicas that are managed by the rebalancer
+   * @param replicaSet All the partition replicas that are managed by the rebalancer
    * @param instanceCount The count of all the active instances that can be used to host partitions.
    */
   ClusterContext(Set<AssignableReplica> replicaSet, int instanceCount) {
@@ -95,8 +94,8 @@ public class ClusterContext {
   void addPartitionToFaultZone(String faultZoneId, String resourceName, String partition) {
     if (!_assignmentForFaultZoneMap.computeIfAbsent(faultZoneId, k -> new HashMap<>())
         .computeIfAbsent(resourceName, k -> new HashSet<>()).add(partition)) {
-      throw new HelixException(String
-          .format("Resource %s already has a replica from partition %s in fault zone %s",
+      throw new HelixException(
+          String.format("Resource %s already has a replica from partition %s in fault zone %s",
               resourceName, partition, faultZoneId));
     }
   }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
index d06cc5f..0e61eb3 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
@@ -44,7 +44,7 @@ public class TestConstraintBasedAlgorithm {
     SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
     SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
     when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(false);
-    when(mockSoftConstraint.getAssignmentOriginScore(any(), any(), any())).thenReturn(1.0f);
+    when(mockSoftConstraint.getAssignmentNormalizedScore(any(), any(), any())).thenReturn(1.0f);
 
     _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
         ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
@@ -62,7 +62,7 @@ public class TestConstraintBasedAlgorithm {
     SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
     SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
     when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(true);
-    when(mockSoftConstraint.getAssignmentOriginScore(any(), any(), any())).thenReturn(1.0f);
+    when(mockSoftConstraint.getAssignmentNormalizedScore(any(), any(), any())).thenReturn(1.0f);
     when(mockSoftConstraintWeightModel.getSumOfScores(any())).thenReturn(1.0f);
     _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
         ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java
new file mode 100644
index 0000000..7ffc40b
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestInstancePartitionsCountConstraint.java
@@ -0,0 +1,60 @@
+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 static org.mockito.Mockito.when;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestInstancePartitionsCountConstraint {
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+
+  private final SoftConstraint _constraint = new InstancePartitionsCountConstraint();
+
+  @Test
+  public void testWhenInstanceIsIdle() {
+    when(_testNode.getAssignedReplicaCount()).thenReturn(0);
+    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 1.0f);
+  }
+
+  @Test
+  public void testWhenInstanceIsFull() {
+    when(_testNode.getAssignedReplicaCount()).thenReturn(10);
+    when(_clusterContext.getEstimatedMaxPartitionCount()).thenReturn(10);
+    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.5f);
+  }
+
+  @Test
+  public void testWhenInstanceHalfOccupied() {
+    when(_testNode.getAssignedReplicaCount()).thenReturn(10);
+    when(_clusterContext.getEstimatedMaxPartitionCount()).thenReturn(20);
+    float score = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.75f);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSoftConstraintNormalizeFunction.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSoftConstraintNormalizeFunction.java
new file mode 100644
index 0000000..b523959
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestSoftConstraintNormalizeFunction.java
@@ -0,0 +1,47 @@
+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;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestSoftConstraintNormalizeFunction {
+  @Test
+  public void testDefaultNormalizeFunction() {
+    int maxScore = 100;
+    int minScore = 0;
+    SoftConstraint softConstraint = new SoftConstraint(maxScore, minScore) {
+      @Override
+      protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
+          ClusterContext clusterContext) {
+        return 0;
+      }
+    };
+
+    for (int i = minScore; i <= maxScore; i++) {
+      float normalized = softConstraint.getNormalizeFunction().scale(i);
+      Assert.assertTrue(normalized <= 1 && normalized >= 0,
+          String.format("input: %s, output: %s", i, normalized));
+    }
+  }
+}


[helix] 24/37: Add soft constraint: ResourcetopStateAntiAffinityConstraint (#465)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 153d38596184f9aefaa676ad4ceddde52f2969f3
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Wed Sep 11 19:22:00 2019 -0700

    Add soft constraint: ResourcetopStateAntiAffinityConstraint (#465)
    
    Add ResourcetopStateAntiAffinityConstraint
    
    The more total top state partitions assigned to the instance, the lower the score, vice versa.
---
 .../ResourceTopStateAntiAffinityConstraint.java    | 46 +++++++++++++++
 .../rebalancer/waged/model/AssignableNode.java     |  6 +-
 ...TestResourceTopStateAntiAffinityConstraint.java | 65 ++++++++++++++++++++++
 3 files changed, 114 insertions(+), 3 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java
new file mode 100644
index 0000000..b1e64b9
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java
@@ -0,0 +1,46 @@
+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 the proposed assignment according to the top state replication count on the instance.
+ * The higher number the number of top state partitions assigned to the instance, the lower the
+ * score, vice versa.
+ */
+public class ResourceTopStateAntiAffinityConstraint extends SoftConstraint {
+  @Override
+  protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    if (!replica.isReplicaTopState()) {
+      return (getMaxScore() + getMinScore()) / 2;
+    }
+
+    int curTopPartitionCountForResource = node.getAssignedTopStatePartitionsCount();
+    int doubleMaxTopStateCount = 2 * clusterContext.getEstimatedMaxTopStateCount();
+
+    return Math.max(
+        ((float) doubleMaxTopStateCount - curTopPartitionCountForResource) / doubleMaxTopStateCount,
+        0);
+  }
+}
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
index f25c289..6966353 100644
--- 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
@@ -187,10 +187,10 @@ public class AssignableNode implements Comparable<AssignableNode> {
   /**
    * @return The total count of assigned top state partitions.
    */
-  public long getAssignedTopStatePartitionsCount() {
-    return _currentAssignedReplicaMap.values().stream()
+  public int getAssignedTopStatePartitionsCount() {
+    return (int) _currentAssignedReplicaMap.values().stream()
         .flatMap(replicaMap -> replicaMap.values().stream())
-        .filter(replica -> replica.isReplicaTopState()).count();
+        .filter(AssignableReplica::isReplicaTopState).count();
   }
 
   /**
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java
new file mode 100644
index 0000000..06ef537
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java
@@ -0,0 +1,65 @@
+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 static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+public class TestResourceTopStateAntiAffinityConstraint {
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+
+  private final SoftConstraint _constraint = new ResourceTopStateAntiAffinityConstraint();
+
+  @Test
+  public void testGetAssignmentScoreWhenReplicaNotTopState() {
+    when(_testReplica.isReplicaTopState()).thenReturn(false);
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, (_constraint.getMaxScore() + _constraint.getMinScore()) / 2);
+    verifyZeroInteractions(_testNode);
+    verifyZeroInteractions(_clusterContext);
+  }
+
+  @Test
+  public void testGetAssignmentScoreWhenReplicaIsTopStateHeavyLoad() {
+    when(_testReplica.isReplicaTopState()).thenReturn(true);
+    when(_testNode.getAssignedTopStatePartitionsCount()).thenReturn(20);
+    when(_clusterContext.getEstimatedMaxTopStateCount()).thenReturn(20);
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.5f);
+  }
+
+  @Test
+  public void testGetAssignmentScoreWhenReplicaIsTopStateLightLoad() {
+    when(_testReplica.isReplicaTopState()).thenReturn(true);
+    when(_testNode.getAssignedTopStatePartitionsCount()).thenReturn(0);
+    when(_clusterContext.getEstimatedMaxTopStateCount()).thenReturn(20);
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 1f);
+  }
+}


[helix] 15/37: Record the replica objects in the AssignableNode in addition to the partition name (#440)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9ee65a0cbc74ef2b25c0bb48043a32763cb70cbb
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Tue Sep 3 16:30:03 2019 -0700

    Record the replica objects in the AssignableNode in addition to the partition name (#440)
    
    The replica instances are required while the rebalance algorithm generating ResourceAssignment based on the AssignableNode instances.
    Refine the methods of the AssignableNode for better code style and readability.
    Also, modify the related test cases to verify state information and new methods.
---
 .../rebalancer/waged/model/AssignableNode.java     | 177 +++++++++++++--------
 .../rebalancer/waged/model/AssignableReplica.java  |  12 ++
 .../waged/model/ClusterModelProvider.java          |   2 +-
 .../rebalancer/waged/model/TestAssignableNode.java |  83 ++++++++--
 .../rebalancer/waged/model/TestClusterModel.java   |   6 +-
 .../waged/model/TestClusterModelProvider.java      |  10 +-
 6 files changed, 203 insertions(+), 87 deletions(-)

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
index e2fd676..35c3c38 100644
--- 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
@@ -33,6 +33,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import static java.lang.Math.max;
 
@@ -51,16 +52,19 @@ public class AssignableNode {
   private Map<String, Integer> _maxCapacity;
   private int _maxPartition; // maximum number of the partitions that can be assigned to the node.
 
-  // proposed assignment tracking
-  // <resource name, partition name set>
-  private Map<String, Set<String>> _currentAssignments;
-  // <resource name, top state partition name>
-  private Map<String, Set<String>> _currentTopStateAssignments;
-  // <capacity key, capacity value>
-  private Map<String, Integer> _currentCapacity;
+  // A map of <resource name, <partition name, replica>> that tracks the replicas assigned to the node.
+  private Map<String, Map<String, AssignableReplica>> _currentAssignedReplicaMap;
+  // A map of <capacity key, capacity value> that tracks the current available node capacity
+  private Map<String, Integer> _currentCapacityMap;
   // The maximum capacity utilization (0.0 - 1.0) across all the capacity categories.
   private float _highestCapacityUtilization;
 
+  /**
+   * @param clusterConfig
+   * @param instanceConfig
+   * @param instanceName
+   * @param existingAssignment A collection of replicas that have been pre-allocated to the node.
+   */
   AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName,
       Collection<AssignableReplica> existingAssignment) {
     _instanceName = instanceName;
@@ -68,9 +72,8 @@ public class AssignableNode {
   }
 
   private void reset() {
-    _currentAssignments = new HashMap<>();
-    _currentTopStateAssignments = new HashMap<>();
-    _currentCapacity = new HashMap<>();
+    _currentAssignedReplicaMap = new HashMap<>();
+    _currentCapacityMap = new HashMap<>();
     _highestCapacityUtilization = 0;
   }
 
@@ -80,8 +83,8 @@ public class AssignableNode {
    * refreshed. This is under the assumption that the capacity mappings of InstanceConfig and ResourceConfig could
    * subject to change. If the assumption is no longer true, this function should become private.
    *
-   * @param clusterConfig  - the Cluster Config of the cluster where the node is located
-   * @param instanceConfig - the Instance Config of the node
+   * @param clusterConfig      - the Cluster Config of the cluster where the node is located
+   * @param instanceConfig     - the Instance Config of the node
    * @param existingAssignment - all the existing replicas that are current assigned to the node
    */
   private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig,
@@ -92,7 +95,7 @@ public class AssignableNode {
     if (instanceCapacity.isEmpty()) {
       instanceCapacity = clusterConfig.getDefaultInstanceCapacityMap();
     }
-    _currentCapacity.putAll(instanceCapacity);
+    _currentCapacityMap.putAll(instanceCapacity);
     _faultZone = computeFaultZone(clusterConfig, instanceConfig);
     _instanceTags = new HashSet<>(instanceConfig.getTags());
     _disabledPartitionsMap = instanceConfig.getDisabledPartitionsMap();
@@ -108,78 +111,110 @@ public class AssignableNode {
    * @param assignableReplica - the replica to be assigned
    */
   void assign(AssignableReplica assignableReplica) {
-    if (!addToAssignmentRecord(assignableReplica, _currentAssignments)) {
-      throw new HelixException(String
-          .format("Resource %s already has a replica from partition %s on node %s",
-              assignableReplica.getResourceName(), assignableReplica.getPartitionName(),
-              getInstanceName()));
-    } else {
-      if (assignableReplica.isReplicaTopState()) {
-        addToAssignmentRecord(assignableReplica, _currentTopStateAssignments);
-      }
-      assignableReplica.getCapacity().entrySet().stream().forEach(
-          capacity -> updateCapacityAndUtilization(capacity.getKey(), capacity.getValue()));
-    }
+    addToAssignmentRecord(assignableReplica);
+    assignableReplica.getCapacity().entrySet().stream()
+        .forEach(capacity -> updateCapacityAndUtilization(capacity.getKey(), capacity.getValue()));
   }
 
   /**
    * Release a replica from the node.
    * If the replication is not on this node, the assignable node is not updated.
    *
-   * @param assignableReplica - the replica to be released
+   * @param replica - the replica to be released
    */
-  void release(AssignableReplica assignableReplica) throws IllegalArgumentException {
-    String resourceName = assignableReplica.getResourceName();
-    String partitionName = assignableReplica.getPartitionName();
+  void release(AssignableReplica replica) throws IllegalArgumentException {
+    String resourceName = replica.getResourceName();
+    String partitionName = replica.getPartitionName();
 
     // Check if the release is necessary
-    if (!_currentAssignments.containsKey(resourceName)) {
+    if (!_currentAssignedReplicaMap.containsKey(resourceName)) {
       LOG.warn("Resource {} is not on node {}. Ignore the release call.", resourceName,
           getInstanceName());
       return;
     }
-    Set<String> partitions = _currentAssignments.get(resourceName);
-    if (!partitions.contains(partitionName)) {
-      LOG.warn(String
-          .format("Resource %s does not have a replica from partition %s on node %s", resourceName,
-              partitionName, getInstanceName()));
+
+    Map<String, AssignableReplica> partitionMap = _currentAssignedReplicaMap.get(resourceName);
+    if (!partitionMap.containsKey(partitionName) || !partitionMap.get(partitionName)
+        .equals(replica)) {
+      LOG.warn("Replica {} is not assigned to node {}. Ignore the release call.",
+          replica.toString(), getInstanceName());
       return;
     }
 
-    partitions.remove(assignableReplica.getPartitionName());
-    if (assignableReplica.isReplicaTopState()) {
-      _currentTopStateAssignments.get(resourceName).remove(partitionName);
-    }
+    AssignableReplica removedReplica = partitionMap.remove(partitionName);
     // Recalculate utilization because of release
     _highestCapacityUtilization = 0;
-    assignableReplica.getCapacity().entrySet().stream()
+    removedReplica.getCapacity().entrySet().stream()
         .forEach(entry -> updateCapacityAndUtilization(entry.getKey(), -1 * entry.getValue()));
   }
 
-  public Map<String, Set<String>> getCurrentAssignmentsMap() {
-    return _currentAssignments;
+  /**
+   * @return A set of all assigned replicas on the node.
+   */
+  public Set<AssignableReplica> getAssignedReplicas() {
+    return _currentAssignedReplicaMap.values().stream()
+        .flatMap(replicaMap -> replicaMap.values().stream()).collect(Collectors.toSet());
   }
 
-  public Set<String> getCurrentAssignmentsByResource(String resource) {
-    return _currentAssignments.getOrDefault(resource, Collections.emptySet());
+  /**
+   * @return The current assignment in a map of <resource name, set of partition names>
+   */
+  public Map<String, Set<String>> getAssignedPartitionsMap() {
+    Map<String, Set<String>> assignmentMap = new HashMap<>();
+    for (String resourceName : _currentAssignedReplicaMap.keySet()) {
+      assignmentMap.put(resourceName, _currentAssignedReplicaMap.get(resourceName).keySet());
+    }
+    return assignmentMap;
   }
 
-  public Set<String> getCurrentTopStateAssignmentsByResource(String resource) {
-    return _currentTopStateAssignments.getOrDefault(resource, Collections.emptySet());
+  /**
+   * @param resource Resource name
+   * @return A set of the current assigned replicas' partition names in the specified resource.
+   */
+  public Set<String> getAssignedPartitionsByResource(String resource) {
+    return _currentAssignedReplicaMap.getOrDefault(resource, Collections.emptyMap()).keySet();
   }
 
-  public int getTopStateAssignmentTotalSize() {
-    return _currentTopStateAssignments.values().stream().mapToInt(Set::size).sum();
+  /**
+   * @param resource Resource name
+   * @return A set of the current assigned replicas' partition names with the top state in the specified resource.
+   */
+  public Set<String> getAssignedTopStatePartitionsByResource(String resource) {
+    return _currentAssignedReplicaMap.getOrDefault(resource, Collections.emptyMap()).entrySet()
+        .stream().filter(partitionEntry -> partitionEntry.getValue().isReplicaTopState())
+        .map(partitionEntry -> partitionEntry.getKey()).collect(Collectors.toSet());
   }
 
-  public int getCurrentAssignmentCount() {
-    return _currentAssignments.values().stream().mapToInt(Set::size).sum();
+  /**
+   * @return The total count of assigned top state partitions.
+   */
+  public long getAssignedTopStatePartitionsCount() {
+    return _currentAssignedReplicaMap.values().stream()
+        .flatMap(replicaMap -> replicaMap.values().stream())
+        .filter(replica -> replica.isReplicaTopState()).count();
   }
 
+  /**
+   * @return The total count of assigned replicas.
+   */
+  public long getAssignedReplicaCount() {
+    return _currentAssignedReplicaMap.values().stream().mapToInt(Map::size).sum();
+  }
+
+  /**
+   * @return The current available capacity.
+   */
   public Map<String, Integer> getCurrentCapacity() {
-    return _currentCapacity;
+    return _currentCapacityMap;
   }
 
+  /**
+   * Return the most concerning capacity utilization number for evenly partition assignment.
+   * The method dynamically returns the highest utilization number among all the capacity categories.
+   * For example, if the current node usage is {CPU: 0.9, MEM: 0.4, DISK: 0.6}. Then this call shall
+   * return 0.9.
+   * @return The highest utilization number of the node among all the capacity category.
+   */
   public float getHighestCapacityUtilization() {
     return _highestCapacityUtilization;
   }
@@ -196,14 +231,23 @@ public class AssignableNode {
     return _faultZone;
   }
 
+  /**
+   * @return A map of <resource name, set of partition names> contains all the partitions that are disabled on the node.
+   */
   public Map<String, List<String>> getDisabledPartitionsMap() {
     return _disabledPartitionsMap;
   }
 
+  /**
+   * @return A map of <capacity category, capacity number> that describes the max capacity of the node.
+   */
   public Map<String, Integer> getMaxCapacity() {
     return _maxCapacity;
   }
 
+  /**
+   * @return The max partition count that are allowed to be allocated on the node.
+   */
   public int getMaxPartition() {
     return _maxPartition;
   }
@@ -268,10 +312,7 @@ public class AssignableNode {
   private void assignNewBatch(Collection<AssignableReplica> replicas) {
     Map<String, Integer> totalPartitionCapacity = new HashMap<>();
     for (AssignableReplica replica : replicas) {
-      addToAssignmentRecord(replica, _currentAssignments);
-      if (replica.isReplicaTopState()) {
-        addToAssignmentRecord(replica, _currentTopStateAssignments);
-      }
+      addToAssignmentRecord(replica);
       // increment the capacity requirement according to partition's capacity configuration.
       for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
         totalPartitionCapacity.compute(capacity.getKey(),
@@ -287,16 +328,28 @@ public class AssignableNode {
     }
   }
 
-  private boolean addToAssignmentRecord(AssignableReplica replica,
-      Map<String, Set<String>> currentAssignments) {
-    return currentAssignments.computeIfAbsent(replica.getResourceName(), k -> new HashSet<>())
-        .add(replica.getPartitionName());
+  /**
+   * @throws HelixException if the replica has already been assigned to the node.
+   */
+  private void addToAssignmentRecord(AssignableReplica replica) {
+    String resourceName = replica.getResourceName();
+    String partitionName = replica.getPartitionName();
+    if (_currentAssignedReplicaMap.containsKey(resourceName) && _currentAssignedReplicaMap
+        .get(resourceName).containsKey(partitionName)) {
+      throw new HelixException(String
+          .format("Resource %s already has a replica with state %s from partition %s on node %s",
+              replica.getResourceName(), replica.getReplicaState(), replica.getPartitionName(),
+              getInstanceName()));
+    } else {
+      _currentAssignedReplicaMap.computeIfAbsent(resourceName, key -> new HashMap<>())
+          .put(partitionName, replica);
+    }
   }
 
   private void updateCapacityAndUtilization(String capacityKey, int valueToSubtract) {
-    if (_currentCapacity.containsKey(capacityKey)) {
-      int newCapacity = _currentCapacity.get(capacityKey) - valueToSubtract;
-      _currentCapacity.put(capacityKey, newCapacity);
+    if (_currentCapacityMap.containsKey(capacityKey)) {
+      int newCapacity = _currentCapacityMap.get(capacityKey) - valueToSubtract;
+      _currentCapacityMap.put(capacityKey, newCapacity);
       // For the purpose of constraint calculation, the max utilization cannot be larger than 100%.
       float utilization = Math.min(
           (float) (_maxCapacity.get(capacityKey) - newCapacity) / _maxCapacity.get(capacityKey), 1);
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
index 0082a2d..ade04bf 100644
--- 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
@@ -107,6 +107,18 @@ public class AssignableReplica implements Comparable<AssignableReplica> {
     return 0;
   }
 
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof AssignableReplica) {
+      return compareTo((AssignableReplica) obj) == 0;
+    } else {
+      return false;
+    }
+  }
+
   public static String generateReplicaKey(String resourceName, String partitionName, String state) {
     return String.format("%s-%s-%s", resourceName, partitionName, state);
   }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index e0a5e35..61f5d8d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -236,7 +236,7 @@ public class ClusterModelProvider {
       Set<AssignableNode> assignableNodes) {
     Map<String, Map<String, Set<String>>> faultZoneAssignmentMap = new HashMap<>();
     assignableNodes.stream().forEach(node -> {
-      for (Map.Entry<String, Set<String>> resourceMap : node.getCurrentAssignmentsMap()
+      for (Map.Entry<String, Set<String>> resourceMap : node.getAssignedPartitionsMap()
           .entrySet()) {
         faultZoneAssignmentMap.computeIfAbsent(node.getFaultZone(), k -> new HashMap<>())
             .computeIfAbsent(resourceMap.getKey(), k -> new HashSet<>())
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index f55d0fc..34a03a9 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 
@@ -48,6 +49,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
     Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
 
+    Set<String> expectedTopStateAssignmentSet1 = new HashSet<>(_partitionNames.subList(0, 1));
+    Set<String> expectedTopStateAssignmentSet2 = new HashSet<>(_partitionNames.subList(2, 3));
     Set<String> expectedAssignmentSet1 = new HashSet<>(_partitionNames.subList(0, 2));
     Set<String> expectedAssignmentSet2 = new HashSet<>(_partitionNames.subList(2, 4));
     Map<String, Set<String>> expectedAssignment = new HashMap<>();
@@ -60,15 +63,28 @@ public class TestAssignableNode extends AbstractTestClusterModel {
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
-    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
-    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 4);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsMap(), expectedAssignment);
+    Assert.assertEquals(assignableNode.getAssignedReplicaCount(), 4);
     Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
-    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
     Assert.assertEquals(assignableNode.getMaxPartition(), 5);
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
-    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
-    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+    Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
+    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
+        expectedAssignmentSet1);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(1)),
+        expectedAssignmentSet2);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(0)),
+            expectedTopStateAssignmentSet1);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(1)),
+            expectedTopStateAssignmentSet2);
+    Assert.assertEquals(assignableNode.getAssignedTopStatePartitionsCount(),
+        expectedTopStateAssignmentSet1.size() + expectedTopStateAssignmentSet2.size());
 
     // Test 2 - release assignment from the AssignableNode
     AssignableReplica removingReplica =
@@ -77,18 +93,39 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     expectedAssignment.get(_resourceNames.get(1)).remove(_partitionNames.get(2));
     expectedCapacityMap.put("item1", 9);
     expectedCapacityMap.put("item2", 18);
+    Iterator<AssignableReplica> iter = assignmentSet.iterator();
+    while (iter.hasNext()) {
+      AssignableReplica replica = iter.next();
+      if (replica.equals(removingReplica)) {
+        iter.remove();
+      }
+    }
+    expectedTopStateAssignmentSet2.remove(_partitionNames.get(2));
 
     assignableNode.release(removingReplica);
 
-    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
-    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 3);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsMap(), expectedAssignment);
+    Assert.assertEquals(assignableNode.getAssignedReplicaCount(), 3);
     Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 11.0 / 20.0, 0.005);
-    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
     Assert.assertEquals(assignableNode.getMaxPartition(), 5);
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
-    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
-    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+    Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
+    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
+        expectedAssignmentSet1);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(1)),
+        expectedAssignmentSet2);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(0)),
+            expectedTopStateAssignmentSet1);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(1)),
+            expectedTopStateAssignmentSet2);
+    Assert.assertEquals(assignableNode.getAssignedTopStatePartitionsCount(),
+        expectedTopStateAssignmentSet1.size() + expectedTopStateAssignmentSet2.size());
 
     // Test 3 - add assignment to the AssignableNode
     AssignableReplica addingReplica =
@@ -97,18 +134,32 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     expectedAssignment.get(_resourceNames.get(1)).add(_partitionNames.get(2));
     expectedCapacityMap.put("item1", 4);
     expectedCapacityMap.put("item2", 8);
+    assignmentSet.add(addingReplica);
 
     assignableNode.assign(addingReplica);
 
-    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
-    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 4);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsMap(), expectedAssignment);
+    Assert.assertEquals(assignableNode.getAssignedReplicaCount(), 4);
     Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
-    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
     Assert.assertEquals(assignableNode.getMaxPartition(), 5);
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
-    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
-    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+    Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
+    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
+        expectedAssignmentSet1);
+    Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(1)),
+        expectedAssignmentSet2);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(0)),
+            expectedTopStateAssignmentSet1);
+    Assert
+        .assertEquals(assignableNode.getAssignedTopStatePartitionsByResource(_resourceNames.get(1)),
+            expectedTopStateAssignmentSet2);
+    Assert.assertEquals(assignableNode.getAssignedTopStatePartitionsCount(),
+        expectedTopStateAssignmentSet1.size() + expectedTopStateAssignmentSet2.size());
   }
 
   @Test
@@ -126,7 +177,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     assignableNode.release(removingReplica);
   }
 
-  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "Resource Resource1 already has a replica from partition Partition1 on node testInstanceId")
+  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "Resource Resource1 already has a replica with state SLAVE from partition Partition1 on node testInstanceId")
   public void testAssignDuplicateReplica() throws IOException {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
     Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
index c07bd98..a45b729 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
@@ -63,7 +63,7 @@ public class TestClusterModel extends AbstractTestClusterModel {
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
         .allMatch(resourceMap -> resourceMap.values().isEmpty()));
     Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
-        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+        .anyMatch(node -> node.getAssignedReplicaCount() != 0));
 
     // The initialization of the context, node and replication has been tested separately. So for
     // cluster model, focus on testing the assignment and release.
@@ -78,7 +78,7 @@ public class TestClusterModel extends AbstractTestClusterModel {
     Assert.assertTrue(
         clusterModel.getContext().getAssignmentForFaultZoneMap().get(assignableNode.getFaultZone())
             .get(replica.getResourceName()).contains(replica.getPartitionName()));
-    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().get(replica.getResourceName())
+    Assert.assertTrue(assignableNode.getAssignedPartitionsMap().get(replica.getResourceName())
         .contains(replica.getPartitionName()));
 
     // Assign a nonexist replication
@@ -109,6 +109,6 @@ public class TestClusterModel extends AbstractTestClusterModel {
         .allMatch(resourceMap -> resourceMap.values().stream()
             .allMatch(partitions -> partitions.isEmpty())));
     Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
-        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+        .anyMatch(node -> node.getAssignedReplicaCount() != 0));
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index 638182f..1ec92a9 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -106,7 +106,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     Assert.assertFalse(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
         .anyMatch(resourceMap -> !resourceMap.isEmpty()));
     Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
-        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+        .anyMatch(node -> node.getAssignedReplicaCount() != 0));
     // Have all 3 instances
     Assert.assertEquals(
         clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)
@@ -168,7 +168,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
         .allMatch(resourceMap -> resourceMap.values().stream()
             .allMatch(partitionSet -> partitionSet.size() == 2)));
     Assert.assertEquals(
-        clusterModel.getAssignableNodes().get(_testInstanceId).getCurrentAssignmentCount(), 4);
+        clusterModel.getAssignableNodes().get(_testInstanceId).getAssignedReplicaCount(), 4);
     // Since each resource has 2 replicas assigned, the assignable replica count should be 10.
     Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
     Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
@@ -183,7 +183,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
         .allMatch(resourceMap -> resourceMap.isEmpty()));
     Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
-        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+        .anyMatch(node -> node.getAssignedReplicaCount() != 0));
     // Shall have 2 resources and 12 replicas
     Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
     Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
@@ -211,7 +211,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     // Only the first instance will have 2 assignment from resource2.
     for (String instance : _instances) {
       Assert
-          .assertEquals(clusterModel.getAssignableNodes().get(instance).getCurrentAssignmentCount(),
+          .assertEquals(clusterModel.getAssignableNodes().get(instance).getAssignedReplicaCount(),
               instance.equals(_testInstanceId) ? 2 : 0);
     }
     // Shall have 2 resources and 12 replicas
@@ -233,7 +233,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     Assert.assertFalse(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
         .anyMatch(resourceMap -> !resourceMap.isEmpty()));
     Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
-        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+        .anyMatch(node -> node.getAssignedReplicaCount() != 0));
     // Have only 2 instances
     Assert.assertEquals(
         clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)


[helix] 02/37: Adding the configuration items of the WAGED rebalancer. (#348)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 39f7adf6774c78f7aab68a5af0d814278655e736
Author: jiajunwang <18...@users.noreply.github.com>
AuthorDate: Fri Jul 26 11:42:52 2019 -0700

    Adding the configuration items of the WAGED rebalancer. (#348)
    
    * Adding the configuration items of the WAGED rebalancer.
    
    Including: Instance Capacity Keys, Rebalance Preferences, Instance Capacity Details, Partition Capacity (the weight) Details.
    Also adding test to cover the new configuration items.
---
 .../java/org/apache/helix/model/ClusterConfig.java | 129 +++++++++++---
 .../org/apache/helix/model/InstanceConfig.java     |  62 +++++--
 .../org/apache/helix/model/ResourceConfig.java     | 139 ++++++++++++++-
 .../org/apache/helix/model/TestClusterConfig.java  | 130 ++++++++++++++
 .../org/apache/helix/model/TestInstanceConfig.java |  66 +++++++-
 .../org/apache/helix/model/TestResourceConfig.java | 186 +++++++++++++++++++++
 6 files changed, 669 insertions(+), 43 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
index 8905dda..ee942c7 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ClusterConfig.java
@@ -19,12 +19,8 @@ package org.apache.helix.model;
  * under the License.
  */
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
@@ -32,6 +28,12 @@ import org.apache.helix.api.config.HelixConfigProperty;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Cluster configurations
  */
@@ -80,7 +82,19 @@ public class ClusterConfig extends HelixProperty {
     DISABLED_INSTANCES,
 
     // Specifies job types and used for quota allocation
-    QUOTA_TYPES
+    QUOTA_TYPES,
+
+    // The required instance capacity keys for resource partition assignment calculation.
+    INSTANCE_CAPACITY_KEYS,
+    // The preference of the rebalance result.
+    // EVENNESS - Evenness of the resource utilization, partition, and top state distribution.
+    // LESS_MOVEMENT - the tendency of keeping the current assignment instead of moving the partition for optimal assignment.
+    REBALANCE_PREFERENCE
+  }
+
+  public enum GlobalRebalancePreferenceKey {
+    EVENNESS,
+    LESS_MOVEMENT
   }
 
   private final static int DEFAULT_MAX_CONCURRENT_TASK_PER_INSTANCE = 40;
@@ -94,6 +108,15 @@ public class ClusterConfig extends HelixProperty {
 
   public final static String TASK_QUOTA_RATIO_NOT_SET = "-1";
 
+  // Default preference for all the aspects should be the same to ensure balanced setup.
+  public final static Map<GlobalRebalancePreferenceKey, Integer>
+      DEFAULT_GLOBAL_REBALANCE_PREFERENCE =
+      ImmutableMap.<GlobalRebalancePreferenceKey, Integer>builder()
+          .put(GlobalRebalancePreferenceKey.EVENNESS, 1)
+          .put(GlobalRebalancePreferenceKey.LESS_MOVEMENT, 1).build();
+  private final static int MAX_REBALANCE_PREFERENCE = 10;
+  private final static int MIN_REBALANCE_PREFERENCE = 0;
+
   /**
    * Instantiate for a specific cluster
    * @param cluster the cluster identifier
@@ -112,21 +135,21 @@ public class ClusterConfig extends HelixProperty {
 
   /**
    * Set task quota type with the ratio of this quota.
-   * @param quotaType String
+   * @param quotaType  String
    * @param quotaRatio int
    */
   public void setTaskQuotaRatio(String quotaType, int quotaRatio) {
     if (_record.getMapField(ClusterConfigProperty.QUOTA_TYPES.name()) == null) {
       _record.setMapField(ClusterConfigProperty.QUOTA_TYPES.name(), new HashMap<String, String>());
     }
-    _record.getMapField(ClusterConfigProperty.QUOTA_TYPES.name()).put(quotaType,
-        Integer.toString(quotaRatio));
+    _record.getMapField(ClusterConfigProperty.QUOTA_TYPES.name())
+        .put(quotaType, Integer.toString(quotaRatio));
   }
 
   /**
    * Set task quota type with the ratio of this quota. Quota ratio must be a String that is
    * parse-able into an int.
-   * @param quotaType String
+   * @param quotaType  String
    * @param quotaRatio String
    */
   public void setTaskQuotaRatio(String quotaType, String quotaRatio) {
@@ -209,8 +232,8 @@ public class ClusterConfig extends HelixProperty {
    * @return
    */
   public Boolean isPersistIntermediateAssignment() {
-    return _record.getBooleanField(ClusterConfigProperty.PERSIST_INTERMEDIATE_ASSIGNMENT.toString(),
-        false);
+    return _record
+        .getBooleanField(ClusterConfigProperty.PERSIST_INTERMEDIATE_ASSIGNMENT.toString(), false);
   }
 
   /**
@@ -232,8 +255,8 @@ public class ClusterConfig extends HelixProperty {
   }
 
   public Boolean isPipelineTriggersDisabled() {
-    return _record.getBooleanField(ClusterConfigProperty.HELIX_DISABLE_PIPELINE_TRIGGERS.toString(),
-        false);
+    return _record
+        .getBooleanField(ClusterConfigProperty.HELIX_DISABLE_PIPELINE_TRIGGERS.toString(), false);
   }
 
   /**
@@ -402,8 +425,8 @@ public class ClusterConfig extends HelixProperty {
    * @return
    */
   public int getNumOfflineInstancesForAutoExit() {
-    return _record.getIntField(ClusterConfigProperty.NUM_OFFLINE_INSTANCES_FOR_AUTO_EXIT.name(),
-        -1);
+    return _record
+        .getIntField(ClusterConfigProperty.NUM_OFFLINE_INSTANCES_FOR_AUTO_EXIT.name(), -1);
   }
 
   /**
@@ -443,9 +466,7 @@ public class ClusterConfig extends HelixProperty {
     if (obj instanceof ClusterConfig) {
       ClusterConfig that = (ClusterConfig) obj;
 
-      if (this.getId().equals(that.getId())) {
-        return true;
-      }
+      return this.getId().equals(that.getId());
     }
     return false;
   }
@@ -489,8 +510,8 @@ public class ClusterConfig extends HelixProperty {
     }
 
     if (!configStrs.isEmpty()) {
-      _record.setListField(ClusterConfigProperty.STATE_TRANSITION_THROTTLE_CONFIGS.name(),
-          configStrs);
+      _record
+          .setListField(ClusterConfigProperty.STATE_TRANSITION_THROTTLE_CONFIGS.name(), configStrs);
     }
   }
 
@@ -578,7 +599,7 @@ public class ClusterConfig extends HelixProperty {
   public int getErrorPartitionThresholdForLoadBalance() {
     return _record.getIntField(
         ClusterConfigProperty.ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE.name(),
-        DEFAULT_ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE);
+            DEFAULT_ERROR_PARTITION_THRESHOLD_FOR_LOAD_BALANCE);
   }
 
   /**
@@ -657,6 +678,70 @@ public class ClusterConfig extends HelixProperty {
   }
 
   /**
+   * Set the required Instance Capacity Keys.
+   * @param capacityKeys
+   */
+  public void setInstanceCapacityKeys(List<String> capacityKeys) {
+    if (capacityKeys == null || capacityKeys.isEmpty()) {
+      throw new IllegalArgumentException("The input instance capacity key list is empty.");
+    }
+    _record.setListField(ClusterConfigProperty.INSTANCE_CAPACITY_KEYS.name(), capacityKeys);
+  }
+
+  /**
+   * @return The required Instance Capacity Keys. If not configured, return an empty list.
+   */
+  public List<String> getInstanceCapacityKeys() {
+    List<String> capacityKeys = _record.getListField(ClusterConfigProperty.INSTANCE_CAPACITY_KEYS.name());
+    if (capacityKeys == null) {
+      return Collections.emptyList();
+    }
+    return capacityKeys;
+  }
+
+  /**
+   * Set the global rebalancer's assignment preference.
+   * @param preference A map of the GlobalRebalancePreferenceKey and the corresponding weight.
+   *                   The ratio of the configured weights will determine the rebalancer's behavior.
+   */
+  public void setGlobalRebalancePreference(Map<GlobalRebalancePreferenceKey, Integer> preference) {
+    Map<String, String> preferenceMap = new HashMap<>();
+
+    preference.entrySet().stream().forEach(entry -> {
+      if (entry.getValue() > MAX_REBALANCE_PREFERENCE
+          || entry.getValue() < MIN_REBALANCE_PREFERENCE) {
+        throw new IllegalArgumentException(String
+            .format("Invalid global rebalance preference configuration. Key %s, Value %d.",
+                entry.getKey().name(), entry.getValue()));
+      }
+      preferenceMap.put(entry.getKey().name(), Integer.toString(entry.getValue()));
+    });
+
+    _record.setMapField(ClusterConfigProperty.REBALANCE_PREFERENCE.name(), preferenceMap);
+  }
+
+  /**
+   * Get the global rebalancer's assignment preference.
+   */
+  public Map<GlobalRebalancePreferenceKey, Integer> getGlobalRebalancePreference() {
+    Map<String, String> preferenceStrMap =
+        _record.getMapField(ClusterConfigProperty.REBALANCE_PREFERENCE.name());
+    if (preferenceStrMap != null && !preferenceStrMap.isEmpty()) {
+      Map<GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+      for (GlobalRebalancePreferenceKey key : GlobalRebalancePreferenceKey.values()) {
+        if (!preferenceStrMap.containsKey(key.name())) {
+          // If any key is not configured with a value, return the default config.
+          return DEFAULT_GLOBAL_REBALANCE_PREFERENCE;
+        }
+        preference.put(key, Integer.parseInt(preferenceStrMap.get(key.name())));
+      }
+      return preference;
+    }
+    // If configuration is not complete, return the default one.
+    return DEFAULT_GLOBAL_REBALANCE_PREFERENCE;
+  }
+
+  /**
    * Get IdealState rules defined in the cluster config.
    * @return
    */
diff --git a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
index f65a1bd..88fd1dd 100644
--- a/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/InstanceConfig.java
@@ -19,6 +19,14 @@ package org.apache.helix.model;
  * under the License.
  */
 
+import com.google.common.base.Splitter;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.util.HelixUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -27,15 +35,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
-import org.apache.helix.HelixException;
-import org.apache.helix.HelixProperty;
-import org.apache.helix.ZNRecord;
-import org.apache.helix.util.HelixUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Splitter;
+import java.util.stream.Collectors;
 
 /**
  * Instance configurations
@@ -55,7 +55,8 @@ public class InstanceConfig extends HelixProperty {
     INSTANCE_WEIGHT,
     DOMAIN,
     DELAY_REBALANCE_ENABLED,
-    MAX_CONCURRENT_TASK
+    MAX_CONCURRENT_TASK,
+    INSTANCE_CAPACITY_MAP
   }
 
   public static final int WEIGHT_NOT_SET = -1;
@@ -505,6 +506,47 @@ public class InstanceConfig extends HelixProperty {
     _record.setIntField(InstanceConfigProperty.MAX_CONCURRENT_TASK.name(), maxConcurrentTask);
   }
 
+  /**
+   * Get the instance capacity information from the map fields
+   *
+   * @return data map if it exists, or empty map
+   */
+  public Map<String, Integer> getInstanceCapacityMap() {
+    Map<String, String> capacityData =
+        _record.getMapField(InstanceConfigProperty.INSTANCE_CAPACITY_MAP.name());
+
+    if (capacityData != null) {
+      return capacityData.entrySet().stream().collect(
+          Collectors.toMap(entry -> entry.getKey(), entry -> Integer.parseInt(entry.getValue())));
+    }
+    return Collections.emptyMap();
+  }
+
+  /**
+   * Set the instance capacity information with an Integer mapping
+   * @param capacityDataMap - map of instance capacity data
+   * @throws IllegalArgumentException - when any of the data value is a negative number or when the map is empty
+   */
+  public void setInstanceCapacityMap(Map<String, Integer> capacityDataMap)
+      throws IllegalArgumentException {
+    if (capacityDataMap == null || capacityDataMap.size() == 0) {
+      throw new IllegalArgumentException("Capacity Data is empty");
+    }
+
+    Map<String, String> capacityData = new HashMap<>();
+
+    capacityDataMap.entrySet().stream().forEach(entry -> {
+      if (entry.getValue() < 0) {
+        throw new IllegalArgumentException(String
+            .format("Capacity Data contains a negative value: %s = %d", entry.getKey(),
+                entry.getValue()));
+      }
+      capacityData.put(entry.getKey(), Integer.toString(entry.getValue()));
+    });
+
+    _record.setMapField(InstanceConfigProperty.INSTANCE_CAPACITY_MAP.name(), capacityData);
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (obj instanceof InstanceConfig) {
diff --git a/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java b/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
index 274640c..1ead08e 100644
--- a/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
+++ b/helix-core/src/main/java/org/apache/helix/model/ResourceConfig.java
@@ -19,19 +19,23 @@ package org.apache.helix.model;
  * under the License.
  */
 
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import java.util.TreeMap;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.config.HelixConfigProperty;
 import org.apache.helix.api.config.RebalanceConfig;
 import org.apache.helix.api.config.StateTransitionTimeoutConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.type.TypeReference;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
 /**
  * Resource configurations
  */
@@ -53,7 +57,8 @@ public class ResourceConfig extends HelixProperty {
     RESOURCE_TYPE,
     GROUP_ROUTING_ENABLED,
     EXTERNAL_VIEW_DISABLED,
-    DELAY_REBALANCE_ENABLED
+    DELAY_REBALANCE_ENABLED,
+    PARTITION_CAPACITY_MAP
   }
 
   public enum ResourceConfigConstants {
@@ -61,6 +66,10 @@ public class ResourceConfig extends HelixProperty {
   }
 
   private static final Logger _logger = LoggerFactory.getLogger(ResourceConfig.class.getName());
+  private static final ObjectMapper _objectMapper = new ObjectMapper();
+
+  public static final String DEFAULT_PARTITION_KEY = "DEFAULT";
+
   /**
    * Instantiate for a specific instance
    *
@@ -92,10 +101,24 @@ public class ResourceConfig extends HelixProperty {
       String stateModelDefRef, String stateModelFactoryName, String numReplica,
       int minActiveReplica, int maxPartitionsPerInstance, String instanceGroupTag,
       Boolean helixEnabled, String resourceGroupName, String resourceType,
-      Boolean groupRoutingEnabled, Boolean externalViewDisabled,
-      RebalanceConfig rebalanceConfig, StateTransitionTimeoutConfig stateTransitionTimeoutConfig,
+      Boolean groupRoutingEnabled, Boolean externalViewDisabled, RebalanceConfig rebalanceConfig,
+      StateTransitionTimeoutConfig stateTransitionTimeoutConfig,
       Map<String, List<String>> listFields, Map<String, Map<String, String>> mapFields,
       Boolean p2pMessageEnabled) {
+    this(resourceId, monitorDisabled, numPartitions, stateModelDefRef, stateModelFactoryName,
+        numReplica, minActiveReplica, maxPartitionsPerInstance, instanceGroupTag, helixEnabled,
+        resourceGroupName, resourceType, groupRoutingEnabled, externalViewDisabled, rebalanceConfig,
+        stateTransitionTimeoutConfig, listFields, mapFields, p2pMessageEnabled, null);
+  }
+
+  private ResourceConfig(String resourceId, Boolean monitorDisabled, int numPartitions,
+    String stateModelDefRef, String stateModelFactoryName, String numReplica,
+    int minActiveReplica, int maxPartitionsPerInstance, String instanceGroupTag,
+        Boolean helixEnabled, String resourceGroupName, String resourceType,
+        Boolean groupRoutingEnabled, Boolean externalViewDisabled,
+        RebalanceConfig rebalanceConfig, StateTransitionTimeoutConfig stateTransitionTimeoutConfig,
+        Map<String, List<String>> listFields, Map<String, Map<String, String>> mapFields,
+        Boolean p2pMessageEnabled, Map<String, Map<String, Integer>> partitionCapacityMap) {
     super(resourceId);
 
     if (monitorDisabled != null) {
@@ -172,6 +195,15 @@ public class ResourceConfig extends HelixProperty {
     if (mapFields != null) {
       _record.setMapFields(mapFields);
     }
+
+    if (partitionCapacityMap != null) {
+      try {
+        setPartitionCapacityMap(partitionCapacityMap);
+      } catch (IOException e) {
+        throw new IllegalArgumentException(
+            "Failed to set partition capacity. Invalid capacity configuration.");
+      }
+    }
   }
 
 
@@ -350,6 +382,64 @@ public class ResourceConfig extends HelixProperty {
   }
 
   /**
+   * Get the partition capacity information from a JSON among the map fields.
+   * <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   *
+   * @return data map if it exists, or empty map
+   * @throws IOException - when JSON conversion fails
+   */
+  public Map<String, Map<String, Integer>> getPartitionCapacityMap() throws IOException {
+    Map<String, String> partitionCapacityData =
+        _record.getMapField(ResourceConfigProperty.PARTITION_CAPACITY_MAP.name());
+    Map<String, Map<String, Integer>> partitionCapacityMap = new HashMap<>();
+    if (partitionCapacityData != null) {
+      for (String partition : partitionCapacityData.keySet()) {
+        Map<String, Integer> capacities = _objectMapper
+            .readValue(partitionCapacityData.get(partition),
+                new TypeReference<Map<String, Integer>>() {
+                });
+        partitionCapacityMap.put(partition, capacities);
+      }
+    }
+    return partitionCapacityMap;
+  }
+
+  /**
+   * Set the partition capacity information with a map <PartitionName or DEFAULT_PARTITION_KEY, <Capacity Key, Capacity Number>>
+   *
+   * @param partitionCapacityMap - map of partition capacity data
+   * @throws IllegalArgumentException - when any of the data value is a negative number or map is empty
+   * @throws IOException              - when JSON parsing fails
+   */
+  public void setPartitionCapacityMap(Map<String, Map<String, Integer>> partitionCapacityMap)
+      throws IllegalArgumentException, IOException {
+    if (partitionCapacityMap == null || partitionCapacityMap.isEmpty()) {
+      throw new IllegalArgumentException("Capacity Map is empty");
+    }
+    if (!partitionCapacityMap.containsKey(DEFAULT_PARTITION_KEY)) {
+      throw new IllegalArgumentException(String
+          .format("The default partition capacity with the default key %s is required.",
+              DEFAULT_PARTITION_KEY));
+    }
+
+    Map<String, String> newCapacityRecord = new HashMap<>();
+    for (String partition : partitionCapacityMap.keySet()) {
+      Map<String, Integer> capacities = partitionCapacityMap.get(partition);
+      // Verify the input is valid
+      if (capacities.isEmpty()) {
+        throw new IllegalArgumentException("Capacity Data is empty");
+      }
+      if (capacities.entrySet().stream().anyMatch(entry -> entry.getValue() < 0)) {
+        throw new IllegalArgumentException(
+            String.format("Capacity Data contains a negative value:%s", capacities.toString()));
+      }
+      newCapacityRecord.put(partition, _objectMapper.writeValueAsString(capacities));
+    }
+
+    _record.setMapField(ResourceConfigProperty.PARTITION_CAPACITY_MAP.name(), newCapacityRecord);
+  }
+
+  /**
    * Put a set of simple configs.
    *
    * @param configsMap
@@ -476,6 +566,7 @@ public class ResourceConfig extends HelixProperty {
     private StateTransitionTimeoutConfig _stateTransitionTimeoutConfig;
     private Map<String, List<String>> _preferenceLists;
     private Map<String, Map<String, String>> _mapFields;
+    private Map<String, Map<String, Integer>> _partitionCapacityMap;
 
     public Builder(String resourceId) {
       _resourceId = resourceId;
@@ -664,6 +755,23 @@ public class ResourceConfig extends HelixProperty {
       return _preferenceLists;
     }
 
+    public Builder setPartitionCapacity(Map<String, Integer> defaultCapacity) {
+      setPartitionCapacity(DEFAULT_PARTITION_KEY, defaultCapacity);
+      return this;
+    }
+
+    public Builder setPartitionCapacity(String partition, Map<String, Integer> capacity) {
+      if (_partitionCapacityMap == null) {
+        _partitionCapacityMap = new HashMap<>();
+      }
+      _partitionCapacityMap.put(partition, capacity);
+      return this;
+    }
+
+    public Map<String, Integer> getPartitionCapacity(String partition) {
+      return _partitionCapacityMap.get(partition);
+    }
+
     public Builder setMapField(String key, Map<String, String> fields) {
       if (_mapFields == null) {
         _mapFields = new TreeMap<>();
@@ -708,6 +816,19 @@ public class ResourceConfig extends HelixProperty {
           }
         }
       }
+
+      if (_partitionCapacityMap != null) {
+        if (_partitionCapacityMap.keySet().stream()
+            .noneMatch(partition -> partition.equals(DEFAULT_PARTITION_KEY))) {
+          throw new IllegalArgumentException(
+              "Partition capacity is configured without the DEFAULT capacity!");
+        }
+        if (_partitionCapacityMap.values().stream()
+            .anyMatch(capacity -> capacity.values().stream().anyMatch(value -> value < 0))) {
+          throw new IllegalArgumentException(
+              "Partition capacity is configured with negative capacity value!");
+        }
+      }
     }
 
     public ResourceConfig build() {
@@ -718,7 +839,7 @@ public class ResourceConfig extends HelixProperty {
           _stateModelFactoryName, _numReplica, _minActiveReplica, _maxPartitionsPerInstance,
           _instanceGroupTag, _helixEnabled, _resourceGroupName, _resourceType, _groupRoutingEnabled,
           _externalViewDisabled, _rebalanceConfig, _stateTransitionTimeoutConfig, _preferenceLists,
-          _mapFields, _p2pMessageEnabled);
+          _mapFields, _p2pMessageEnabled, _partitionCapacityMap);
     }
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java b/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java
new file mode 100644
index 0000000..209b196
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/model/TestClusterConfig.java
@@ -0,0 +1,130 @@
+package org.apache.helix.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.
+ */
+
+import com.google.common.collect.ImmutableList;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.helix.model.ClusterConfig.GlobalRebalancePreferenceKey.EVENNESS;
+import static org.apache.helix.model.ClusterConfig.GlobalRebalancePreferenceKey.LESS_MOVEMENT;
+
+public class TestClusterConfig {
+
+  @Test
+  public void testGetCapacityKeys() {
+    List<String> keys = ImmutableList.of("CPU", "MEMORY", "Random");
+
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.getRecord()
+        .setListField(ClusterConfig.ClusterConfigProperty.INSTANCE_CAPACITY_KEYS.name(), keys);
+
+    Assert.assertEquals(testConfig.getInstanceCapacityKeys(), keys);
+  }
+
+  @Test
+  public void testGetCapacityKeysEmpty() {
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    Assert.assertEquals(testConfig.getInstanceCapacityKeys(), Collections.emptyList());
+  }
+
+  @Test
+  public void testSetCapacityKeys() {
+    List<String> keys = ImmutableList.of("CPU", "MEMORY", "Random");
+
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.setInstanceCapacityKeys(keys);
+
+    Assert.assertEquals(keys, testConfig.getRecord()
+        .getListField(ClusterConfig.ClusterConfigProperty.INSTANCE_CAPACITY_KEYS.name()));
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testSetCapacityKeysEmptyList() {
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.setInstanceCapacityKeys(Collections.emptyList());
+  }
+
+  @Test
+  public void testGetRebalancePreference() {
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+    preference.put(EVENNESS, 5);
+    preference.put(LESS_MOVEMENT, 3);
+
+    Map<String, String> mapFieldData = new HashMap<>();
+    for (ClusterConfig.GlobalRebalancePreferenceKey key : preference.keySet()) {
+      mapFieldData.put(key.name(), String.valueOf(preference.get(key)));
+    }
+
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.getRecord()
+        .setMapField(ClusterConfig.ClusterConfigProperty.REBALANCE_PREFERENCE.name(), mapFieldData);
+
+    Assert.assertEquals(testConfig.getGlobalRebalancePreference(), preference);
+  }
+
+  @Test
+  public void testGetRebalancePreferenceDefault() {
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    Assert.assertEquals(testConfig.getGlobalRebalancePreference(),
+        ClusterConfig.DEFAULT_GLOBAL_REBALANCE_PREFERENCE);
+
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+    preference.put(EVENNESS, 5);
+    testConfig.setGlobalRebalancePreference(preference);
+
+    Assert.assertEquals(testConfig.getGlobalRebalancePreference(),
+        ClusterConfig.DEFAULT_GLOBAL_REBALANCE_PREFERENCE);
+  }
+
+  @Test
+  public void testSetRebalancePreference() {
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+    preference.put(EVENNESS, 5);
+    preference.put(LESS_MOVEMENT, 3);
+
+    Map<String, String> mapFieldData = new HashMap<>();
+    for (ClusterConfig.GlobalRebalancePreferenceKey key : preference.keySet()) {
+      mapFieldData.put(key.name(), String.valueOf(preference.get(key)));
+    }
+
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.setGlobalRebalancePreference(preference);
+
+    Assert.assertEquals(testConfig.getRecord()
+            .getMapField(ClusterConfig.ClusterConfigProperty.REBALANCE_PREFERENCE.name()),
+        mapFieldData);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class)
+  public void testSetRebalancePreferenceInvalidNumber() {
+    Map<ClusterConfig.GlobalRebalancePreferenceKey, Integer> preference = new HashMap<>();
+    preference.put(EVENNESS, -1);
+    preference.put(LESS_MOVEMENT, 3);
+
+    ClusterConfig testConfig = new ClusterConfig("testId");
+    testConfig.setGlobalRebalancePreference(preference);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestInstanceConfig.java b/helix-core/src/test/java/org/apache/helix/model/TestInstanceConfig.java
index 38b1c92..f0da05f 100644
--- a/helix-core/src/test/java/org/apache/helix/model/TestInstanceConfig.java
+++ b/helix-core/src/test/java/org/apache/helix/model/TestInstanceConfig.java
@@ -19,12 +19,14 @@ package org.apache.helix.model;
  * under the License.
  */
 
-import java.util.Map;
-
+import com.google.common.collect.ImmutableMap;
 import org.apache.helix.ZNRecord;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Created with IntelliJ IDEA.
@@ -58,4 +60,64 @@ public class TestInstanceConfig {
     Map<String, String> parsedDomain = instanceConfig.getDomainAsMap();
     Assert.assertTrue(parsedDomain.isEmpty());
   }
+
+  @Test
+  public void testGetInstanceCapacityMap() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    Map<String, String> capacityDataMapString = ImmutableMap.of("item1", "1",
+        "item2", "2",
+        "item3", "3");
+
+    ZNRecord rec = new ZNRecord("testId");
+    rec.setMapField(InstanceConfig.InstanceConfigProperty.INSTANCE_CAPACITY_MAP.name(), capacityDataMapString);
+    InstanceConfig testConfig = new InstanceConfig(rec);
+
+    Assert.assertTrue(testConfig.getInstanceCapacityMap().equals(capacityDataMap));
+  }
+
+  @Test
+  public void testGetInstanceCapacityMapEmpty() {
+    InstanceConfig testConfig = new InstanceConfig("testId");
+
+    Assert.assertTrue(testConfig.getInstanceCapacityMap().equals(Collections.emptyMap()));
+  }
+
+  @Test
+  public void testSetInstanceCapacityMap() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    Map<String, String> capacityDataMapString = ImmutableMap.of("item1", "1",
+        "item2", "2",
+        "item3", "3");
+
+    InstanceConfig testConfig = new InstanceConfig("testConfig");
+    testConfig.setInstanceCapacityMap(capacityDataMap);
+
+    Assert.assertEquals(testConfig.getRecord().getMapField(InstanceConfig.InstanceConfigProperty.
+        INSTANCE_CAPACITY_MAP.name()), capacityDataMapString);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Capacity Data is empty")
+  public void testSetInstanceCapacityMapEmpty() {
+    Map<String, Integer> capacityDataMap = new HashMap<>();
+
+    InstanceConfig testConfig = new InstanceConfig("testConfig");
+    testConfig.setInstanceCapacityMap(capacityDataMap);
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class,
+      expectedExceptionsMessageRegExp = "Capacity Data contains a negative value: item3 = -3")
+  public void testSetInstanceCapacityMapInvalid() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", -3);
+
+    InstanceConfig testConfig = new InstanceConfig("testConfig");
+    testConfig.setInstanceCapacityMap(capacityDataMap);
+  }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestResourceConfig.java b/helix-core/src/test/java/org/apache/helix/model/TestResourceConfig.java
new file mode 100644
index 0000000..8099486
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/model/TestResourceConfig.java
@@ -0,0 +1,186 @@
+package org.apache.helix.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.
+ */
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.helix.ZNRecord;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestResourceConfig {
+  private static final ObjectMapper _objectMapper = new ObjectMapper();
+
+  @Test
+  public void testGetPartitionCapacityMap() throws IOException {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    ZNRecord rec = new ZNRecord("testId");
+    rec.setMapField(ResourceConfig.ResourceConfigProperty.PARTITION_CAPACITY_MAP.name(), Collections
+        .singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY,
+            _objectMapper.writeValueAsString(capacityDataMap)));
+    ResourceConfig testConfig = new ResourceConfig(rec);
+
+    Assert.assertTrue(testConfig.getPartitionCapacityMap().get(ResourceConfig.DEFAULT_PARTITION_KEY)
+        .equals(capacityDataMap));
+  }
+
+  @Test
+  public void testGetPartitionCapacityMapEmpty() throws IOException {
+    ResourceConfig testConfig = new ResourceConfig("testId");
+
+    Assert.assertTrue(testConfig.getPartitionCapacityMap().equals(Collections.emptyMap()));
+  }
+
+  @Test(expectedExceptions = IOException.class)
+  public void testGetPartitionCapacityMapInvalidJson() throws IOException {
+    ZNRecord rec = new ZNRecord("testId");
+    rec.setMapField(ResourceConfig.ResourceConfigProperty.PARTITION_CAPACITY_MAP.name(),
+        Collections.singletonMap("test", "gibberish"));
+    ResourceConfig testConfig = new ResourceConfig(rec);
+
+    testConfig.getPartitionCapacityMap();
+  }
+
+  @Test(dependsOnMethods = "testGetPartitionCapacityMap", expectedExceptions = IOException.class)
+  public void testGetPartitionCapacityMapInvalidJsonType() throws IOException {
+    Map<String, String> capacityDataMap = ImmutableMap.of("item1", "1",
+        "item2", "2",
+        "item3", "three");
+
+    ZNRecord rec = new ZNRecord("testId");
+    rec.setMapField(ResourceConfig.ResourceConfigProperty.PARTITION_CAPACITY_MAP.name(), Collections
+        .singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY,
+            _objectMapper.writeValueAsString(capacityDataMap)));
+    ResourceConfig testConfig = new ResourceConfig(rec);
+
+    testConfig.getPartitionCapacityMap();
+  }
+
+  @Test
+  public void testSetPartitionCapacityMap() throws IOException {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    ResourceConfig testConfig = new ResourceConfig("testConfig");
+    testConfig.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMap));
+
+    Assert.assertEquals(testConfig.getRecord().getMapField(ResourceConfig.ResourceConfigProperty.
+            PARTITION_CAPACITY_MAP.name()).get(ResourceConfig.DEFAULT_PARTITION_KEY),
+        _objectMapper.writeValueAsString(capacityDataMap));
+  }
+
+  @Test
+  public void testSetMultiplePartitionCapacityMap() throws IOException {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    Map<String, Map<String, Integer>> totalCapacityMap =
+        ImmutableMap.of(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMap,
+        "partition2", capacityDataMap,
+        "partition3", capacityDataMap);
+
+    ResourceConfig testConfig = new ResourceConfig("testConfig");
+    testConfig.setPartitionCapacityMap(totalCapacityMap);
+
+    Assert.assertNull(testConfig.getRecord().getMapField(ResourceConfig.ResourceConfigProperty.
+        PARTITION_CAPACITY_MAP.name()).get("partition1"));
+    Assert.assertEquals(testConfig.getRecord().getMapField(ResourceConfig.ResourceConfigProperty.
+        PARTITION_CAPACITY_MAP.name()).get(ResourceConfig.DEFAULT_PARTITION_KEY),
+        _objectMapper.writeValueAsString(capacityDataMap));
+    Assert.assertEquals(testConfig.getRecord().getMapField(ResourceConfig.ResourceConfigProperty.
+            PARTITION_CAPACITY_MAP.name()).get("partition2"),
+        _objectMapper.writeValueAsString(capacityDataMap));
+    Assert.assertEquals(testConfig.getRecord().getMapField(ResourceConfig.ResourceConfigProperty.
+            PARTITION_CAPACITY_MAP.name()).get("partition3"),
+        _objectMapper.writeValueAsString(capacityDataMap));
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Capacity Data is empty")
+  public void testSetPartitionCapacityMapEmpty() throws IOException {
+    Map<String, Integer> capacityDataMap = new HashMap<>();
+
+    ResourceConfig testConfig = new ResourceConfig("testConfig");
+    testConfig.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMap));
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "The default partition capacity with the default key DEFAULT is required.")
+  public void testSetPartitionCapacityMapWithoutDefault() throws IOException {
+    Map<String, Integer> capacityDataMap = new HashMap<>();
+
+    ResourceConfig testConfig = new ResourceConfig("testConfig");
+    testConfig.setPartitionCapacityMap(
+        Collections.singletonMap("Random", capacityDataMap));
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Capacity Data contains a negative value:.+")
+  public void testSetPartitionCapacityMapInvalid() throws IOException {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", -3);
+
+    ResourceConfig testConfig = new ResourceConfig("testConfig");
+    testConfig.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMap));
+  }
+
+  @Test
+  public void testWithResourceBuilder() throws IOException {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    ResourceConfig.Builder builder = new ResourceConfig.Builder("testConfig");
+    builder.setPartitionCapacity(capacityDataMap);
+    builder.setPartitionCapacity("partition1", capacityDataMap);
+
+    Assert.assertEquals(
+        builder.build().getPartitionCapacityMap().get(ResourceConfig.DEFAULT_PARTITION_KEY),
+        capacityDataMap);
+    Assert.assertEquals(
+        builder.build().getPartitionCapacityMap().get("partition1"),
+        capacityDataMap);
+    Assert.assertNull(
+        builder.build().getPartitionCapacityMap().get("Random"));
+  }
+
+  @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "The default partition capacity with the default key DEFAULT is required.")
+  public void testWithResourceBuilderInvalidInput() {
+    Map<String, Integer> capacityDataMap = ImmutableMap.of("item1", 1,
+        "item2", 2,
+        "item3", 3);
+
+    ResourceConfig.Builder builder = new ResourceConfig.Builder("testConfig");
+    builder.setPartitionCapacity("Random", capacityDataMap);
+
+    builder.build();
+  }
+}


[helix] 06/37: Implement Cluster Model Provider. (#392)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit dc828b8a336aa679d6b5e55f702137e63aaef1c5
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Tue Aug 13 11:25:23 2019 -0700

    Implement Cluster Model Provider. (#392)
    
    * Implement Cluster Model Provider.
    
    The model provider is called in the WAGED rebalancer to generate CLuster Model based on the current cluster status.
    The major responsibility of the provider is to parse all the assignable replicas and identify which replicas need to be reassigned. Note that if the current best possible assignment is still valid, the rebalancer won't need to calculate for the partition assignment.
    
    Also, add unit tests to verify the main logic.
---
 .../rebalancer/waged/ClusterDataDetector.java      |   3 +-
 .../rebalancer/waged/model/AssignableNode.java     |   2 +-
 .../rebalancer/waged/model/ClusterModel.java       |   1 -
 .../waged/model/ClusterModelProvider.java          | 247 +++++++++++++++++++++
 .../waged/model/AbstractTestClusterModel.java      |  29 ++-
 .../waged/model/TestClusterModelProvider.java      | 247 +++++++++++++++++++++
 6 files changed, 517 insertions(+), 12 deletions(-)

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
index 07f16dd..0423edf 100644
--- 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
@@ -35,11 +35,12 @@ public class ClusterDataDetector<T extends BaseControllerDataProvider> {
    * All the cluster change type that may trigger a WAGED rebalancer re-calculation.
    */
   public enum ChangeType {
+    BaselineAssignmentChange,
     InstanceConfigChange,
     ClusterConfigChange,
     ResourceConfigChange,
-    InstanceStateChange,
     ResourceIdealStatesChange,
+    InstanceStateChange,
     OtherChange
   }
 
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
index 989323e..5fc04d7 100644
--- 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
@@ -52,7 +52,7 @@ public class AssignableNode {
   private int _maxPartition; // maximum number of the partitions that can be assigned to the node.
 
   // proposed assignment tracking
-  // <resource name, partition name>
+  // <resource name, partition name set>
   private Map<String, Set<String>> _currentAssignments;
   // <resource name, top state partition name>
   private Map<String, Set<String>> _currentTopStateAssignments;
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
index 1be527a..6c4e67b 100644
--- 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
@@ -20,7 +20,6 @@ package org.apache.helix.controller.rebalancer.waged.model;
  */
 
 import org.apache.helix.HelixException;
-import org.apache.helix.model.IdealState;
 import org.apache.helix.model.ResourceAssignment;
 
 import java.util.Collections;
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
new file mode 100644
index 0000000..9de023b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -0,0 +1,247 @@
+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.
+ */
+
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.model.StateModelDefinition;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This util class generates Cluster Model object based on the controller's data cache.
+ */
+public class ClusterModelProvider {
+
+  /**
+   * @param dataProvider           The controller's data cache.
+   * @param resourceMap            The full list of the resources to be rebalanced. Note that any
+   *                               resources that are not in this list will be removed from the
+   *                               final assignment.
+   * @param activeInstances        The 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 Generate a new Cluster Model object according to the current cluster status.
+   */
+  public static ClusterModel generateClusterModel(ResourceControllerDataProvider dataProvider,
+      Map<String, Resource> resourceMap, Set<String> activeInstances,
+      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
+      Map<String, ResourceAssignment> baselineAssignment,
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
+    // Generate replica objects for all the resource partitions.
+    // <resource, replica set>
+    Map<String, Set<AssignableReplica>> replicaMap =
+        parseAllReplicas(dataProvider, resourceMap, activeInstances.size());
+
+    // Check if the replicas need to be reassigned.
+    Map<String, Set<AssignableReplica>> allocatedReplicas =
+        new HashMap<>(); // <instanceName, replica set>
+    Set<AssignableReplica> toBeAssignedReplicas =
+        findToBeAssignedReplicas(replicaMap, clusterChanges, activeInstances,
+            bestPossibleAssignment, allocatedReplicas);
+
+    // Construct all the assignable nodes and initialize with the allocated replicas.
+    Set<AssignableNode> assignableNodes =
+        parseAllNodes(dataProvider.getClusterConfig(), dataProvider.getInstanceConfigMap(),
+            activeInstances, allocatedReplicas);
+
+    // Construct and initialize cluster context.
+    ClusterContext context = new ClusterContext(
+        replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()),
+        activeInstances.size());
+    // Initial the cluster context with the allocated assignments.
+    context.setAssignmentForFaultZoneMap(mapAssignmentToFaultZone(assignableNodes));
+
+    return new ClusterModel(context, toBeAssignedReplicas, assignableNodes, baselineAssignment,
+        bestPossibleAssignment);
+  }
+
+  /**
+   * Find the minimum set of replicas that need to be reassigned.
+   * A replica needs to be reassigned if one of the following condition is true:
+   * 1. Cluster topology (the cluster config / any instance config) has been updated.
+   * 2. The baseline assignment has been updated.
+   * 3. The resource config has been updated.
+   * 4. The resource idealstate has been updated. TODO remove this condition when all resource configurations are migrated to resource config.
+   * 5. If the current best possible assignment does not contain the partition's valid assignment.
+   *
+   * @param replicaMap             A map contains all the replicas grouped by resource name.
+   * @param clusterChanges         A map contains all the important metadata updates that happened after the previous rebalance.
+   * @param activeInstances        All the instances that are alive and enabled.
+   * @param bestPossibleAssignment The current best possible assignment.
+   * @param allocatedReplicas      Return the allocated replicas grouped by the target instance name.
+   * @return The replicas that need to be reassigned.
+   */
+  private static Set<AssignableReplica> findToBeAssignedReplicas(
+      Map<String, Set<AssignableReplica>> replicaMap,
+      Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges, Set<String> activeInstances,
+      Map<String, ResourceAssignment> bestPossibleAssignment,
+      Map<String, Set<AssignableReplica>> allocatedReplicas) {
+    Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
+    if (clusterChanges.containsKey(ClusterDataDetector.ChangeType.ClusterConfigChange)
+        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.InstanceConfigChange)
+        || clusterChanges.containsKey(ClusterDataDetector.ChangeType.BaselineAssignmentChange)) {
+      // If the cluster topology or baseline assignment has been modified, need to reassign all replicas
+      toBeAssignedReplicas
+          .addAll(replicaMap.values().stream().flatMap(Set::stream).collect(Collectors.toSet()));
+    } else {
+      // check each resource to identify the allocated replicas and to-be-assigned replicas.
+      for (String resourceName : replicaMap.keySet()) {
+        Set<AssignableReplica> replicas = replicaMap.get(resourceName);
+        // 1. if the resource config/idealstate is changed, need to reassign.
+        // 2. if the resource does appear in the best possible assignment, need to reassign.
+        if (clusterChanges.getOrDefault(ClusterDataDetector.ChangeType.ResourceConfigChange,
+            Collections.emptySet()).contains(resourceName) || clusterChanges
+            .getOrDefault(ClusterDataDetector.ChangeType.ResourceIdealStatesChange,
+                Collections.emptySet()).contains(resourceName) || !bestPossibleAssignment
+            .containsKey(resourceName)) {
+          toBeAssignedReplicas.addAll(replicas);
+          continue; // go to check next resource
+        } else {
+          // check for every best possible assignments to identify if the related replicas need to reassign.
+          ResourceAssignment assignment = bestPossibleAssignment.get(resourceName);
+          // <partition, <instance, state>>
+          Map<String, Map<String, String>> stateMap = assignment.getMappedPartitions().stream()
+              .collect(Collectors.toMap(partition -> partition.getPartitionName(),
+                  partition -> new HashMap<>(assignment.getReplicaMap(partition))));
+          for (AssignableReplica replica : replicas) {
+            // Find any ACTIVE instance allocation that has the same state with the replica
+            Optional<Map.Entry<String, String>> instanceNameOptional =
+                stateMap.getOrDefault(replica.getPartitionName(), Collections.emptyMap()).entrySet()
+                    .stream().filter(instanceStateMap ->
+                    instanceStateMap.getValue().equals(replica.getReplicaState()) && activeInstances
+                        .contains(instanceStateMap.getKey())).findAny();
+            // 3. if no such an instance in the bestPossible assignment, need to reassign the replica
+            if (!instanceNameOptional.isPresent()) {
+              toBeAssignedReplicas.add(replica);
+              continue; // go to check the next replica
+            } else {
+              String instanceName = instanceNameOptional.get().getKey();
+              // * cleanup the best possible state map record,
+              // * so the selected instance won't be picked up again for the another replica check
+              stateMap.getOrDefault(replica.getPartitionName(), Collections.emptyMap())
+                  .remove(instanceName);
+              // the current best possible assignment for this replica is valid,
+              // add to the allocated replica list.
+              allocatedReplicas.computeIfAbsent(instanceName, key -> new HashSet<>()).add(replica);
+            }
+          }
+        }
+      }
+    }
+    return toBeAssignedReplicas;
+  }
+
+  /**
+   * Parse all the nodes that can be assigned replicas based on the configurations.
+   *
+   * @param clusterConfig     The cluster configuration.
+   * @param instanceConfigMap A map of all the instance configuration.
+   * @param activeInstances   All the instances that are online and enabled.
+   * @param allocatedReplicas A map of all the assigned replicas, which will not be reassigned during the rebalance.
+   * @return A map of assignable node set, <InstanceName, node set>.
+   */
+  private static Set<AssignableNode> parseAllNodes(ClusterConfig clusterConfig,
+      Map<String, InstanceConfig> instanceConfigMap, Set<String> activeInstances,
+      Map<String, Set<AssignableReplica>> allocatedReplicas) {
+    return activeInstances.stream().map(
+        instanceName -> new AssignableNode(clusterConfig, instanceConfigMap.get(instanceName),
+            instanceName, allocatedReplicas.getOrDefault(instanceName, Collections.emptySet())))
+        .collect(Collectors.toSet());
+  }
+
+  /**
+   * Parse all the replicas that need to be reallocated from the cluster data cache.
+   *
+   * @param dataProvider The cluster status cache that contains the current cluster status.
+   * @param resourceMap  All the valid resources that are managed by the rebalancer.
+   * @return A map of assignable replica set, <ResourceName, replica set>.
+   */
+  private static Map<String, Set<AssignableReplica>> parseAllReplicas(
+      ResourceControllerDataProvider dataProvider, Map<String, Resource> resourceMap,
+      int instanceCount) {
+    Map<String, Set<AssignableReplica>> totalReplicaMap = new HashMap<>();
+
+    for (String resourceName : resourceMap.keySet()) {
+      ResourceConfig config = dataProvider.getResourceConfig(resourceName);
+      IdealState is = dataProvider.getIdealState(resourceName);
+      if (is == null) {
+        throw new HelixException(
+            "Cannot find the resource ideal state for resource: " + resourceName);
+      }
+      String defName = is.getStateModelDefRef();
+      StateModelDefinition def = dataProvider.getStateModelDef(defName);
+      if (def == null) {
+        throw new IllegalArgumentException(String
+            .format("Cannot find state model definition %s for resource %s.",
+                is.getStateModelDefRef(), resourceName));
+      }
+
+      Map<String, Integer> stateCountMap =
+          def.getStateCountMap(instanceCount, is.getReplicaCount(instanceCount));
+
+      for (String partition : is.getPartitionSet()) {
+        for (Map.Entry<String, Integer> entry : stateCountMap.entrySet()) {
+          String state = entry.getKey();
+          for (int i = 0; i < entry.getValue(); i++) {
+            totalReplicaMap.computeIfAbsent(resourceName, key -> new HashSet<>()).add(
+                new AssignableReplica(config, partition, state,
+                    def.getStatePriorityMap().get(state)));
+          }
+        }
+      }
+    }
+    return totalReplicaMap;
+  }
+
+  /**
+   * @return A map contains the assignments for each fault zone. <fault zone, <resource, set of partitions>>
+   */
+  private static Map<String, Map<String, Set<String>>> mapAssignmentToFaultZone(
+      Set<AssignableNode> assignableNodes) {
+    Map<String, Map<String, Set<String>>> faultZoneAssignmentMap = new HashMap<>();
+    assignableNodes.stream().forEach(node -> {
+      for (Map.Entry<String, Set<String>> resourceMap : node.getCurrentAssignmentsMap()
+          .entrySet()) {
+        faultZoneAssignmentMap.computeIfAbsent(node.getFaultZone(), k -> new HashMap<>())
+            .computeIfAbsent(resourceMap.getKey(), k -> new HashSet<>())
+            .addAll(resourceMap.getValue());
+      }
+    });
+    return faultZoneAssignmentMap;
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
index 0e2b43a..d99a3fb 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -41,6 +41,7 @@ import java.util.Set;
 import static org.mockito.Mockito.when;
 
 public abstract class AbstractTestClusterModel {
+  protected static String _sessionId = "testSessionId";
   protected String _testInstanceId;
   protected List<String> _resourceNames;
   protected List<String> _partitionNames;
@@ -73,16 +74,27 @@ public abstract class AbstractTestClusterModel {
     _testFaultZoneId = "testZone";
   }
 
+  InstanceConfig createMockInstanceConfig(String instanceId) {
+    InstanceConfig testInstanceConfig = new InstanceConfig(instanceId);
+    testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
+    testInstanceConfig.addTag(_testInstanceTags.get(0));
+    testInstanceConfig.setInstanceEnabled(true);
+    testInstanceConfig.setZoneId(_testFaultZoneId);
+    return testInstanceConfig;
+  }
+
+  LiveInstance createMockLiveInstance(String instanceId) {
+    LiveInstance testLiveInstance = new LiveInstance(instanceId);
+    testLiveInstance.setSessionId(_sessionId);
+    return testLiveInstance;
+  }
+
   protected ResourceControllerDataProvider setupClusterDataCache() throws IOException {
     ResourceControllerDataProvider testCache = Mockito.mock(ResourceControllerDataProvider.class);
 
     // 1. Set up the default instance information with capacity configuration.
-    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceId");
-    testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
-    testInstanceConfig.addTag(_testInstanceTags.get(0));
+    InstanceConfig testInstanceConfig = createMockInstanceConfig(_testInstanceId);
     testInstanceConfig.setInstanceEnabledForPartition("TestResource", "TestPartition", false);
-    testInstanceConfig.setInstanceEnabled(true);
-    testInstanceConfig.setZoneId(_testFaultZoneId);
     Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
     instanceConfigMap.put(_testInstanceId, testInstanceConfig);
     when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
@@ -95,8 +107,7 @@ public abstract class AbstractTestClusterModel {
     when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
 
     // 3. Mock the live instance node for the default instance.
-    LiveInstance testLiveInstance = new LiveInstance(_testInstanceId);
-    testLiveInstance.setSessionId("testSessionId");
+    LiveInstance testLiveInstance = createMockLiveInstance(_testInstanceId);
     Map<String, LiveInstance> liveInstanceMap = new HashMap<>();
     liveInstanceMap.put(_testInstanceId, testLiveInstance);
     when(testCache.getLiveInstances()).thenReturn(liveInstanceMap);
@@ -130,7 +141,7 @@ public abstract class AbstractTestClusterModel {
     Map<String, CurrentState> currentStatemap = new HashMap<>();
     currentStatemap.put(_resourceNames.get(0), testCurrentStateResource1);
     currentStatemap.put(_resourceNames.get(1), testCurrentStateResource2);
-    when(testCache.getCurrentState(_testInstanceId, "testSessionId")).thenReturn(currentStatemap);
+    when(testCache.getCurrentState(_testInstanceId, _sessionId)).thenReturn(currentStatemap);
 
     // 5. Set up the resource config for the two resources with the partition weight.
     Map<String, Integer> capacityDataMapResource1 = new HashMap<>();
@@ -162,7 +173,7 @@ public abstract class AbstractTestClusterModel {
   protected Set<AssignableReplica> generateReplicas(ResourceControllerDataProvider dataProvider) {
     // Create assignable replica based on the current state.
     Map<String, CurrentState> currentStatemap =
-        dataProvider.getCurrentState(_testInstanceId, "testSessionId");
+        dataProvider.getCurrentState(_testInstanceId, _sessionId);
     Set<AssignableReplica> assignmentSet = new HashSet<>();
     for (CurrentState cs : currentStatemap.values()) {
       ResourceConfig resourceConfig = dataProvider.getResourceConfig(cs.getResourceName());
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
new file mode 100644
index 0000000..f92a66c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -0,0 +1,247 @@
+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.
+ */
+
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.waged.ClusterDataDetector;
+import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceAssignment;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
+
+public class TestClusterModelProvider extends AbstractTestClusterModel {
+  Set<String> _instances;
+
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+    _instances = new HashSet<>();
+    _instances.add(_testInstanceId);
+  }
+
+  @Override
+  protected ResourceControllerDataProvider setupClusterDataCache() throws IOException {
+    ResourceControllerDataProvider testCache = super.setupClusterDataCache();
+
+    // Set up mock idealstate
+    Map<String, IdealState> isMap = new HashMap<>();
+    for (String resource : _resourceNames) {
+      IdealState is = new IdealState(resource);
+      is.setNumPartitions(_partitionNames.size());
+      is.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO);
+      is.setStateModelDefRef("MasterSlave");
+      is.setReplicas("3");
+      is.setRebalancerClassName(WagedRebalancer.class.getName());
+      _partitionNames.stream()
+          .forEach(partition -> is.setPreferenceList(partition, Collections.emptyList()));
+      isMap.put(resource, is);
+    }
+    when(testCache.getIdealState(anyString())).thenAnswer(
+        (Answer<IdealState>) invocationOnMock -> isMap.get(invocationOnMock.getArguments()[0]));
+
+    // Set up 2 more instances
+    for (int i = 1; i < 3; i++) {
+      String instanceName = _testInstanceId + i;
+      _instances.add(instanceName);
+      // 1. Set up the default instance information with capacity configuration.
+      InstanceConfig testInstanceConfig = createMockInstanceConfig(instanceName);
+      Map<String, InstanceConfig> instanceConfigMap = testCache.getInstanceConfigMap();
+      instanceConfigMap.put(instanceName, testInstanceConfig);
+      when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+      // 2. Mock the live instance node for the default instance.
+      LiveInstance testLiveInstance = createMockLiveInstance(instanceName);
+      Map<String, LiveInstance> liveInstanceMap = testCache.getLiveInstances();
+      liveInstanceMap.put(instanceName, testLiveInstance);
+      when(testCache.getLiveInstances()).thenReturn(liveInstanceMap);
+    }
+
+    return testCache;
+  }
+
+  @Test
+  public void testGenerateClusterModel() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    // 1. test generating a cluster model with empty assignment
+    ClusterModel clusterModel = ClusterModelProvider.generateClusterModel(testCache,
+        _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        _instances, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
+    // There should be no existing assignment.
+    Assert.assertFalse(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .anyMatch(resourceMap -> !resourceMap.isEmpty()));
+    Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
+        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+    // Have all 3 instances
+    Assert.assertEquals(
+        clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)
+            .collect(Collectors.toSet()), _instances);
+    // Shall have 2 resources and 12 replicas
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 12));
+
+    // 2. test with only one active node
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        Collections.singleton(_testInstanceId), Collections.emptyMap(), Collections.emptyMap(),
+        Collections.emptyMap());
+    // Have only one instance
+    Assert.assertEquals(
+        clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)
+            .collect(Collectors.toSet()), Collections.singleton(_testInstanceId));
+    // Shall have 4 assignable replicas because there is only one valid node.
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 4));
+
+    // 3. test with no active instance
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        Collections.emptySet(), Collections.emptyMap(), Collections.emptyMap(),
+        Collections.emptyMap());
+    // Have only one instance
+    Assert.assertEquals(clusterModel.getAssignableNodes().size(), 0);
+    // Shall have 0 assignable replicas because there is only n0 valid node.
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.isEmpty()));
+
+    // 4. test with best possible assignment
+    // Mock a best possible assignment based on the current states.
+    Map<String, ResourceAssignment> bestPossibleAssignment = new HashMap<>();
+    for (String resource : _resourceNames) {
+      // <partition, <instance, state>>
+      Map<String, Map<String, String>> assignmentMap = new HashMap<>();
+      CurrentState cs = testCache.getCurrentState(_testInstanceId, _sessionId).get(resource);
+      if (cs != null) {
+        for (Map.Entry<String, String> stateEntry : cs.getPartitionStateMap().entrySet()) {
+          assignmentMap.computeIfAbsent(stateEntry.getKey(), k -> new HashMap<>())
+              .put(_testInstanceId, stateEntry.getValue());
+        }
+        ResourceAssignment assignment = new ResourceAssignment(resource);
+        assignmentMap.keySet().stream().forEach(partition -> assignment
+            .addReplicaMap(new Partition(partition), assignmentMap.get(partition)));
+        bestPossibleAssignment.put(resource, assignment);
+      }
+    }
+
+    // Generate a cluster model based on the best possible assignment
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        _instances, Collections.emptyMap(), Collections.emptyMap(), bestPossibleAssignment);
+    // There should be 4 existing assignments in total (each resource has 2) in the specified instance
+    Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .allMatch(resourceMap -> resourceMap.values().stream()
+            .allMatch(partitionSet -> partitionSet.size() == 2)));
+    Assert.assertEquals(
+        clusterModel.getAssignableNodes().get(_testInstanceId).getCurrentAssignmentCount(), 4);
+    // Since each resource has 2 replicas assigned, the assignable replica count should be 10.
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 10));
+
+    // 5. test with best possible assignment but cluster topology is changed
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ClusterConfigChange,
+            Collections.emptySet()), Collections.emptyMap(), bestPossibleAssignment);
+    // There should be no existing assignment since the topology change invalidates all existing assignment
+    Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .allMatch(resourceMap -> resourceMap.isEmpty()));
+    Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
+        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+    // Shall have 2 resources and 12 replicas
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 12));
+
+    // 6. test with best possible assignment and one resource config change
+    // Generate a cluster model based on the same best possible assignment, but resource1 config is changed
+    String changedResourceName = _resourceNames.get(0);
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        _instances, Collections.singletonMap(ClusterDataDetector.ChangeType.ResourceConfigChange,
+            Collections.singleton(changedResourceName)), Collections.emptyMap(),
+        bestPossibleAssignment);
+    // There should be no existing assignment for all the resource except for resource2.
+    Assert.assertEquals(clusterModel.getContext().getAssignmentForFaultZoneMap().size(), 1);
+    Map<String, Set<String>> resourceAssignmentMap =
+        clusterModel.getContext().getAssignmentForFaultZoneMap().get(_testFaultZoneId);
+    // Should be only resource2 in the map
+    Assert.assertEquals(resourceAssignmentMap.size(), 1);
+    for (String resource : _resourceNames) {
+      Assert
+          .assertEquals(resourceAssignmentMap.getOrDefault(resource, Collections.emptySet()).size(),
+              resource.equals(changedResourceName) ? 0 : 2);
+    }
+    // Only the first instance will have 2 assignment from resource2.
+    for (String instance : _instances) {
+      Assert
+          .assertEquals(clusterModel.getAssignableNodes().get(instance).getCurrentAssignmentCount(),
+              instance.equals(_testInstanceId) ? 2 : 0);
+    }
+    // Shall have 2 resources and 12 replicas
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().keySet().size(), 2);
+    for (String resource : _resourceNames) {
+      Assert.assertEquals(clusterModel.getAssignableReplicaMap().get(resource).size(),
+          resource.equals(changedResourceName) ? 12 : 10);
+    }
+
+    // 7. test with best possible assignment but the instance becomes inactive
+    // Generate a cluster model based on the best possible assignment, but the assigned node is disabled
+    Set<String> limitedActiveInstances = new HashSet<>(_instances);
+    limitedActiveInstances.remove(_testInstanceId);
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        limitedActiveInstances, Collections.emptyMap(), Collections.emptyMap(),
+        bestPossibleAssignment);
+    // There should be no existing assignment.
+    Assert.assertFalse(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .anyMatch(resourceMap -> !resourceMap.isEmpty()));
+    Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
+        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+    // Have only 2 instances
+    Assert.assertEquals(
+        clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)
+            .collect(Collectors.toSet()), limitedActiveInstances);
+    // Since only 2 instances are active, we shall have 8 assignable replicas in each resource.
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 8));
+
+  }
+}


[helix] 32/37: Separate AssignableNode properties by Immutable and Mutable (#485)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0b87426cfd92ab319234d6b988269b49a47107ae
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Sat Sep 28 00:49:13 2019 -0700

    Separate AssignableNode properties by Immutable and Mutable (#485)
    
    Make AssignableNode properties different by Immutable and Mutable
    - It helps detect any wrong usage of these properties early
---
 .../waged/constraints/NodeCapacityConstraint.java  |   2 +-
 .../rebalancer/waged/model/AssignableNode.java     | 119 ++++++++++-----------
 .../waged/model/ClusterModelProvider.java          |   2 +-
 .../constraints/TestNodeCapacityConstraint.java    |   4 +-
 .../rebalancer/waged/model/TestAssignableNode.java |  15 ++-
 5 files changed, 66 insertions(+), 76 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java
index 5fc2faf..827d6ce 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/NodeCapacityConstraint.java
@@ -30,7 +30,7 @@ class NodeCapacityConstraint extends HardConstraint {
   @Override
   boolean isAssignmentValid(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext) {
-    Map<String, Integer> nodeCapacity = node.getCurrentCapacity();
+    Map<String, Integer> nodeCapacity = node.getRemainingCapacity();
     Map<String, Integer> replicaCapacity = replica.getCapacity();
 
     for (String key : replicaCapacity.keySet()) {
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
index 20de6da..2a68e15 100644
--- 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
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -35,6 +34,10 @@ import org.apache.helix.model.InstanceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
 /**
  * This class represents a possible allocation of the replication.
  * Note that any usage updates to the AssignableNode are not thread safe.
@@ -42,39 +45,25 @@ import org.slf4j.LoggerFactory;
 public class AssignableNode implements Comparable<AssignableNode> {
   private static final Logger LOG = LoggerFactory.getLogger(AssignableNode.class.getName());
 
-  // basic node information
+  // Immutable Instance Properties
   private final String _instanceName;
-  private Set<String> _instanceTags;
-  private String _faultZone;
-  private Map<String, List<String>> _disabledPartitionsMap;
-  private Map<String, Integer> _maxCapacity;
-  private int _maxPartition; // maximum number of the partitions that can be assigned to the node.
-
+  private final String _faultZone;
+  // maximum number of the partitions that can be assigned to the instance.
+  private final int _maxPartition;
+  private final ImmutableSet<String> _instanceTags;
+  private final ImmutableMap<String, List<String>> _disabledPartitionsMap;
+  private final ImmutableMap<String, Integer> _maxAllowedCapacity;
+
+  // Mutable (Dynamic) Instance Properties
   // A map of <resource name, <partition name, replica>> that tracks the replicas assigned to the
   // node.
   private Map<String, Map<String, AssignableReplica>> _currentAssignedReplicaMap;
   // A map of <capacity key, capacity value> that tracks the current available node capacity
-  private Map<String, Integer> _currentCapacityMap;
+  private Map<String, Integer> _remainingCapacity;
   // The maximum capacity utilization (0.0 - 1.0) across all the capacity categories.
   private float _highestCapacityUtilization;
 
   /**
-   * @param clusterConfig
-   * @param instanceConfig
-   * @param instanceName
-   */
-  AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName) {
-    _instanceName = instanceName;
-    refresh(clusterConfig, instanceConfig);
-  }
-
-  private void reset() {
-    _currentAssignedReplicaMap = new HashMap<>();
-    _currentCapacityMap = new HashMap<>();
-    _highestCapacityUtilization = 0;
-  }
-
-  /**
    * Update the node with a ClusterDataCache. This resets the current assignment and recalculates
    * currentCapacity.
    * NOTE: While this is required to be used in the constructor, this can also be used when the
@@ -82,29 +71,31 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * refreshed. This is under the assumption that the capacity mappings of InstanceConfig and
    * ResourceConfig could
    * subject to change. If the assumption is no longer true, this function should become private.
-   * @param clusterConfig - the Cluster Config of the cluster where the node is located
-   * @param instanceConfig - the Instance Config of the node
    */
-  private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig) {
-    reset();
-
+  AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName) {
+    _instanceName = instanceName;
     Map<String, Integer> instanceCapacity = fetchInstanceCapacity(clusterConfig, instanceConfig);
-    _currentCapacityMap.putAll(instanceCapacity);
     _faultZone = computeFaultZone(clusterConfig, instanceConfig);
-    _instanceTags = new HashSet<>(instanceConfig.getTags());
-    _disabledPartitionsMap = instanceConfig.getDisabledPartitionsMap();
-    _maxCapacity = instanceCapacity;
+    _instanceTags = ImmutableSet.copyOf(instanceConfig.getTags());
+    _disabledPartitionsMap = ImmutableMap.copyOf(instanceConfig.getDisabledPartitionsMap());
+    // make a copy of max capacity
+    _maxAllowedCapacity = ImmutableMap.copyOf(instanceCapacity);
+    _remainingCapacity = new HashMap<>(instanceCapacity);
     _maxPartition = clusterConfig.getMaxPartitionsPerInstance();
+    _currentAssignedReplicaMap = new HashMap<>();
+    _highestCapacityUtilization = 0f;
   }
 
   /**
    * This function should only be used to assign a set of new partitions that are not allocated on
-   * this node.
+   * this node. It's because the any exception could occur at the middle of batch assignment and the
+   * previous finished assignment cannot be reverted
    * Using this function avoids the overhead of updating capacity repeatedly.
    */
-  void assignNewBatch(Collection<AssignableReplica> replicas) {
+  void assignInitBatch(Collection<AssignableReplica> replicas) {
     Map<String, Integer> totalPartitionCapacity = new HashMap<>();
     for (AssignableReplica replica : replicas) {
+      // TODO: the exception could occur in the middle of for loop and the previous added records cannot be reverted
       addToAssignmentRecord(replica);
       // increment the capacity requirement according to partition's capacity configuration.
       for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
@@ -115,8 +106,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
     }
 
     // Update the global state after all single replications' calculation is done.
-    for (String key : totalPartitionCapacity.keySet()) {
-      updateCapacityAndUtilization(key, totalPartitionCapacity.get(key));
+    for (String capacityKey : totalPartitionCapacity.keySet()) {
+      updateCapacityAndUtilization(capacityKey, totalPartitionCapacity.get(capacityKey));
     }
   }
 
@@ -127,7 +118,7 @@ public class AssignableNode implements Comparable<AssignableNode> {
   void assign(AssignableReplica assignableReplica) {
     addToAssignmentRecord(assignableReplica);
     assignableReplica.getCapacity().entrySet().stream()
-        .forEach(capacity -> updateCapacityAndUtilization(capacity.getKey(), capacity.getValue()));
+            .forEach(capacity -> updateCapacityAndUtilization(capacity.getKey(), capacity.getValue()));
   }
 
   /**
@@ -218,8 +209,16 @@ public class AssignableNode implements Comparable<AssignableNode> {
   /**
    * @return The current available capacity.
    */
-  public Map<String, Integer> getCurrentCapacity() {
-    return _currentCapacityMap;
+  public Map<String, Integer> getRemainingCapacity() {
+    return _remainingCapacity;
+  }
+
+  /**
+   * @return A map of <capacity category, capacity number> that describes the max capacity of the
+   *         node.
+   */
+  public Map<String, Integer> getMaxCapacity() {
+    return _maxAllowedCapacity;
   }
 
   /**
@@ -228,7 +227,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * categories.
    * For example, if the current node usage is {CPU: 0.9, MEM: 0.4, DISK: 0.6}. Then this call shall
    * return 0.9.
-   *
    * @return The highest utilization number of the node among all the capacity category.
    */
   public float getHighestCapacityUtilization() {
@@ -260,14 +258,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
   }
 
   /**
-   * @return A map of <capacity category, capacity number> that describes the max capacity of the
-   *         node.
-   */
-  public Map<String, Integer> getMaxCapacity() {
-    return _maxCapacity;
-  }
-
-  /**
    * @return The max partition count that are allowed to be allocated on the node.
    */
   public int getMaxPartition() {
@@ -294,14 +284,15 @@ public class AssignableNode implements Comparable<AssignableNode> {
     if (topologyStr == null || faultZoneType == null) {
       LOG.debug("Topology configuration is not complete. Topology define: {}, Fault Zone Type: {}",
           topologyStr, faultZoneType);
-      // Use the instance name, or the deprecated ZoneId field (if exists) as the default fault zone.
+      // Use the instance name, or the deprecated ZoneId field (if exists) as the default fault
+      // zone.
       String zoneId = instanceConfig.getZoneId();
       return zoneId == null ? instanceConfig.getInstanceName() : zoneId;
     } else {
       // Get the fault zone information from the complete topology definition.
       String[] topologyDef = topologyStr.trim().split("/");
-      if (topologyDef.length == 0 ||
-          Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
+      if (topologyDef.length == 0
+          || Arrays.stream(topologyDef).noneMatch(type -> type.equals(faultZoneType))) {
         throw new HelixException(
             "The configured topology definition is empty or does not contain the fault zone type.");
       }
@@ -350,22 +341,22 @@ public class AssignableNode implements Comparable<AssignableNode> {
     }
   }
 
-  private void updateCapacityAndUtilization(String capacityKey, int valueToSubtract) {
-    if (_currentCapacityMap.containsKey(capacityKey)) {
-      int newCapacity = _currentCapacityMap.get(capacityKey) - valueToSubtract;
-      _currentCapacityMap.put(capacityKey, newCapacity);
-      // For the purpose of constraint calculation, the max utilization cannot be larger than 100%.
-      float utilization = Math.min(
-          (float) (_maxCapacity.get(capacityKey) - newCapacity) / _maxCapacity.get(capacityKey), 1);
-      _highestCapacityUtilization = Math.max(_highestCapacityUtilization, utilization);
+  private void updateCapacityAndUtilization(String capacityKey, int usage) {
+    if (!_remainingCapacity.containsKey(capacityKey)) {
+      //if the capacityKey belongs to replicas does not exist in the instance's capacity,
+      // it will be treated as if it has unlimited capacity of that capacityKey
+      return;
     }
-    // else if the capacityKey does not exist in the capacity map, this method essentially becomes
-    // a NOP; in other words, this node will be treated as if it has unlimited capacity.
+    int newCapacity = _remainingCapacity.get(capacityKey) - usage;
+    _remainingCapacity.put(capacityKey, newCapacity);
+    // For the purpose of constraint calculation, the max utilization cannot be larger than 100%.
+    float utilization = Math.min((float) (_maxAllowedCapacity.get(capacityKey) - newCapacity)
+        / _maxAllowedCapacity.get(capacityKey), 1);
+    _highestCapacityUtilization = Math.max(_highestCapacityUtilization, utilization);
   }
 
   /**
    * Get and validate the instance capacity from instance config.
-   *
    * @throws HelixException if any required capacity key is not configured in the instance config.
    */
   private Map<String, Integer> fetchInstanceCapacity(ClusterConfig clusterConfig,
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 2b53422..276b998 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -80,7 +80,7 @@ public class ClusterModelProvider {
             bestPossibleAssignment, allocatedReplicas);
 
     // Update the allocated replicas to the assignable nodes.
-    assignableNodes.stream().forEach(node -> node.assignNewBatch(
+    assignableNodes.stream().forEach(node -> node.assignInitBatch(
         allocatedReplicas.getOrDefault(node.getInstanceName(), Collections.emptySet())));
 
     // Construct and initialize cluster context.
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java
index 511f881..4365a42 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestNodeCapacityConstraint.java
@@ -39,7 +39,7 @@ public class TestNodeCapacityConstraint {
   @Test
   public void testConstraintValidWhenNodeHasEnoughSpace() {
     String key = "testKey";
-    when(_testNode.getCurrentCapacity()).thenReturn(ImmutableMap.of(key,  10));
+    when(_testNode.getRemainingCapacity()).thenReturn(ImmutableMap.of(key,  10));
     when(_testReplica.getCapacity()).thenReturn(ImmutableMap.of(key, 5));
     Assert.assertTrue(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
   }
@@ -47,7 +47,7 @@ public class TestNodeCapacityConstraint {
   @Test
   public void testConstraintInValidWhenNodeHasInsufficientSpace() {
     String key = "testKey";
-    when(_testNode.getCurrentCapacity()).thenReturn(ImmutableMap.of(key,  1));
+    when(_testNode.getRemainingCapacity()).thenReturn(ImmutableMap.of(key,  1));
     when(_testReplica.getCapacity()).thenReturn(ImmutableMap.of(key, 5));
     Assert.assertFalse(_constraint.isAssignmentValid(_testNode, _testReplica, _clusterContext));
   }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index 6975901..b48587f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -19,9 +19,10 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import static org.mockito.Mockito.when;
+
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -37,8 +38,6 @@ import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.mockito.Mockito.when;
-
 public class TestAssignableNode extends AbstractTestClusterModel {
   @BeforeClass
   public void initialize() {
@@ -65,7 +64,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
-    assignableNode.assignNewBatch(assignmentSet);
+    assignableNode.assignInitBatch(assignmentSet);
     Assert.assertEquals(assignableNode.getAssignedPartitionsMap(), expectedAssignment);
     Assert.assertEquals(assignableNode.getAssignedReplicaCount(), 4);
     Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
@@ -74,7 +73,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
     Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
-    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getRemainingCapacity(), expectedCapacityMap);
     Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
     Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
         expectedAssignmentSet1);
@@ -114,7 +113,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
     Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
-    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getRemainingCapacity(), expectedCapacityMap);
     Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
     Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
         expectedAssignmentSet1);
@@ -147,7 +146,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
     Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
     Assert.assertEquals(assignableNode.getDisabledPartitionsMap(), _disabledPartitionsMap);
-    Assert.assertEquals(assignableNode.getCurrentCapacity(), expectedCapacityMap);
+    Assert.assertEquals(assignableNode.getRemainingCapacity(), expectedCapacityMap);
     Assert.assertEquals(assignableNode.getAssignedReplicas(), assignmentSet);
     Assert.assertEquals(assignableNode.getAssignedPartitionsByResource(_resourceNames.get(0)),
         expectedAssignmentSet1);
@@ -184,7 +183,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
         testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
-    assignableNode.assignNewBatch(assignmentSet);
+    assignableNode.assignInitBatch(assignmentSet);
     AssignableReplica duplicateReplica = new AssignableReplica(testCache.getClusterConfig(),
         testCache.getResourceConfig(_resourceNames.get(0)), _partitionNames.get(0), "SLAVE", 2);
     assignableNode.assign(duplicateReplica);


[helix] 16/37: Add BucketDataAccessor for large writes

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 34f826094a045fa0c323ef83da99433ac30fd495
Author: Hunter Lee <hu...@linkedin.com>
AuthorDate: Fri Sep 6 16:34:53 2019 -0700

    Add BucketDataAccessor for large writes
    
    For the new WAGED rebalancer, it's necessary to have a data accessor that will allow writes of data exceeding 1MB. ZooKeeper's ZNode size is capped at 1MB, so BucketDataAccessor interface and ZkBucketDataAccessor help us achieve this.
    Changelist:
    1. Add BucketDataAccessor and ZkBucketDataAccessor
    2. Add necessary serializers
    3. Add an integration test against ZK
---
 .../java/org/apache/helix/BucketDataAccessor.java  |  53 ++++
 .../manager/zk/ZNRecordJacksonSerializer.java      |  67 +++++
 .../helix/manager/zk/ZkBucketDataAccessor.java     | 326 +++++++++++++++++++++
 .../helix/manager/zk/TestZkBucketDataAccessor.java | 233 +++++++++++++++
 4 files changed, 679 insertions(+)

diff --git a/helix-core/src/main/java/org/apache/helix/BucketDataAccessor.java b/helix-core/src/main/java/org/apache/helix/BucketDataAccessor.java
new file mode 100644
index 0000000..2008c23
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/BucketDataAccessor.java
@@ -0,0 +1,53 @@
+package org.apache.helix;
+/*
+ * 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.io.IOException;
+
+public interface BucketDataAccessor {
+
+  /**
+   * Write a HelixProperty in buckets, compressed.
+   * @param path path to which the metadata will be written to
+   * @param value HelixProperty to write
+   * @param <T>
+   * @throws IOException
+   */
+  <T extends HelixProperty> boolean compressedBucketWrite(String path, T value) throws IOException;
+
+  /**
+   * Read a HelixProperty that was written in buckets, compressed.
+   * @param path
+   * @param helixPropertySubType the subtype of HelixProperty the data was written in
+   * @param <T>
+   */
+  <T extends HelixProperty> HelixProperty compressedBucketRead(String path,
+      Class<T> helixPropertySubType);
+
+  /**
+   * Delete the HelixProperty in the given path.
+   * @param path
+   */
+  void compressedBucketDelete(String path);
+
+  /**
+   * Close the connection to the metadata store.
+   */
+  void disconnect();
+}
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java
new file mode 100644
index 0000000..989017a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZNRecordJacksonSerializer.java
@@ -0,0 +1,67 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.io.IOException;
+import org.I0Itec.zkclient.exception.ZkMarshallingError;
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.helix.HelixException;
+import org.apache.helix.ZNRecord;
+import org.codehaus.jackson.map.ObjectMapper;
+
+/**
+ * ZNRecordJacksonSerializer serializes ZNRecord objects into a byte array using MessagePack's
+ * serializer. Note that this serializer doesn't check for the size of the resulting binary.
+ */
+public class ZNRecordJacksonSerializer implements ZkSerializer {
+  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+  @Override
+  public byte[] serialize(Object record) throws ZkMarshallingError {
+    if (!(record instanceof ZNRecord)) {
+      // null is NOT an instance of any class
+      throw new HelixException("Input object is not of type ZNRecord (was " + record + ")");
+    }
+    ZNRecord znRecord = (ZNRecord) record;
+
+    try {
+      return OBJECT_MAPPER.writeValueAsBytes(znRecord);
+    } catch (IOException e) {
+      throw new HelixException(
+          String.format("Exception during serialization. ZNRecord id: %s", znRecord.getId()), e);
+    }
+  }
+
+  @Override
+  public Object deserialize(byte[] bytes) throws ZkMarshallingError {
+    if (bytes == null || bytes.length == 0) {
+      // reading a parent/null node
+      return null;
+    }
+
+    ZNRecord record;
+    try {
+      record = OBJECT_MAPPER.readValue(bytes, ZNRecord.class);
+    } catch (IOException e) {
+      throw new HelixException("Exception during deserialization!", e);
+    }
+    return record;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java
new file mode 100644
index 0000000..24c7c8e
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZkBucketDataAccessor.java
@@ -0,0 +1,326 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.I0Itec.zkclient.exception.ZkMarshallingError;
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BaseDataAccessor;
+import org.apache.helix.BucketDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.manager.zk.client.DedicatedZkClientFactory;
+import org.apache.helix.manager.zk.client.HelixZkClient;
+import org.apache.helix.manager.zk.client.SharedZkClientFactory;
+import org.apache.helix.util.GZipCompressionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZkBucketDataAccessor implements BucketDataAccessor, AutoCloseable {
+  private static Logger LOG = LoggerFactory.getLogger(ZkBucketDataAccessor.class);
+
+  private static final int DEFAULT_NUM_VERSIONS = 2;
+  private static final String BUCKET_SIZE_KEY = "BUCKET_SIZE";
+  private static final String DATA_SIZE_KEY = "DATA_SIZE";
+  private static final String WRITE_LOCK_KEY = "WRITE_LOCK";
+  private static final String LAST_SUCCESS_KEY = "LAST_SUCCESS";
+
+  // 100 KB for default bucket size
+  private static final int DEFAULT_BUCKET_SIZE = 50 * 1024;
+  private final int _bucketSize;
+  private final int _numVersions;
+  private ZkSerializer _zkSerializer;
+  private HelixZkClient _zkClient;
+  private HelixZkClient _znRecordClient;
+  private BaseDataAccessor _zkBaseDataAccessor;
+  private BaseDataAccessor<ZNRecord> _znRecordBaseDataAccessor;
+
+  /**
+   * Constructor that allows a custom bucket size.
+   * @param zkAddr
+   * @param bucketSize
+   * @param numVersions number of versions to store in ZK
+   */
+  public ZkBucketDataAccessor(String zkAddr, int bucketSize, int numVersions) {
+    // There are two HelixZkClients:
+    // 1. _zkBaseDataAccessor for writes of binary data
+    // 2. _znRecordBaseDataAccessor for writes of ZNRecord (metadata)
+    _zkClient = DedicatedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddr));
+    _zkClient.setZkSerializer(new ZkSerializer() {
+      @Override
+      public byte[] serialize(Object data) throws ZkMarshallingError {
+        if (data instanceof byte[]) {
+          return (byte[]) data;
+        }
+        throw new HelixException("ZkBucketDataAccesor only supports a byte array as an argument!");
+      }
+
+      @Override
+      public Object deserialize(byte[] data) throws ZkMarshallingError {
+        return data;
+      }
+    });
+    _zkBaseDataAccessor = new ZkBaseDataAccessor(_zkClient);
+
+    // TODO: Optimize serialization with Jackson
+    // TODO: Or use a better binary serialization protocol
+    // TODO: Consider making this also binary
+    // TODO: Consider an async write for the metadata as well
+    _znRecordClient = SharedZkClientFactory.getInstance()
+        .buildZkClient(new HelixZkClient.ZkConnectionConfig(zkAddr));
+    _znRecordBaseDataAccessor = new ZkBaseDataAccessor<>(_znRecordClient);
+    _znRecordClient.setZkSerializer(new ZNRecordSerializer());
+
+    _zkSerializer = new ZNRecordJacksonSerializer();
+    _bucketSize = bucketSize;
+    _numVersions = numVersions;
+  }
+
+  /**
+   * Constructor that uses a default bucket size.
+   * @param zkAddr
+   */
+  public ZkBucketDataAccessor(String zkAddr) {
+    this(zkAddr, DEFAULT_BUCKET_SIZE, DEFAULT_NUM_VERSIONS);
+  }
+
+  @Override
+  public <T extends HelixProperty> boolean compressedBucketWrite(String path, T value)
+      throws IOException {
+    // Take the ZNrecord and serialize it (get byte[])
+    byte[] serializedRecord = _zkSerializer.serialize(value.getRecord());
+    // Compress the byte[]
+    byte[] compressedRecord = GZipCompressionUtil.compress(serializedRecord);
+    // Compute N - number of buckets
+    int numBuckets = (compressedRecord.length + _bucketSize - 1) / _bucketSize;
+
+    if (tryLock(path)) {
+      try {
+        // Read or initialize metadata and compute the last success version index
+        ZNRecord metadataRecord =
+            _znRecordBaseDataAccessor.get(path, null, AccessOption.PERSISTENT);
+        if (metadataRecord == null) {
+          metadataRecord = new ZNRecord(extractIdFromPath(path));
+        }
+        int lastSuccessIndex =
+            (metadataRecord.getIntField(LAST_SUCCESS_KEY, -1) + 1) % _numVersions;
+        String dataPath = path + "/" + lastSuccessIndex;
+
+        List<String> paths = new ArrayList<>();
+        List<Object> buckets = new ArrayList<>();
+
+        int ptr = 0;
+        int counter = 0;
+        while (counter < numBuckets) {
+          paths.add(dataPath + "/" + counter);
+          if (counter == numBuckets - 1) {
+            // Special treatment for the last bucket
+            buckets.add(Arrays.copyOfRange(compressedRecord, ptr,
+                ptr + compressedRecord.length % _bucketSize));
+          } else {
+            buckets.add(Arrays.copyOfRange(compressedRecord, ptr, ptr + _bucketSize));
+          }
+          ptr += _bucketSize;
+          counter++;
+        }
+
+        // Do a cleanup of previous data
+        if (!_zkBaseDataAccessor.remove(dataPath, AccessOption.PERSISTENT)) {
+          // Clean-up is not critical so upon failure, we log instead of throwing an exception
+          LOG.warn("Failed to clean up previous bucketed data in data path: {}", dataPath);
+        }
+
+        // Do an async set to ZK
+        boolean[] success =
+            _zkBaseDataAccessor.setChildren(paths, buckets, AccessOption.PERSISTENT);
+        // Exception and fail the write if any failed
+        for (boolean s : success) {
+          if (!s) {
+            throw new HelixException(
+                String.format("Failed to write the data buckets for path: %s", path));
+          }
+        }
+
+        // Data write completed, so update the metadata with last success index
+        // Note that the metadata ZNodes is written using sync write
+        metadataRecord.setIntField(BUCKET_SIZE_KEY, _bucketSize);
+        metadataRecord.setLongField(DATA_SIZE_KEY, compressedRecord.length);
+        metadataRecord.setIntField(LAST_SUCCESS_KEY, lastSuccessIndex);
+        if (!_znRecordBaseDataAccessor.set(path, metadataRecord, AccessOption.PERSISTENT)) {
+          throw new HelixException(
+              String.format("Failed to write the metadata at path: %s!", path));
+        }
+      } finally {
+        // Critical section for write ends here
+        unlock(path);
+      }
+      return true;
+    }
+    throw new HelixException(String.format("Could not acquire lock for write. Path: %s", path));
+  }
+
+  @Override
+  public <T extends HelixProperty> HelixProperty compressedBucketRead(String path,
+      Class<T> helixPropertySubType) {
+    return helixPropertySubType.cast(compressedBucketRead(path));
+  }
+
+  @Override
+  public void compressedBucketDelete(String path) {
+    if (!_zkBaseDataAccessor.remove(path, AccessOption.PERSISTENT)) {
+      throw new HelixException(String.format("Failed to delete the bucket data! Path: %s", path));
+    }
+  }
+
+  @Override
+  public void disconnect() {
+    if (!_zkClient.isClosed()) {
+      _zkClient.close();
+    }
+    if (!_znRecordClient.isClosed()) {
+      _znRecordClient.close();
+    }
+  }
+
+  private HelixProperty compressedBucketRead(String path) {
+    // TODO: Incorporate parallelism into reads instead of locking the whole thing against other
+    // reads and writes
+    if (tryLock(path)) {
+      try {
+        // Retrieve the metadata
+        ZNRecord metadataRecord =
+            _znRecordBaseDataAccessor.get(path, null, AccessOption.PERSISTENT);
+        if (metadataRecord == null) {
+          throw new HelixException(
+              String.format("Metadata ZNRecord does not exist for path: %s", path));
+        }
+
+        int bucketSize = metadataRecord.getIntField(BUCKET_SIZE_KEY, -1);
+        int dataSize = metadataRecord.getIntField(DATA_SIZE_KEY, -1);
+        int lastSuccessIndex = metadataRecord.getIntField(LAST_SUCCESS_KEY, -1);
+        if (lastSuccessIndex == -1) {
+          throw new HelixException(String.format("Metadata ZNRecord does not have %s! Path: %s",
+              LAST_SUCCESS_KEY, path));
+        }
+        if (bucketSize == -1) {
+          throw new HelixException(
+              String.format("Metadata ZNRecord does not have %s! Path: %s", BUCKET_SIZE_KEY, path));
+        }
+        if (dataSize == -1) {
+          throw new HelixException(
+              String.format("Metadata ZNRecord does not have %s! Path: %s", DATA_SIZE_KEY, path));
+        }
+
+        // Compute N - number of buckets
+        int numBuckets = (dataSize + _bucketSize - 1) / _bucketSize;
+        byte[] compressedRecord = new byte[dataSize];
+        String dataPath = path + "/" + lastSuccessIndex;
+
+        List<String> paths = new ArrayList<>();
+        for (int i = 0; i < numBuckets; i++) {
+          paths.add(dataPath + "/" + i);
+        }
+
+        // Async get
+        List buckets = _zkBaseDataAccessor.get(paths, null, AccessOption.PERSISTENT, true);
+
+        // Combine buckets into one byte array
+        int copyPtr = 0;
+        for (int i = 0; i < numBuckets; i++) {
+          if (i == numBuckets - 1) {
+            // Special treatment for the last bucket
+            System.arraycopy(buckets.get(i), 0, compressedRecord, copyPtr, dataSize % bucketSize);
+          } else {
+            System.arraycopy(buckets.get(i), 0, compressedRecord, copyPtr, bucketSize);
+            copyPtr += bucketSize;
+          }
+        }
+
+        // Decompress the byte array
+        ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(compressedRecord);
+        byte[] serializedRecord;
+        try {
+          serializedRecord = GZipCompressionUtil.uncompress(byteArrayInputStream);
+        } catch (IOException e) {
+          throw new HelixException(String.format("Failed to decompress path: %s!", path), e);
+        }
+
+        // Deserialize the record to retrieve the original
+        ZNRecord originalRecord = (ZNRecord) _zkSerializer.deserialize(serializedRecord);
+        return new HelixProperty(originalRecord);
+      } finally {
+        // Critical section for read ends here
+        unlock(path);
+      }
+    }
+    throw new HelixException(String.format("Could not acquire lock for read. Path: %s", path));
+  }
+
+  /**
+   * Returns the last string element in a split String array by /.
+   * @param path
+   * @return
+   */
+  private String extractIdFromPath(String path) {
+    String[] splitPath = path.split("/");
+    return splitPath[splitPath.length - 1];
+  }
+
+  /**
+   * Acquires the lock (create an ephemeral node) only if it is free (no ephemeral node already
+   * exists) at the time of invocation.
+   * @param path
+   * @return
+   */
+  private boolean tryLock(String path) {
+    // Check if another write is taking place and if not, create an ephemeral node to simulate
+    // acquiring of a lock
+    return !_zkBaseDataAccessor.exists(path + "/" + WRITE_LOCK_KEY, AccessOption.EPHEMERAL)
+        && _zkBaseDataAccessor.set(path + "/" + WRITE_LOCK_KEY, new byte[0],
+            AccessOption.EPHEMERAL);
+  }
+
+  /**
+   * Releases the lock (removes the ephemeral node).
+   * @param path
+   */
+  private void unlock(String path) {
+    // Write succeeded, so release the lock
+    if (!_zkBaseDataAccessor.remove(path + "/" + WRITE_LOCK_KEY, AccessOption.EPHEMERAL)) {
+      throw new HelixException(String.format("Could not remove ephemeral node for path: %s", path));
+    }
+    // TODO: In case of remove failure, we risk a lock never getting released.
+    // TODO: Consider two possible improvements
+    // TODO: 1. Use ephemeral owner id for the same connection to reclaim the lock
+    // TODO: 2. Use "lease" - lock with a timeout
+  }
+
+  @Override
+  public void close() throws Exception {
+    disconnect();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBucketDataAccessor.java b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBucketDataAccessor.java
new file mode 100644
index 0000000..4c28835
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/manager/zk/TestZkBucketDataAccessor.java
@@ -0,0 +1,233 @@
+package org.apache.helix.manager.zk;
+
+/*
+ * 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 com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.helix.AccessOption;
+import org.apache.helix.BucketDataAccessor;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixProperty;
+import org.apache.helix.TestHelper;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.common.ZkTestBase;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+public class TestZkBucketDataAccessor extends ZkTestBase {
+
+  private static final String PATH = "/" + TestHelper.getTestClassName();
+  private static final String NAME_KEY = TestHelper.getTestClassName();
+  private static final String LAST_SUCCESS_KEY = "LAST_SUCCESS";
+  private static final String BUCKET_SIZE_KEY = "BUCKET_SIZE";
+  private static final String WRITE_LOCK_KEY = "WRITE_LOCK";
+
+  // Populate list and map fields for content comparison
+  private static final List<String> LIST_FIELD = ImmutableList.of("1", "2");
+  private static final Map<String, String> MAP_FIELD = ImmutableMap.of("1", "2");
+
+  private BucketDataAccessor _bucketDataAccessor;
+
+  @BeforeClass
+  public void beforeClass() {
+    _bucketDataAccessor = new ZkBucketDataAccessor(ZK_ADDR);
+  }
+
+  @AfterClass
+  public void afterClass() {
+    _bucketDataAccessor.disconnect();
+  }
+
+  /**
+   * Attempt writing a simple HelixProperty using compressedBucketWrite.
+   * @throws IOException
+   */
+  @Test
+  public void testCompressedBucketWrite() throws IOException {
+    ZNRecord record = new ZNRecord(NAME_KEY);
+    record.setSimpleField(NAME_KEY, NAME_KEY);
+    record.setListField(NAME_KEY, LIST_FIELD);
+    record.setMapField(NAME_KEY, MAP_FIELD);
+    Assert.assertTrue(_bucketDataAccessor.compressedBucketWrite(PATH, new HelixProperty(record)));
+  }
+
+  /**
+   * The record written in {@link #testCompressedBucketWrite()} is the same record that was written.
+   */
+  @Test(dependsOnMethods = "testCompressedBucketWrite")
+  public void testCompressedBucketRead() {
+    HelixProperty readRecord = _bucketDataAccessor.compressedBucketRead(PATH, HelixProperty.class);
+    Assert.assertEquals(readRecord.getRecord().getSimpleField(NAME_KEY), NAME_KEY);
+    Assert.assertEquals(readRecord.getRecord().getListField(NAME_KEY), LIST_FIELD);
+    Assert.assertEquals(readRecord.getRecord().getMapField(NAME_KEY), MAP_FIELD);
+    _bucketDataAccessor.compressedBucketDelete(PATH);
+  }
+
+  /**
+   * Do 10 writes and check that there are 5 versions of the data.
+   */
+  @Test(dependsOnMethods = "testCompressedBucketRead")
+  public void testManyWritesWithVersionCounts() throws IOException {
+    int bucketSize = 50 * 1024;
+    int numVersions = 5;
+    int expectedLastSuccessfulIndex = 4;
+    String path = PATH + "2";
+    ZNRecord record = new ZNRecord(NAME_KEY);
+    record.setSimpleField(NAME_KEY, NAME_KEY);
+    record.setListField(NAME_KEY, LIST_FIELD);
+    record.setMapField(NAME_KEY, MAP_FIELD);
+
+    BucketDataAccessor bucketDataAccessor =
+        new ZkBucketDataAccessor(ZK_ADDR, bucketSize, numVersions);
+    for (int i = 0; i < 10; i++) {
+      bucketDataAccessor.compressedBucketWrite(path, new HelixProperty(record));
+    }
+
+    // Check that there are numVersions number of children under path
+    List<String> children = _baseAccessor.getChildNames(path, AccessOption.PERSISTENT);
+    Assert.assertEquals(children.size(), numVersions);
+
+    // Check that last successful index is 4 (since we did 10 writes)
+    ZNRecord metadata = _baseAccessor.get(path, null, AccessOption.PERSISTENT);
+    Assert.assertEquals(metadata.getIntField(LAST_SUCCESS_KEY, -1), expectedLastSuccessfulIndex);
+
+    // Clean up
+    bucketDataAccessor.compressedBucketDelete(path);
+    bucketDataAccessor.disconnect();
+  }
+
+  /**
+   * Write a HelixProperty with large number of entries using BucketDataAccessor and read it back.
+   */
+  @Test(dependsOnMethods = "testManyWritesWithVersionCounts")
+  public void testLargeWriteAndRead() throws IOException {
+    String name = "largeResourceAssignment";
+    HelixProperty property = createLargeHelixProperty(name, 100000);
+
+    // Perform large write
+    long before = System.currentTimeMillis();
+    _bucketDataAccessor.compressedBucketWrite("/" + name, property);
+    long after = System.currentTimeMillis();
+    System.out.println("Write took " + (after - before) + " ms");
+
+    // Read it back
+    before = System.currentTimeMillis();
+    HelixProperty readRecord =
+        _bucketDataAccessor.compressedBucketRead("/" + name, HelixProperty.class);
+    after = System.currentTimeMillis();
+    System.out.println("Read took " + (after - before) + " ms");
+
+    // Check against the original HelixProperty
+    Assert.assertEquals(readRecord, property);
+  }
+
+  /**
+   * Tests that each write cleans up previous bucketed data. This method writes some small amount of
+   * data and checks that the data buckets from the large write performed in the previous test
+   * method have been cleaned up.
+   * @throws IOException
+   */
+  @Test(dependsOnMethods = "testLargeWriteAndRead")
+  public void testCleanupBeforeWrite() throws IOException {
+    // Create a HelixProperty of a very small size with the same name as the large HelixProperty
+    // created from the previous method
+    String name = "largeResourceAssignment";
+    HelixProperty property = new HelixProperty(name);
+    property.getRecord().setIntField("Hi", 10);
+
+    // Get the bucket count from the write performed in the previous method
+    ZNRecord metadata = _baseAccessor.get("/" + name, null, AccessOption.PERSISTENT);
+    int origBucketSize = metadata.getIntField(BUCKET_SIZE_KEY, -1);
+
+    // Perform a write twice to overwrite both versions
+    _bucketDataAccessor.compressedBucketWrite("/" + name, property);
+    _bucketDataAccessor.compressedBucketWrite("/" + name, property);
+
+    // Check that the children count for version 0 (version for the large write) is 1
+    Assert.assertEquals(
+        _baseAccessor.getChildNames("/" + name + "/0", AccessOption.PERSISTENT).size(), 1);
+
+    // Clean up
+    _bucketDataAccessor.compressedBucketDelete("/" + name);
+  }
+
+  /**
+   * Test that no concurrent reads and writes are allowed by triggering multiple operations after
+   * creating an artificial lock.
+   * @throws IOException
+   */
+  @Test(dependsOnMethods = "testCleanupBeforeWrite")
+  public void testFailureToAcquireLock() throws Exception {
+    String name = "acquireLock";
+    // Use a large HelixProperty to simulate a write that keeps the lock for some time
+    HelixProperty property = createLargeHelixProperty(name, 100);
+
+    // Artificially create the ephemeral ZNode
+    _baseAccessor.create("/" + name + "/" + WRITE_LOCK_KEY, new ZNRecord(name),
+        AccessOption.EPHEMERAL);
+
+    // Test write
+    try {
+      _bucketDataAccessor.compressedBucketWrite("/" + name, property);
+      Assert.fail("Should fail due to an already-existing lock ZNode!");
+    } catch (HelixException e) {
+      // Expect an exception
+    }
+
+    // Test read
+    try {
+      _bucketDataAccessor.compressedBucketRead("/" + name, HelixProperty.class);
+      Assert.fail("Should fail due to an already-existing lock ZNode!");
+    } catch (HelixException e) {
+      // Expect an exception
+    }
+
+    // Clean up
+    _bucketDataAccessor.compressedBucketDelete("/" + name);
+  }
+
+  private HelixProperty createLargeHelixProperty(String name, int numEntries) {
+    HelixProperty property = new HelixProperty(name);
+    for (int i = 0; i < numEntries; i++) {
+      // Create a random string every time
+      byte[] arrayKey = new byte[20];
+      byte[] arrayVal = new byte[20];
+      new Random().nextBytes(arrayKey);
+      new Random().nextBytes(arrayVal);
+      String randomStrKey = new String(arrayKey, StandardCharsets.UTF_8);
+      String randomStrVal = new String(arrayVal, StandardCharsets.UTF_8);
+
+      // Dummy mapField
+      Map<String, String> mapField = new HashMap<>();
+      mapField.put(randomStrKey, randomStrVal);
+
+      property.getRecord().setMapField(randomStrKey, mapField);
+    }
+    return property;
+  }
+}


[helix] 17/37: Implement the basic constraint based algorithm (#381)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9f73d8dacbecc768317c7051a07fe5a9d7d5f778
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Fri Sep 6 16:57:25 2019 -0700

    Implement the basic constraint based algorithm (#381)
    
    Implement basic constraint algorithm: Greedy based, each time it picks the best scores given each replica and assigns the replica to the node. It doesn't guarantee to achieve global optimal but local optimal result
    
    The algorithm is based on a given set of constraints
    
    * HardConstraint: Approve or deny the assignment given its condition, any assignment cannot bypass any "hard constraint"
    * SoftConstraint: Evaluate the assignment by points/rewards/scores, a higher point means a better assignment
    The goal is to avoid all "hard constraints" while accumulating the most points(rewards) from "soft constraints"
---
 .../org/apache/helix/HelixRebalanceException.java  |  13 ++-
 .../{constraints => }/RebalanceAlgorithm.java      |  27 +++--
 .../rebalancer/waged/WagedRebalancer.java          |  13 +--
 .../constraints/ConstraintBasedAlgorithm.java      | 129 +++++++++++++++++++++
 .../ConstraintBasedAlgorithmFactory.java           |  41 +++++++
 .../constraints/ConstraintsRebalanceAlgorithm.java |  48 --------
 .../constraints/SoftConstraintWeightModel.java     |   6 +-
 .../rebalancer/waged/model/OptimalAssignment.java  |  69 +++++++++++
 .../constraints/TestConstraintBasedAlgorithm.java  |  74 ++++++++++++
 .../waged/model/ClusterModelTestHelper.java        |  50 ++++++++
 10 files changed, 396 insertions(+), 74 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
index c01b173..d01fc60 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
@@ -23,21 +23,26 @@ package org.apache.helix;
  * Exception thrown by Helix due to rebalance failures.
  */
 public class HelixRebalanceException extends Exception {
-  enum RebalanceFailureType {
+  public enum Type {
     INVALID_CLUSTER_STATUS,
     INVALID_REBALANCER_STATUS,
     FAILED_TO_CALCULATE,
     UNKNOWN_FAILURE
   }
 
-  private final RebalanceFailureType _type;
+  private final Type _type;
 
-  public HelixRebalanceException(String message, RebalanceFailureType type, Throwable cause) {
+  public HelixRebalanceException(String message, Type type, Throwable cause) {
     super(String.format("%s. Failure Type: %s", message, type.name()), cause);
     _type = type;
   }
 
-  public RebalanceFailureType getFailureType() {
+  public HelixRebalanceException(String message, Type type) {
+    super(String.format("%s. Failure Type: %s", message, type.name()));
+    _type = type;
+  }
+
+  public Type getFailureType() {
     return _type;
   }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
similarity index 53%
rename from helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java
rename to helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
index b652836..1374162 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/RebalanceAlgorithm.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/RebalanceAlgorithm.java
@@ -1,4 +1,4 @@
-package org.apache.helix.controller.rebalancer.waged.constraints;
+package org.apache.helix.controller.rebalancer.waged;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -9,7 +9,7 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
  * "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
+ *     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
@@ -19,26 +19,25 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
  * under the License.
  */
 
+import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
-import org.apache.helix.model.ResourceAssignment;
-
-import java.util.Map;
+import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
 
 /**
- * A generic rebalance algorithm interface for the WAGED rebalancer.
+ * A generic interface to generate the optimal assignment given the runtime cluster environment.
  *
- * @see <a href="Rebalance Algorithm">https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer#rebalance-algorithm-adapter</a>
+ * <pre>
+ * @see <a href="https://github.com/apache/helix/wiki/
+ * Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer
+ * #rebalance-algorithm-adapter">Rebalance Algorithm</a>
+ * </pre>
  */
 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 of <ResourceName, ResourceAssignment>.
+   * @param clusterModel The run time cluster model that contains all necessary information
+   * @return An instance of {@link OptimalAssignment}
    */
-  Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
-      Map<String, Map<HardConstraint, Integer>> failureReasons);
+  OptimalAssignment calculate(ClusterModel clusterModel) throws HelixRebalanceException;
 }
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
index 43b2564..5b9634e 100644
--- 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
@@ -19,23 +19,22 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.changedetector.ResourceChangeDetector;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
-import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintsRebalanceAlgorithm;
-import org.apache.helix.controller.rebalancer.waged.constraints.RebalanceAlgorithm;
+import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintBasedAlgorithmFactory;
 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.
@@ -69,8 +68,8 @@ public class WagedRebalancer {
   public WagedRebalancer(HelixManager helixManager) {
     // TODO init the metadata store according to their requirement when integrate, or change to final static method if possible.
     _assignmentMetadataStore = new AssignmentMetadataStore();
-    // TODO init the algorithm according to the requirement when integrate.
-    _rebalanceAlgorithm = new ConstraintsRebalanceAlgorithm();
+    // TODO parse the cluster setting
+    _rebalanceAlgorithm = ConstraintBasedAlgorithmFactory.getInstance();
 
     // Use the mapping calculator in DelayedAutoRebalancer for calculating the final assignment
     // output.
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
new file mode 100644
index 0000000..99d8d2a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithm.java
@@ -0,0 +1,129 @@
+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 java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
+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;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
+
+/**
+ * The algorithm is based on a given set of constraints
+ * - HardConstraint: Approve or deny the assignment given its condition, any assignment cannot
+ * bypass any "hard constraint"
+ * - SoftConstraint: Evaluate the assignment by points/rewards/scores, a higher point means a better
+ * assignment
+ * The goal is to accumulate the most points(rewards) from "soft constraints" while avoiding any
+ * "hard constraints"
+ */
+class ConstraintBasedAlgorithm implements RebalanceAlgorithm {
+  private static final Logger LOG = LoggerFactory.getLogger(ConstraintBasedAlgorithm.class);
+  private final List<HardConstraint> _hardConstraints;
+  private final List<SoftConstraint> _softConstraints;
+  private final SoftConstraintWeightModel _softConstraintsWeightModel;
+
+  ConstraintBasedAlgorithm(List<HardConstraint> hardConstraints,
+      List<SoftConstraint> softConstraints, SoftConstraintWeightModel softConstraintWeightModel) {
+    _hardConstraints = hardConstraints;
+    _softConstraints = softConstraints;
+    _softConstraintsWeightModel = softConstraintWeightModel;
+  }
+
+  @Override
+  public OptimalAssignment calculate(ClusterModel clusterModel) throws HelixRebalanceException {
+    OptimalAssignment optimalAssignment = new OptimalAssignment();
+    Map<String, Set<AssignableReplica>> replicasByResource = clusterModel.getAssignableReplicaMap();
+    List<AssignableNode> nodes = new ArrayList<>(clusterModel.getAssignableNodes().values());
+
+    // TODO: different orders of resource/replica could lead to different greedy assignments, will
+    // revisit and improve the performance
+    for (String resource : replicasByResource.keySet()) {
+      for (AssignableReplica replica : replicasByResource.get(resource)) {
+        Optional<AssignableNode> maybeBestNode =
+            getNodeWithHighestPoints(replica, nodes, clusterModel.getContext(), optimalAssignment);
+        // stop immediately if any replica cannot find best assignable node
+        if (optimalAssignment.hasAnyFailure()) {
+          String errorMessage = String.format(
+              "Unable to find any available candidate node for partition %s; Fail reasons: %s",
+              replica.getPartitionName(), optimalAssignment.getFailures());
+          throw new HelixRebalanceException(errorMessage,
+              HelixRebalanceException.Type.FAILED_TO_CALCULATE);
+        }
+        maybeBestNode.ifPresent(node -> clusterModel.assign(replica.getResourceName(),
+            replica.getPartitionName(), replica.getReplicaState(), node.getInstanceName()));
+      }
+    }
+
+    return optimalAssignment.convertFrom(clusterModel);
+  }
+
+  private Optional<AssignableNode> getNodeWithHighestPoints(AssignableReplica replica,
+      List<AssignableNode> assignableNodes, ClusterContext clusterContext,
+      OptimalAssignment optimalAssignment) {
+    Map<AssignableNode, List<HardConstraint>> hardConstraintFailures = new HashMap<>();
+    List<AssignableNode> candidateNodes = assignableNodes.stream().filter(candidateNode -> {
+      boolean isValid = true;
+      // need to record all the failure reasons and it gives us the ability to debug/fix the runtime
+      // cluster environment
+      for (HardConstraint hardConstraint : _hardConstraints) {
+        if (!hardConstraint.isAssignmentValid(candidateNode, replica, clusterContext)) {
+          hardConstraintFailures.computeIfAbsent(candidateNode, node -> new ArrayList<>())
+              .add(hardConstraint);
+          isValid = false;
+        }
+      }
+      return isValid;
+    }).collect(Collectors.toList());
+    if (candidateNodes.isEmpty()) {
+      optimalAssignment.recordAssignmentFailure(replica,
+          Maps.transformValues(hardConstraintFailures, this::convertFailureReasons));
+      return Optional.empty();
+    }
+
+    Function<AssignableNode, Float> calculatePoints =
+        (candidateNode) -> _softConstraintsWeightModel.getSumOfScores(_softConstraints.stream()
+            .collect(Collectors.toMap(Function.identity(), softConstraint -> softConstraint
+                .getAssignmentOriginScore(candidateNode, replica, clusterContext))));
+
+    return candidateNodes.stream().max(Comparator.comparing(calculatePoints));
+  }
+
+  private List<String> convertFailureReasons(List<HardConstraint> hardConstraints) {
+    return hardConstraints.stream().map(HardConstraint::getDescription)
+        .collect(Collectors.toList());
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
new file mode 100644
index 0000000..895fa61
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintBasedAlgorithmFactory.java
@@ -0,0 +1,41 @@
+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 java.util.ArrayList;
+import java.util.List;
+
+import org.apache.helix.controller.rebalancer.waged.RebalanceAlgorithm;
+import org.apache.helix.model.ClusterConfig;
+
+public class ConstraintBasedAlgorithmFactory {
+
+  // TODO: the parameter comes from cluster config, will tune how these 2 integers will change the
+  // soft constraint weight model
+  public static RebalanceAlgorithm getInstance() {
+    // TODO initialize constraints, depending on constraints implementations PRs
+    List<HardConstraint> hardConstraints = new ArrayList<>();
+    List<SoftConstraint> softConstraints = new ArrayList<>();
+    SoftConstraintWeightModel softConstraintWeightModel = new SoftConstraintWeightModel();
+
+    return new ConstraintBasedAlgorithm(hardConstraints, softConstraints,
+        softConstraintWeightModel);
+  }
+}
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
deleted file mode 100644
index 8adaa73..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ConstraintsRebalanceAlgorithm.java
+++ /dev/null
@@ -1,48 +0,0 @@
-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.ClusterModel;
-import org.apache.helix.model.ResourceAssignment;
-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, Integer> _failureReasonCounterMap = new HashMap<>();
-
-  public ConstraintsRebalanceAlgorithm() {
-    // TODO Constraints initialization
-  }
-
-  @Override
-  public Map<String, ResourceAssignment> rebalance(ClusterModel clusterModel,
-      Map<String, Map<HardConstraint, Integer>> failureReasons) {
-    return new HashMap<>();
-  }
-}
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
index 10201ce..41e4334 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/SoftConstraintWeightModel.java
@@ -29,8 +29,12 @@ import com.google.common.collect.ImmutableMap;
 class SoftConstraintWeightModel {
   private static Map<? extends SoftConstraint, Float> MODEL;
 
+  // TODO either define the weights in property files or zookeeper node or static human input
+  SoftConstraintWeightModel() {
+
+  }
+
   static {
-    // TODO either define the weights in property files or zookeeper node or static human input
     MODEL = ImmutableMap.<SoftConstraint, Float> builder()
         .put(LeastPartitionCountConstraint.INSTANCE, 1.0f).build();
   }
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java
new file mode 100644
index 0000000..31cb181
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/OptimalAssignment.java
@@ -0,0 +1,69 @@
+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.
+ */
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.model.ResourceAssignment;
+
+/**
+ * The data model represents the optimal assignment of N replicas assigned to M instances;
+ * It's mostly used as the return parameter of an assignment calculation algorithm; If the algorithm
+ * failed to find optimal assignment given the endeavor, the user could check the failure reasons
+ */
+public class OptimalAssignment {
+  private Map<AssignableNode, List<AssignableReplica>> _optimalAssignment = new HashMap<>();
+  private Map<AssignableReplica, Map<AssignableNode, List<String>>> _failedAssignments =
+      new HashMap<>();
+
+  public OptimalAssignment() {
+
+  }
+
+  public void updateAssignments(ClusterModel clusterModel) {
+
+  }
+
+  // TODO: determine the output of final assignment format
+  public Map<String, ResourceAssignment> getOptimalResourceAssignment() {
+    throw new UnsupportedOperationException("Not implemented yet");
+  }
+
+  // TODO: the convert method is not the best choice so far, will revisit the data model
+  public OptimalAssignment convertFrom(ClusterModel clusterModel) {
+    return this;
+  }
+
+  public void recordAssignmentFailure(AssignableReplica replica,
+      Map<AssignableNode, List<String>> failedReasons) {
+    _failedAssignments.put(replica, failedReasons);
+  }
+
+  public boolean hasAnyFailure() {
+    return !_failedAssignments.isEmpty();
+  }
+
+  public String getFailures() {
+    // TODO: format the error string
+    return _failedAssignments.toString();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
new file mode 100644
index 0000000..d06cc5f
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestConstraintBasedAlgorithm.java
@@ -0,0 +1,74 @@
+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 static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModelTestHelper;
+import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
+import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableList;
+
+public class TestConstraintBasedAlgorithm {
+  private ConstraintBasedAlgorithm _algorithm;
+
+  @BeforeMethod
+  public void beforeMethod() {
+    HardConstraint mockHardConstraint = mock(HardConstraint.class);
+    SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
+    SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
+    when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(false);
+    when(mockSoftConstraint.getAssignmentOriginScore(any(), any(), any())).thenReturn(1.0f);
+
+    _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
+        ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
+  }
+
+  @Test(expectedExceptions = HelixRebalanceException.class)
+  public void testCalculateNoValidAssignment() throws IOException, HelixRebalanceException {
+    ClusterModel clusterModel = new ClusterModelTestHelper().getDefaultClusterModel();
+    _algorithm.calculate(clusterModel);
+  }
+
+  @Test
+  public void testCalculateWithValidAssignment() throws IOException, HelixRebalanceException {
+    HardConstraint mockHardConstraint = mock(HardConstraint.class);
+    SoftConstraint mockSoftConstraint = mock(SoftConstraint.class);
+    SoftConstraintWeightModel mockSoftConstraintWeightModel = mock(SoftConstraintWeightModel.class);
+    when(mockHardConstraint.isAssignmentValid(any(), any(), any())).thenReturn(true);
+    when(mockSoftConstraint.getAssignmentOriginScore(any(), any(), any())).thenReturn(1.0f);
+    when(mockSoftConstraintWeightModel.getSumOfScores(any())).thenReturn(1.0f);
+    _algorithm = new ConstraintBasedAlgorithm(ImmutableList.of(mockHardConstraint),
+        ImmutableList.of(mockSoftConstraint), mockSoftConstraintWeightModel);
+    ClusterModel clusterModel = new ClusterModelTestHelper().getDefaultClusterModel();
+    OptimalAssignment optimalAssignment = _algorithm.calculate(clusterModel);
+
+    Assert.assertFalse(optimalAssignment.hasAnyFailure());
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
new file mode 100644
index 0000000..76f1141
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
@@ -0,0 +1,50 @@
+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.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+
+public class ClusterModelTestHelper extends AbstractTestClusterModel {
+
+  public ClusterModel getDefaultClusterModel() throws IOException {
+    initialize();
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignableReplicas = generateReplicas(testCache);
+    Set<AssignableNode> assignableNodes = generateNodes(testCache);
+
+    ClusterContext context = new ClusterContext(assignableReplicas, 2);
+    return new ClusterModel(context, assignableReplicas, assignableNodes, Collections.emptyMap(),
+        Collections.emptyMap());
+  }
+
+  private Set<AssignableNode> generateNodes(ResourceControllerDataProvider testCache) {
+    Set<AssignableNode> nodeSet = new HashSet<>();
+    testCache.getInstanceConfigMap().values().stream()
+            .forEach(config -> nodeSet.add(new AssignableNode(testCache.getClusterConfig(),
+                    testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName(),
+                    Collections.emptyList())));
+    return nodeSet;
+  }
+}


[helix] 27/37: Improve ResourceTopStateAntiAffinityConstraint (#475)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit eb95a19f900a472405c383185dba7e0d7085adf5
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Tue Sep 17 10:09:49 2019 -0700

    Improve ResourceTopStateAntiAffinityConstraint (#475)
    
    - fix the min max range to be [0,1]
    - add unit test for normalized score
---
 .../ResourceTopStateAntiAffinityConstraint.java    | 11 ++++++++--
 ...TestResourceTopStateAntiAffinityConstraint.java | 25 ++++++++++++++++++----
 2 files changed, 30 insertions(+), 6 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java
index b1e64b9..8681dc5 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourceTopStateAntiAffinityConstraint.java
@@ -28,12 +28,19 @@ import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
  * The higher number the number of top state partitions assigned to the instance, the lower the
  * score, vice versa.
  */
-public class ResourceTopStateAntiAffinityConstraint extends SoftConstraint {
+class ResourceTopStateAntiAffinityConstraint extends SoftConstraint {
+  private static final float MAX_SCORE = 1f;
+  private static final float MIN_SCORE = 0f;
+
+  ResourceTopStateAntiAffinityConstraint() {
+    super(MAX_SCORE, MIN_SCORE);
+  }
+
   @Override
   protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
       ClusterContext clusterContext) {
     if (!replica.isReplicaTopState()) {
-      return (getMaxScore() + getMinScore()) / 2;
+      return (getMaxScore() + getMinScore()) / 2.0f;
     }
 
     int curTopPartitionCountForResource = node.getAssignedTopStatePartitionsCount();
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java
index 06ef537..c3414d4 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourceTopStateAntiAffinityConstraint.java
@@ -27,20 +27,31 @@ import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterContext;
 import org.mockito.Mockito;
 import org.testng.Assert;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public class TestResourceTopStateAntiAffinityConstraint {
-  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
-  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
-  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private AssignableReplica _testReplica;
+  private AssignableNode _testNode;
+  private ClusterContext _clusterContext;
 
   private final SoftConstraint _constraint = new ResourceTopStateAntiAffinityConstraint();
 
+  @BeforeMethod
+  public void init() {
+    _testReplica = Mockito.mock(AssignableReplica.class);
+    _testNode = Mockito.mock(AssignableNode.class);
+    _clusterContext = Mockito.mock(ClusterContext.class);
+  }
+
   @Test
   public void testGetAssignmentScoreWhenReplicaNotTopState() {
     when(_testReplica.isReplicaTopState()).thenReturn(false);
     float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
-    Assert.assertEquals(score, (_constraint.getMaxScore() + _constraint.getMinScore()) / 2);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.5f);
+    Assert.assertEquals(normalizedScore, 0.5f);
     verifyZeroInteractions(_testNode);
     verifyZeroInteractions(_clusterContext);
   }
@@ -51,7 +62,10 @@ public class TestResourceTopStateAntiAffinityConstraint {
     when(_testNode.getAssignedTopStatePartitionsCount()).thenReturn(20);
     when(_clusterContext.getEstimatedMaxTopStateCount()).thenReturn(20);
     float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
     Assert.assertEquals(score, 0.5f);
+    Assert.assertEquals(normalizedScore, 0.5f);
   }
 
   @Test
@@ -60,6 +74,9 @@ public class TestResourceTopStateAntiAffinityConstraint {
     when(_testNode.getAssignedTopStatePartitionsCount()).thenReturn(0);
     when(_clusterContext.getEstimatedMaxTopStateCount()).thenReturn(20);
     float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore =
+        _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
     Assert.assertEquals(score, 1f);
+    Assert.assertEquals(normalizedScore, 1f);
   }
 }


[helix] 19/37: Implement the WAGED rebalancer with the limited functionality. (#443)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 702c547d987137102c3e9086516cce20fc1e3076
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Mon Sep 9 10:35:59 2019 -0700

    Implement the WAGED rebalancer with the limited functionality. (#443)
    
    The implemented rebalancer supports basic rebalance logic. It does not contain the logic to support delayed rebalance and user-defined preference list.
    
    Added unit test to cover the main workflow of the WAGED rebalancer.
---
 .../org/apache/helix/HelixRebalanceException.java  |   4 +-
 .../rebalancer/waged/WagedRebalancer.java          | 313 ++++++++++++++--
 .../rebalancer/waged/model/AssignableNode.java     |   7 +-
 .../waged/MockAssignmentMetadataStore.java         |  55 +++
 .../rebalancer/waged/TestWagedRebalancer.java      | 415 +++++++++++++++++++++
 .../waged/constraints/MockRebalanceAlgorithm.java  |  84 +++++
 .../waged/model/AbstractTestClusterModel.java      |   4 +-
 7 files changed, 850 insertions(+), 32 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
index d01fc60..a8b5055 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixRebalanceException.java
@@ -33,12 +33,12 @@ public class HelixRebalanceException extends Exception {
   private final Type _type;
 
   public HelixRebalanceException(String message, Type type, Throwable cause) {
-    super(String.format("%s. Failure Type: %s", message, type.name()), cause);
+    super(String.format("%s Failure Type: %s", message, type.name()), cause);
     _type = type;
   }
 
   public HelixRebalanceException(String message, Type type) {
-    super(String.format("%s. Failure Type: %s", message, type.name()));
+    super(String.format("%s Failure Type: %s", message, type.name()));
     _type = type;
   }
 
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
index 5b9634e..866c7c9 100644
--- 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
@@ -19,9 +19,18 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.changedetector.ResourceChangeDetector;
@@ -29,14 +38,18 @@ import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintBasedAlgorithmFactory;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModelProvider;
+import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceAssignment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * A placeholder before we have the implementation.
  * Weight-Aware Globally-Even Distribute Rebalancer.
  *
  * @see <a href="https://github.com/apache/helix/wiki/Design-Proposal---Weight-Aware-Globally-Even-Distribute-Rebalancer">
@@ -46,50 +59,296 @@ import org.slf4j.LoggerFactory;
 public class WagedRebalancer {
   private static final Logger LOG = LoggerFactory.getLogger(WagedRebalancer.class);
 
+  // When any of the following change happens, the rebalancer needs to do a global rebalance which
+  // contains 1. baseline recalculate, 2. partial rebalance that is based on the new baseline.
+  private static final Set<HelixConstants.ChangeType> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES =
+      Collections.unmodifiableSet(new HashSet<>(Arrays
+          .asList(HelixConstants.ChangeType.RESOURCE_CONFIG,
+              HelixConstants.ChangeType.CLUSTER_CONFIG,
+              HelixConstants.ChangeType.INSTANCE_CONFIG)));
+  // The cluster change detector is a stateful object.
+  // Make it static to avoid unnecessary reinitialization.
+  private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
+      new ThreadLocal<>();
+  private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
+
   // --------- The following fields are placeholders and need replacement. -----------//
   // TODO Shall we make the metadata store a static threadlocal object as well to avoid reinitialization?
   private final AssignmentMetadataStore _assignmentMetadataStore;
   private final RebalanceAlgorithm _rebalanceAlgorithm;
   // ------------------------------------------------------------------------------------//
 
-  // The cluster change detector is a stateful object. Make it static to avoid unnecessary
-  // reinitialization.
-  private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
-      new ThreadLocal<>();
-  private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
+  public WagedRebalancer(HelixManager helixManager) {
+    this(
+        // TODO init the metadata store according to their requirement when integrate,
+        //  or change to final static method if possible.
+        new AssignmentMetadataStore(),
+        // TODO parse the cluster setting
+        ConstraintBasedAlgorithmFactory.getInstance(),
+        // Use DelayedAutoRebalancer as the mapping calculator for the final assignment output.
+        // Mapping calculator will translate the best possible assignment into the applicable state
+        // mapping based on the current states.
+        // TODO abstract and separate the main mapping calculator logic from DelayedAutoRebalancer
+        new DelayedAutoRebalancer());
+  }
 
-  private ResourceChangeDetector getChangeDetector() {
-    if (CHANGE_DETECTOR_THREAD_LOCAL.get() == null) {
-      CHANGE_DETECTOR_THREAD_LOCAL.set(new ResourceChangeDetector());
-    }
-    return CHANGE_DETECTOR_THREAD_LOCAL.get();
+  private WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
+      RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator) {
+    _assignmentMetadataStore = assignmentMetadataStore;
+    _rebalanceAlgorithm = algorithm;
+    _mappingCalculator = mappingCalculator;
   }
 
-  public WagedRebalancer(HelixManager helixManager) {
-    // TODO init the metadata store according to their requirement when integrate, or change to final static method if possible.
-    _assignmentMetadataStore = new AssignmentMetadataStore();
-    // TODO parse the cluster setting
-    _rebalanceAlgorithm = ConstraintBasedAlgorithmFactory.getInstance();
-
-    // Use the mapping calculator in DelayedAutoRebalancer for calculating the final assignment
-    // output.
-    // This calculator will translate the best possible assignment into an applicable state mapping
-    // based on the current states.
-    // TODO abstract and separate the mapping calculator logic from the DelayedAutoRebalancer
-    _mappingCalculator = new DelayedAutoRebalancer();
+  @VisibleForTesting
+  protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
+      RebalanceAlgorithm algorithm) {
+    this(assignmentMetadataStore, algorithm, new DelayedAutoRebalancer());
   }
 
   /**
-   * Compute the new IdealStates for all the resources input. The IdealStates include both the new
+   * Compute the new IdealStates for all the input resources. The IdealStates include both new
    * partition assignment (in the listFiles) and the new replica state mapping (in the mapFields).
+   *
    * @param clusterData        The Cluster status data provider.
    * @param resourceMap        A map containing all the rebalancing resources.
-   * @param currentStateOutput The present Current State of the cluster.
-   * @return A map containing the computed new IdealStates.
+   * @param currentStateOutput The present Current States of the resources.
+   * @return A map of the new IdealStates with the resource name as key.
    */
   public Map<String, IdealState> computeNewIdealStates(ResourceControllerDataProvider clusterData,
       Map<String, Resource> resourceMap, final CurrentStateOutput currentStateOutput)
       throws HelixRebalanceException {
-    return new HashMap<>();
+    LOG.info("Start computing new ideal states for resources: {}", resourceMap.keySet().toString());
+
+    // Find the compatible resources: 1. FULL_AUTO 2. Configured to use the WAGED rebalancer
+    resourceMap = resourceMap.entrySet().stream().filter(resourceEntry -> {
+      IdealState is = clusterData.getIdealState(resourceEntry.getKey());
+      return is != null && is.getRebalanceMode().equals(IdealState.RebalanceMode.FULL_AUTO)
+          && getClass().getName().equals(is.getRebalancerClassName());
+    }).collect(Collectors
+        .toMap(resourceEntry -> resourceEntry.getKey(), resourceEntry -> resourceEntry.getValue()));
+
+    if (resourceMap.isEmpty()) {
+      LOG.warn("There is no valid resource to be rebalanced by {}",
+          this.getClass().getSimpleName());
+      return Collections.emptyMap();
+    } else {
+      LOG.info("Valid resources that will be rebalanced by {}: {}", this.getClass().getSimpleName(),
+          resourceMap.keySet().toString());
+    }
+
+    // Calculate the target assignment based on the current cluster status.
+    Map<String, IdealState> newIdealStates = computeBestPossibleStates(clusterData, resourceMap);
+
+    // Construct the new best possible states according to the current state and target assignment.
+    // Note that the new ideal state might be an intermediate state between the current state and the target assignment.
+    for (IdealState is : newIdealStates.values()) {
+      String resourceName = is.getResourceName();
+      // Adjust the states according to the current state.
+      ResourceAssignment finalAssignment = _mappingCalculator
+          .computeBestPossiblePartitionState(clusterData, is, resourceMap.get(resourceName),
+              currentStateOutput);
+
+      // Clean up the state mapping fields. Use the final assignment that is calculated by the
+      // mapping calculator to replace them.
+      is.getRecord().getMapFields().clear();
+      for (Partition partition : finalAssignment.getMappedPartitions()) {
+        Map<String, String> newStateMap = finalAssignment.getReplicaMap(partition);
+        // if the final states cannot be generated, override the best possible state with empty map.
+        is.setInstanceStateMap(partition.getPartitionName(),
+            newStateMap == null ? Collections.emptyMap() : newStateMap);
+      }
+    }
+
+    LOG.info("Finish computing new ideal states for resources: {}",
+        resourceMap.keySet().toString());
+    return newIdealStates;
+  }
+
+  // Coordinate baseline recalculation and partial rebalance according to the cluster changes.
+  private Map<String, IdealState> computeBestPossibleStates(
+      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap)
+      throws HelixRebalanceException {
+    getChangeDetector().updateSnapshots(clusterData);
+    // Get all the modified and new items' information
+    Map<HelixConstants.ChangeType, Set<String>> clusterChanges =
+        getChangeDetector().getChangeTypes().stream()
+            .collect(Collectors.toMap(changeType -> changeType, changeType -> {
+              Set<String> itemKeys = new HashSet<>();
+              itemKeys.addAll(getChangeDetector().getAdditionsByType(changeType));
+              itemKeys.addAll(getChangeDetector().getChangesByType(changeType));
+              return itemKeys;
+            }));
+
+    if (clusterChanges.keySet().stream()
+        .anyMatch(changeType -> GLOBAL_REBALANCE_REQUIRED_CHANGE_TYPES.contains(changeType))) {
+      refreshBaseline(clusterData, clusterChanges, resourceMap);
+      // Inject a cluster config change for large scale partial rebalance once the baseline changed.
+      clusterChanges.putIfAbsent(HelixConstants.ChangeType.CLUSTER_CONFIG, Collections.emptySet());
+    }
+
+    Map<String, ResourceAssignment> newAssignment =
+        partialRebalance(clusterData, clusterChanges, resourceMap);
+
+    // Convert the assignments into IdealState for the following state mapping calculation.
+    Map<String, IdealState> finalIdealState = new HashMap<>();
+    for (String resourceName : newAssignment.keySet()) {
+      IdealState newIdeaState;
+      try {
+        IdealState currentIdealState = clusterData.getIdealState(resourceName);
+        Map<String, Integer> statePriorityMap =
+            clusterData.getStateModelDef(currentIdealState.getStateModelDefRef())
+                .getStatePriorityMap();
+        // Create a new IdealState instance contains the new calculated assignment in the preference list.
+        newIdeaState = generateIdealStateWithAssignment(resourceName, currentIdealState,
+            newAssignment.get(resourceName), statePriorityMap);
+      } catch (Exception ex) {
+        throw new HelixRebalanceException(
+            "Fail to calculate the new IdealState for resource: " + resourceName,
+            HelixRebalanceException.Type.INVALID_CLUSTER_STATUS, ex);
+      }
+      finalIdealState.put(resourceName, newIdeaState);
+    }
+    return finalIdealState;
+  }
+
+  // TODO make the Baseline calculation async if complicated algorithm is used for the Baseline
+  private void refreshBaseline(ResourceControllerDataProvider clusterData,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap)
+      throws HelixRebalanceException {
+    // For baseline calculation
+    // 1. Ignore node status (disable/offline).
+    // 2. Use the baseline as the previous best possible assignment since there is no "baseline" for
+    // the baseline.
+    LOG.info("Start calculating the new baseline.");
+    Map<String, ResourceAssignment> currentBaseline;
+    try {
+      currentBaseline = _assignmentMetadataStore.getBaseline();
+    } catch (Exception ex) {
+      throw new HelixRebalanceException("Failed to get the current baseline assignment.",
+          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    }
+    Map<String, ResourceAssignment> baseline =
+        calculateAssignment(clusterData, clusterChanges, resourceMap, clusterData.getAllInstances(),
+            Collections.emptyMap(), currentBaseline);
+    try {
+      _assignmentMetadataStore.persistBaseline(baseline);
+    } catch (Exception ex) {
+      throw new HelixRebalanceException("Failed to persist the new baseline assignment.",
+          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    }
+    LOG.info("Finish calculating the new baseline.");
+  }
+
+  private Map<String, ResourceAssignment> partialRebalance(
+      ResourceControllerDataProvider clusterData,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap)
+      throws HelixRebalanceException {
+    LOG.info("Start calculating the new best possible assignment.");
+    Set<String> activeInstances = clusterData.getEnabledLiveInstances();
+    Map<String, ResourceAssignment> baseline;
+    Map<String, ResourceAssignment> prevBestPossibleAssignment;
+    try {
+      baseline = _assignmentMetadataStore.getBaseline();
+      prevBestPossibleAssignment = _assignmentMetadataStore.getBestPossibleAssignment();
+    } catch (Exception ex) {
+      throw new HelixRebalanceException("Failed to get the persisted assignment records.",
+          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    }
+    Map<String, ResourceAssignment> newAssignment =
+        calculateAssignment(clusterData, clusterChanges, resourceMap, activeInstances, baseline,
+            prevBestPossibleAssignment);
+    try {
+      // TODO Test to confirm if persisting the final assignment (with final partition states)
+      // would be a better option.
+      _assignmentMetadataStore.persistBestPossibleAssignment(newAssignment);
+    } catch (Exception ex) {
+      throw new HelixRebalanceException("Failed to persist the new best possible assignment.",
+          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS, ex);
+    }
+    LOG.info("Finish calculating the new best possible assignment.");
+    return newAssignment;
+  }
+
+  /**
+   * Generate the cluster model based on the input and calculate the optimal assignment.
+   *
+   * @param clusterData                the cluster data cache.
+   * @param clusterChanges             the detected cluster changes.
+   * @param resourceMap                the rebalancing resources.
+   * @param activeNodes                the alive and enabled nodes.
+   * @param baseline                   the baseline assignment for the algorithm as a reference.
+   * @param prevBestPossibleAssignment the previous best possible assignment for the algorithm as a reference.
+   * @return the new optimal assignment for the resources.
+   */
+  private Map<String, ResourceAssignment> calculateAssignment(
+      ResourceControllerDataProvider clusterData,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap,
+      Set<String> activeNodes, Map<String, ResourceAssignment> baseline,
+      Map<String, ResourceAssignment> prevBestPossibleAssignment) throws HelixRebalanceException {
+    long startTime = System.currentTimeMillis();
+    LOG.info("Start calculating for an assignment");
+    ClusterModel clusterModel;
+    try {
+      clusterModel = ClusterModelProvider
+          .generateClusterModel(clusterData, resourceMap, activeNodes, clusterChanges, baseline,
+              prevBestPossibleAssignment);
+    } catch (Exception ex) {
+      throw new HelixRebalanceException("Failed to generate cluster model.",
+          HelixRebalanceException.Type.INVALID_CLUSTER_STATUS, ex);
+    }
+
+    OptimalAssignment optimalAssignment = _rebalanceAlgorithm.calculate(clusterModel);
+    Map<String, ResourceAssignment> newAssignment =
+        optimalAssignment.getOptimalResourceAssignment();
+
+    LOG.info("Finish calculating. Time spent: {}ms.", System.currentTimeMillis() - startTime);
+    return newAssignment;
+  }
+
+  private ResourceChangeDetector getChangeDetector() {
+    if (CHANGE_DETECTOR_THREAD_LOCAL.get() == null) {
+      CHANGE_DETECTOR_THREAD_LOCAL.set(new ResourceChangeDetector());
+    }
+    return CHANGE_DETECTOR_THREAD_LOCAL.get();
+  }
+
+  // Generate a new IdealState based on the input newAssignment.
+  // The assignment will be propagate to the preference lists.
+  // Note that we will recalculate the states based on the current state, so there is no need to
+  // update the mapping fields in the IdealState output.
+  private IdealState generateIdealStateWithAssignment(String resourceName,
+      IdealState currentIdealState, ResourceAssignment newAssignment,
+      Map<String, Integer> statePriorityMap) {
+    IdealState newIdealState = new IdealState(resourceName);
+    // Copy the simple fields
+    newIdealState.getRecord().setSimpleFields(currentIdealState.getRecord().getSimpleFields());
+    // Sort the preference list according to state priority.
+    newIdealState.setPreferenceLists(getPreferenceLists(newAssignment, statePriorityMap));
+    // Note the state mapping in the new assignment won't be directly propagate to the map fields.
+    // The rebalancer will calculate for the final state mapping considering the current states.
+    return newIdealState;
+  }
+
+  // Generate the preference lists from the state mapping based on state priority.
+  private Map<String, List<String>> getPreferenceLists(ResourceAssignment newAssignment,
+      Map<String, Integer> statePriorityMap) {
+    Map<String, List<String>> preferenceList = new HashMap<>();
+    for (Partition partition : newAssignment.getMappedPartitions()) {
+      List<String> nodes = new ArrayList<>(newAssignment.getReplicaMap(partition).keySet());
+      // To ensure backward compatibility, sort the preference list according to state priority.
+      nodes.sort((node1, node2) -> {
+        int statePriority1 =
+            statePriorityMap.get(newAssignment.getReplicaMap(partition).get(node1));
+        int statePriority2 =
+            statePriorityMap.get(newAssignment.getReplicaMap(partition).get(node2));
+        if (statePriority1 == statePriority2) {
+          return node1.compareTo(node2);
+        } else {
+          return statePriority1 - statePriority2;
+        }
+      });
+      preferenceList.put(partition.getPartitionName(), nodes);
+    }
+    return preferenceList;
   }
 }
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
index 33677e5..4141d20 100644
--- 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
@@ -41,7 +41,7 @@ import static java.lang.Math.max;
  * This class represents a possible allocation of the replication.
  * Note that any usage updates to the AssignableNode are not thread safe.
  */
-public class AssignableNode {
+public class AssignableNode implements Comparable<AssignableNode> {
   private static final Logger LOG = LoggerFactory.getLogger(AssignableNode.class.getName());
 
   // basic node information
@@ -384,4 +384,9 @@ public class AssignableNode {
   public int hashCode() {
     return _instanceName.hashCode();
   }
+
+  @Override
+  public int compareTo(AssignableNode o) {
+    return _instanceName.compareTo(o.getInstanceName());
+  }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
new file mode 100644
index 0000000..ea8c164
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/MockAssignmentMetadataStore.java
@@ -0,0 +1,55 @@
+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.ResourceAssignment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A mock up metadata store for unit test.
+ * This mock datastore persist assignments in memory only.
+ */
+public class MockAssignmentMetadataStore extends AssignmentMetadataStore {
+  private Map<String, ResourceAssignment> _persistGlobalBaseline = new HashMap<>();
+  private Map<String, ResourceAssignment> _persistBestPossibleAssignment = new HashMap<>();
+
+  public Map<String, ResourceAssignment> getBaseline() {
+    return _persistGlobalBaseline;
+  }
+
+  public void persistBaseline(Map<String, ResourceAssignment> globalBaseline) {
+    _persistGlobalBaseline = globalBaseline;
+  }
+
+  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
+    return _persistBestPossibleAssignment;
+  }
+
+  public void persistBestPossibleAssignment(Map<String, ResourceAssignment> bestPossibleAssignment) {
+    _persistBestPossibleAssignment = bestPossibleAssignment;
+  }
+
+  public void clearMetadataStore() {
+    _persistBestPossibleAssignment.clear();
+    _persistGlobalBaseline.clear();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
new file mode 100644
index 0000000..6759a10
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
@@ -0,0 +1,415 @@
+package org.apache.helix.controller.rebalancer.waged;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixRebalanceException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
+import org.apache.helix.controller.rebalancer.waged.constraints.MockRebalanceAlgorithm;
+import org.apache.helix.controller.rebalancer.waged.model.AbstractTestClusterModel;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.ResourceAssignment;
+import org.mockito.Mockito;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.when;
+
+public class TestWagedRebalancer extends AbstractTestClusterModel {
+  private Set<String> _instances;
+  private MockRebalanceAlgorithm _algorithm;
+
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+    _instances = new HashSet<>();
+    _instances.add(_testInstanceId);
+    _algorithm = new MockRebalanceAlgorithm();
+  }
+
+  @Override
+  protected ResourceControllerDataProvider setupClusterDataCache() throws IOException {
+    ResourceControllerDataProvider testCache = super.setupClusterDataCache();
+
+    // Set up mock idealstate
+    Map<String, IdealState> isMap = new HashMap<>();
+    for (String resource : _resourceNames) {
+      IdealState is = new IdealState(resource);
+      is.setNumPartitions(_partitionNames.size());
+      is.setRebalanceMode(IdealState.RebalanceMode.FULL_AUTO);
+      is.setStateModelDefRef("MasterSlave");
+      is.setReplicas("3");
+      is.setRebalancerClassName(WagedRebalancer.class.getName());
+      _partitionNames.stream()
+          .forEach(partition -> is.setPreferenceList(partition, Collections.emptyList()));
+      isMap.put(resource, is);
+    }
+    when(testCache.getIdealState(anyString())).thenAnswer(
+        (Answer<IdealState>) invocationOnMock -> isMap.get(invocationOnMock.getArguments()[0]));
+    when(testCache.getIdealStates()).thenReturn(isMap);
+
+    // Set up 2 more instances
+    for (int i = 1; i < 3; i++) {
+      String instanceName = _testInstanceId + i;
+      _instances.add(instanceName);
+      // 1. Set up the default instance information with capacity configuration.
+      InstanceConfig testInstanceConfig = createMockInstanceConfig(instanceName);
+      Map<String, InstanceConfig> instanceConfigMap = testCache.getInstanceConfigMap();
+      instanceConfigMap.put(instanceName, testInstanceConfig);
+      when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+      // 2. Mock the live instance node for the default instance.
+      LiveInstance testLiveInstance = createMockLiveInstance(instanceName);
+      Map<String, LiveInstance> liveInstanceMap = testCache.getLiveInstances();
+      liveInstanceMap.put(instanceName, testLiveInstance);
+      when(testCache.getLiveInstances()).thenReturn(liveInstanceMap);
+      when(testCache.getEnabledInstances()).thenReturn(liveInstanceMap.keySet());
+      when(testCache.getEnabledLiveInstances()).thenReturn(liveInstanceMap.keySet());
+    }
+
+    return testCache;
+  }
+
+  @Test
+  public void testRebalance() throws IOException, HelixRebalanceException {
+    // Init mock metadatastore for the unit test
+    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+
+    // Generate the input for the rebalancer.
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    Map<String, ResourceAssignment> algorithmResult = _algorithm.getRebalanceResult();
+    // Since there is no special condition, the calculated IdealStates should be exactly the same
+    // as the mock algorithm result.
+    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testPartialRebalance() throws IOException, HelixRebalanceException {
+    // Init mock metadatastore for the unit test
+    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+
+    // Generate the input for the rebalancer.
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+
+    // Test with partial resources listed in the resourceMap input.
+    // Remove the first resource from the input. Note it still exists in the cluster data cache.
+    metadataStore.clearMetadataStore();
+    resourceMap.remove(_resourceNames.get(0));
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    Map<String, ResourceAssignment> algorithmResult = _algorithm.getRebalanceResult();
+    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testRebalanceWithCurrentState() throws IOException, HelixRebalanceException {
+    // Init mock metadatastore for the unit test
+    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+
+    // Generate the input for the rebalancer.
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+
+    // Test with current state exists, so the rebalancer should calculate for the intermediate state
+    // Create current state based on the cluster data cache.
+    CurrentStateOutput currentStateOutput = new CurrentStateOutput();
+    for (String instanceName : _instances) {
+      for (Map.Entry<String, CurrentState> csEntry : clusterData
+          .getCurrentState(instanceName, _sessionId).entrySet()) {
+        String resourceName = csEntry.getKey();
+        CurrentState cs = csEntry.getValue();
+        for (Map.Entry<String, String> partitionStateEntry : cs.getPartitionStateMap().entrySet()) {
+          currentStateOutput
+              .setCurrentState(resourceName, new Partition(partitionStateEntry.getKey()),
+                  instanceName, partitionStateEntry.getValue());
+        }
+      }
+    }
+
+    // The state calculation will be adjusted based on the current state.
+    // So test the following cases:
+    // 1.1. Disable a resource, and the partitions in CS will be offline.
+    String disabledResourceName = _resourceNames.get(0);
+    clusterData.getIdealState(disabledResourceName).enable(false);
+    // 1.2. Adding more unknown partitions to the CS, so they will be dropped.
+    String droppingResourceName = _resourceNames.get(1);
+    String droppingPartitionName = "UnknownPartition";
+    String droppingFromInstance = _testInstanceId;
+    currentStateOutput.setCurrentState(droppingResourceName, new Partition(droppingPartitionName),
+        droppingFromInstance, "SLAVE");
+    resourceMap.get(droppingResourceName).addPartition(droppingPartitionName);
+
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, currentStateOutput);
+    // All the replica state should be OFFLINE
+    IdealState disabledIdealState = newIdealStates.get(disabledResourceName);
+    for (String partition : disabledIdealState.getPartitionSet()) {
+      Assert.assertTrue(disabledIdealState.getInstanceStateMap(partition).values().stream()
+          .allMatch(state -> state.equals("OFFLINE")));
+    }
+    // the dropped partition should be dropped.
+    IdealState droppedIdealState = newIdealStates.get(droppingResourceName);
+    Assert.assertEquals(
+        droppedIdealState.getInstanceStateMap(droppingPartitionName).get(droppingFromInstance),
+        "DROPPED");
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testNonCompatibleConfiguration() throws IOException, HelixRebalanceException {
+    WagedRebalancer rebalancer = new WagedRebalancer(new MockAssignmentMetadataStore(), _algorithm);
+
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    String nonCompatibleResourceName = _resourceNames.get(0);
+    clusterData.getIdealState(nonCompatibleResourceName)
+        .setRebalancerClassName(CrushRebalanceStrategy.class.getName());
+    // The input resource Map shall contain all the valid resources.
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    Map<String, ResourceAssignment> algorithmResult = _algorithm.getRebalanceResult();
+    // The output shall not contains the nonCompatibleResource.
+    resourceMap.remove(nonCompatibleResourceName);
+    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+  }
+
+  // TODO test with invalid capacity configuration which will fail the cluster model constructing.
+  @Test(dependsOnMethods = "testRebalance")
+  public void testInvalidClusterStatus() throws IOException {
+    WagedRebalancer rebalancer = new WagedRebalancer(new MockAssignmentMetadataStore(), _algorithm);
+
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    String invalidResource = _resourceNames.get(0);
+    // The state model does not exist
+    clusterData.getIdealState(invalidResource).setStateModelDefRef("foobar");
+    // The input resource Map shall contain all the valid resources.
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().keySet().stream().collect(
+        Collectors.toMap(resourceName -> resourceName, resourceName -> new Resource(resourceName)));
+    try {
+      rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+      Assert.fail("Rebalance shall fail.");
+    } catch (HelixRebalanceException ex) {
+      Assert.assertEquals(ex.getFailureType(), HelixRebalanceException.Type.INVALID_CLUSTER_STATUS);
+      Assert.assertEquals(ex.getMessage(),
+          "Failed to generate cluster model. Failure Type: INVALID_CLUSTER_STATUS");
+    }
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testInvalidRebalancerStatus() throws IOException {
+    // Mock a metadata store that will fail on all the calls.
+    AssignmentMetadataStore metadataStore = Mockito.mock(AssignmentMetadataStore.class);
+    when(metadataStore.getBaseline())
+        .thenThrow(new RuntimeException("Mock Error. Metadata store fails."));
+    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    // The input resource Map shall contain all the valid resources.
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().keySet().stream().collect(
+        Collectors.toMap(resourceName -> resourceName, resourceName -> new Resource(resourceName)));
+    try {
+      rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+      Assert.fail("Rebalance shall fail.");
+    } catch (HelixRebalanceException ex) {
+      Assert.assertEquals(ex.getFailureType(),
+          HelixRebalanceException.Type.INVALID_REBALANCER_STATUS);
+      Assert.assertEquals(ex.getMessage(),
+          "Failed to get the persisted assignment records. Failure Type: INVALID_REBALANCER_STATUS");
+    }
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testAlgorithmExepction() throws IOException, HelixRebalanceException {
+    RebalanceAlgorithm badAlgorithm = Mockito.mock(RebalanceAlgorithm.class);
+    when(badAlgorithm.calculate(any())).thenThrow(new HelixRebalanceException("Algorithm fails.",
+        HelixRebalanceException.Type.FAILED_TO_CALCULATE));
+
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(new MockAssignmentMetadataStore(), badAlgorithm);
+
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().keySet().stream().collect(
+        Collectors.toMap(resourceName -> resourceName, resourceName -> new Resource(resourceName)));
+    try {
+      rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+      Assert.fail("Rebalance shall fail.");
+    } catch (HelixRebalanceException ex) {
+      Assert.assertEquals(ex.getFailureType(), HelixRebalanceException.Type.FAILED_TO_CALCULATE);
+      Assert.assertEquals(ex.getMessage(), "Algorithm fails. Failure Type: FAILED_TO_CALCULATE");
+    }
+  }
+
+  @Test(dependsOnMethods = "testRebalance")
+  public void testRebalanceOnChanges() throws IOException, HelixRebalanceException {
+    // Test continuously rebalance with the same rebalancer with different internal state. Ensure
+    // that the rebalancer handles different input (different cluster changes) based on the internal
+    // state in a correct way.
+
+    // Note that this test relies on the MockRebalanceAlgorithm implementation. The mock algorithm
+    // won't propagate any existing assignment from the cluster model.
+
+    // Init mock metadatastore for the unit test
+    MockAssignmentMetadataStore metadataStore = new MockAssignmentMetadataStore();
+    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+
+    // 1. rebalance with baseline calculation done
+    // Generate the input for the rebalancer.
+    ResourceControllerDataProvider clusterData = setupClusterDataCache();
+    // Cluster config change will trigger baseline to be recalculated.
+    when(clusterData.getRefreshedChangeTypes())
+        .thenReturn(Collections.singleton(HelixConstants.ChangeType.CLUSTER_CONFIG));
+    Map<String, Resource> resourceMap = clusterData.getIdealStates().entrySet().stream()
+        .collect(Collectors.toMap(entry -> entry.getKey(), entry -> {
+          Resource resource = new Resource(entry.getKey());
+          entry.getValue().getPartitionSet().stream()
+              .forEach(partition -> resource.addPartition(partition));
+          return resource;
+        }));
+    Map<String, IdealState> newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    Map<String, ResourceAssignment> algorithmResult = _algorithm.getRebalanceResult();
+    // Since there is no special condition, the calculated IdealStates should be exactly the same
+    // as the mock algorithm result.
+    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+    Map<String, ResourceAssignment> baseline = metadataStore.getBaseline();
+    Assert.assertEquals(baseline, algorithmResult);
+    Map<String, ResourceAssignment> bestPossibleAssignment =
+        metadataStore.getBestPossibleAssignment();
+    Assert.assertEquals(bestPossibleAssignment, algorithmResult);
+
+    // 2. rebalance with one ideal state changed only
+    String changedResourceName = _resourceNames.get(0);
+    // Create a new cluster data cache to simulate cluster change
+    clusterData = setupClusterDataCache();
+    when(clusterData.getRefreshedChangeTypes())
+        .thenReturn(Collections.singleton(HelixConstants.ChangeType.IDEAL_STATE));
+    IdealState is = clusterData.getIdealState(changedResourceName);
+    // Update the tag so the ideal state will be marked as changed.
+    is.setInstanceGroupTag("newTag");
+
+    // Although the input contains 2 resources, the rebalancer shall only call the algorithm to
+    // rebalance the changed one.
+    newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    Map<String, ResourceAssignment> partialAlgorithmResult = _algorithm.getRebalanceResult();
+
+    // Verify that only the changed resource has been included in the calculation.
+    validateRebalanceResult(
+        Collections.singletonMap(changedResourceName, new Resource(changedResourceName)),
+        newIdealStates, partialAlgorithmResult);
+    // Baseline should be empty, because there is no cluster topology change.
+    baseline = metadataStore.getBaseline();
+    Assert.assertEquals(baseline, Collections.emptyMap());
+    // Best possible assignment contains the new assignment of only one resource.
+    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    Assert.assertEquals(bestPossibleAssignment, partialAlgorithmResult);
+
+    // * Before the next test, recover the best possible assignment record.
+    metadataStore.persistBestPossibleAssignment(algorithmResult);
+
+    // 3. rebalance with current state change only
+    // Create a new cluster data cache to simulate cluster change
+    clusterData = setupClusterDataCache();
+    when(clusterData.getRefreshedChangeTypes())
+        .thenReturn(Collections.singleton(HelixConstants.ChangeType.CURRENT_STATE));
+    // Modify any current state
+    CurrentState cs =
+        clusterData.getCurrentState(_testInstanceId, _sessionId).get(_resourceNames.get(0));
+    // Update the tag so the ideal state will be marked as changed.
+    cs.setInfo(_partitionNames.get(0), "mock update");
+
+    // Although the input contains 2 resources, the rebalancer shall not try to recalculate
+    // assignment since there is only current state change.
+    newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    algorithmResult = _algorithm.getRebalanceResult();
+
+    // Verify that only the changed resource has been included in the calculation.
+    validateRebalanceResult(Collections.emptyMap(), newIdealStates, algorithmResult);
+    // Both assignment state should be empty.
+    baseline = metadataStore.getBaseline();
+    Assert.assertEquals(baseline, Collections.emptyMap());
+    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    Assert.assertEquals(bestPossibleAssignment, Collections.emptyMap());
+
+    // 4. rebalance with no change but best possible state record missing.
+    // This usually happens when the persisted assignment state is gone.
+    clusterData = setupClusterDataCache(); // Note this mock data cache won't report any change.
+    // Even with no change, since the previous assignment is empty, the rebalancer will still
+    // calculate the assignment for both resources.
+    newIdealStates =
+        rebalancer.computeNewIdealStates(clusterData, resourceMap, new CurrentStateOutput());
+    algorithmResult = _algorithm.getRebalanceResult();
+    // Verify that both resource has been included in the calculation.
+    validateRebalanceResult(resourceMap, newIdealStates, algorithmResult);
+    // Both assignment state should be empty since no cluster topology change.
+    baseline = metadataStore.getBaseline();
+    Assert.assertEquals(baseline, Collections.emptyMap());
+    // The best possible assignment should be present.
+    bestPossibleAssignment = metadataStore.getBestPossibleAssignment();
+    Assert.assertEquals(bestPossibleAssignment, algorithmResult);
+  }
+
+  private void validateRebalanceResult(Map<String, Resource> resourceMap,
+      Map<String, IdealState> newIdealStates, Map<String, ResourceAssignment> expectedResult) {
+    Assert.assertEquals(newIdealStates.keySet(), resourceMap.keySet());
+    for (String resourceName : expectedResult.keySet()) {
+      Assert.assertTrue(newIdealStates.containsKey(resourceName));
+      IdealState is = newIdealStates.get(resourceName);
+      ResourceAssignment assignment = expectedResult.get(resourceName);
+      Assert.assertEquals(is.getPartitionSet(), new HashSet<>(
+          assignment.getMappedPartitions().stream().map(partition -> partition.getPartitionName())
+              .collect(Collectors.toSet())));
+      for (String partitionName : is.getPartitionSet()) {
+        Assert.assertEquals(is.getInstanceStateMap(partitionName),
+            assignment.getReplicaMap(new Partition(partitionName)));
+      }
+    }
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java
new file mode 100644
index 0000000..2a39482
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/MockRebalanceAlgorithm.java
@@ -0,0 +1,84 @@
+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.AssignableNode;
+import org.apache.helix.controller.rebalancer.waged.model.AssignableReplica;
+import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
+import org.apache.helix.controller.rebalancer.waged.model.OptimalAssignment;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.ResourceAssignment;
+import org.mockito.Mockito;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.mockito.Mockito.when;
+
+/**
+ * A mock up rebalance algorithm for unit test.
+ * Note that the mock algorithm won't propagate the existing assignment to the output as a real
+ * algorithm will do. This is for the convenience of testing.
+ */
+public class MockRebalanceAlgorithm implements RebalanceAlgorithm {
+  Map<String, ResourceAssignment> _resultHistory = Collections.emptyMap();
+
+  @Override
+  public OptimalAssignment calculate(ClusterModel clusterModel) {
+    // If no predefined rebalance result setup, do card dealing.
+    Map<String, ResourceAssignment> result = new HashMap<>();
+    Iterator<AssignableNode> nodeIterator =
+        clusterModel.getAssignableNodes().values().stream().sorted().iterator();
+    for (String resource : clusterModel.getAssignableReplicaMap().keySet()) {
+      Iterator<AssignableReplica> replicaIterator =
+          clusterModel.getAssignableReplicaMap().get(resource).stream().sorted().iterator();
+      while (replicaIterator.hasNext()) {
+        AssignableReplica replica = replicaIterator.next();
+        if (!nodeIterator.hasNext()) {
+          nodeIterator = clusterModel.getAssignableNodes().values().stream().sorted().iterator();
+        }
+        AssignableNode node = nodeIterator.next();
+
+        // Put the assignment
+        ResourceAssignment assignment = result.computeIfAbsent(replica.getResourceName(),
+            resourceName -> new ResourceAssignment(resourceName));
+        Partition partition = new Partition(replica.getPartitionName());
+        if (assignment.getReplicaMap(partition).isEmpty()) {
+          assignment.addReplicaMap(partition, new HashMap<>());
+        }
+        assignment.getReplicaMap(partition).put(node.getInstanceName(), replica.getReplicaState());
+      }
+    }
+
+    _resultHistory = result;
+
+    // TODO remove this mockito when OptimalAssignment.getOptimalResourceAssignment is ready.
+    OptimalAssignment optimalAssignment = Mockito.mock(OptimalAssignment.class);
+    when(optimalAssignment.getOptimalResourceAssignment()).thenReturn(result);
+    return optimalAssignment;
+  }
+
+  public Map<String, ResourceAssignment> getRebalanceResult() {
+    return _resultHistory;
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
index a8a5de5..0f799b3 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -74,7 +74,7 @@ public abstract class AbstractTestClusterModel {
     _testFaultZoneId = "testZone";
   }
 
-  InstanceConfig createMockInstanceConfig(String instanceId) {
+  protected InstanceConfig createMockInstanceConfig(String instanceId) {
     InstanceConfig testInstanceConfig = new InstanceConfig(instanceId);
     testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
     testInstanceConfig.addTag(_testInstanceTags.get(0));
@@ -83,7 +83,7 @@ public abstract class AbstractTestClusterModel {
     return testInstanceConfig;
   }
 
-  LiveInstance createMockLiveInstance(String instanceId) {
+  protected LiveInstance createMockLiveInstance(String instanceId) {
     LiveInstance testLiveInstance = new LiveInstance(instanceId);
     testLiveInstance.setSessionId(_sessionId);
     return testLiveInstance;


[helix] 28/37: Adjust the expected replica count according to fault zone count. (#476)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 679547fb511754cce2863a51e10ffcc6d8ab98b0
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Tue Sep 17 13:41:56 2019 -0700

    Adjust the expected replica count according to fault zone count. (#476)
    
    The rebalancer should determine the expected replica count according to the fault zone instead of the node count only.
---
 .../rebalancer/waged/model/AssignableNode.java     | 56 ++++++++++------------
 .../waged/model/ClusterModelProvider.java          | 28 ++++++-----
 .../waged/model/ClusterModelTestHelper.java        |  3 +-
 .../rebalancer/waged/model/TestAssignableNode.java | 24 ++++------
 .../rebalancer/waged/model/TestClusterModel.java   |  3 +-
 .../waged/model/TestClusterModelProvider.java      | 33 ++++++++-----
 6 files changed, 76 insertions(+), 71 deletions(-)

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
index 6966353..a3460fb 100644
--- 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
@@ -64,12 +64,10 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * @param clusterConfig
    * @param instanceConfig
    * @param instanceName
-   * @param existingAssignment A collection of replicas that have been pre-allocated to the node.
    */
-  AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName,
-      Collection<AssignableReplica> existingAssignment) {
+  AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName) {
     _instanceName = instanceName;
-    refresh(clusterConfig, instanceConfig, existingAssignment);
+    refresh(clusterConfig, instanceConfig);
   }
 
   private void reset() {
@@ -88,10 +86,8 @@ public class AssignableNode implements Comparable<AssignableNode> {
    * subject to change. If the assumption is no longer true, this function should become private.
    * @param clusterConfig - the Cluster Config of the cluster where the node is located
    * @param instanceConfig - the Instance Config of the node
-   * @param existingAssignment - all the existing replicas that are current assigned to the node
    */
-  private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig,
-      Collection<AssignableReplica> existingAssignment) {
+  private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig) {
     reset();
 
     Map<String, Integer> instanceCapacity = fetchInstanceCapacity(clusterConfig, instanceConfig);
@@ -101,8 +97,29 @@ public class AssignableNode implements Comparable<AssignableNode> {
     _disabledPartitionsMap = instanceConfig.getDisabledPartitionsMap();
     _maxCapacity = instanceCapacity;
     _maxPartition = clusterConfig.getMaxPartitionsPerInstance();
+  }
+
+  /**
+   * This function should only be used to assign a set of new partitions that are not allocated on
+   * this node.
+   * Using this function avoids the overhead of updating capacity repeatedly.
+   */
+  void assignNewBatch(Collection<AssignableReplica> replicas) {
+    Map<String, Integer> totalPartitionCapacity = new HashMap<>();
+    for (AssignableReplica replica : replicas) {
+      addToAssignmentRecord(replica);
+      // increment the capacity requirement according to partition's capacity configuration.
+      for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
+        totalPartitionCapacity.compute(capacity.getKey(),
+            (key, totalValue) -> (totalValue == null) ? capacity.getValue()
+                : totalValue + capacity.getValue());
+      }
+    }
 
-    assignNewBatch(existingAssignment);
+    // Update the global state after all single replications' calculation is done.
+    for (String key : totalPartitionCapacity.keySet()) {
+      updateCapacityAndUtilization(key, totalPartitionCapacity.get(key));
+    }
   }
 
   /**
@@ -315,29 +332,6 @@ public class AssignableNode implements Comparable<AssignableNode> {
   }
 
   /**
-   * This function should only be used to assign a set of new partitions that are not allocated on
-   * this node.
-   * Using this function avoids the overhead of updating capacity repeatedly.
-   */
-  private void assignNewBatch(Collection<AssignableReplica> replicas) {
-    Map<String, Integer> totalPartitionCapacity = new HashMap<>();
-    for (AssignableReplica replica : replicas) {
-      addToAssignmentRecord(replica);
-      // increment the capacity requirement according to partition's capacity configuration.
-      for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
-        totalPartitionCapacity.compute(capacity.getKey(),
-            (key, totalValue) -> (totalValue == null) ? capacity.getValue()
-                : totalValue + capacity.getValue());
-      }
-    }
-
-    // Update the global state after all single replications' calculation is done.
-    for (String key : totalPartitionCapacity.keySet()) {
-      updateCapacityAndUtilization(key, totalPartitionCapacity.get(key));
-    }
-  }
-
-  /**
    * @throws HelixException if the replica has already been assigned to the node.
    */
   private void addToAssignmentRecord(AssignableReplica replica) {
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index 3570164..20024c7 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -62,10 +62,15 @@ public class ClusterModelProvider {
       Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
       Map<String, ResourceAssignment> baselineAssignment,
       Map<String, ResourceAssignment> bestPossibleAssignment) {
+    // Construct all the assignable nodes and initialize with the allocated replicas.
+    Set<AssignableNode> assignableNodes =
+        parseAllNodes(dataProvider.getClusterConfig(), dataProvider.getInstanceConfigMap(),
+            activeInstances);
+
     // Generate replica objects for all the resource partitions.
     // <resource, replica set>
     Map<String, Set<AssignableReplica>> replicaMap =
-        parseAllReplicas(dataProvider, resourceMap, activeInstances.size());
+        parseAllReplicas(dataProvider, resourceMap, assignableNodes);
 
     // Check if the replicas need to be reassigned.
     Map<String, Set<AssignableReplica>> allocatedReplicas =
@@ -74,10 +79,9 @@ public class ClusterModelProvider {
         findToBeAssignedReplicas(replicaMap, clusterChanges, activeInstances,
             bestPossibleAssignment, allocatedReplicas);
 
-    // Construct all the assignable nodes and initialize with the allocated replicas.
-    Set<AssignableNode> assignableNodes =
-        parseAllNodes(dataProvider.getClusterConfig(), dataProvider.getInstanceConfigMap(),
-            activeInstances, allocatedReplicas);
+    // Update the allocated replicas to the assignable nodes.
+    assignableNodes.stream().forEach(node -> node.assignNewBatch(
+        allocatedReplicas.getOrDefault(node.getInstanceName(), Collections.emptySet())));
 
     // Construct and initialize cluster context.
     ClusterContext context = new ClusterContext(
@@ -171,15 +175,13 @@ public class ClusterModelProvider {
    * @param clusterConfig     The cluster configuration.
    * @param instanceConfigMap A map of all the instance configuration.
    * @param activeInstances   All the instances that are online and enabled.
-   * @param allocatedReplicas A map of all the assigned replicas, which will not be reassigned during the rebalance.
    * @return A map of assignable node set, <InstanceName, node set>.
    */
   private static Set<AssignableNode> parseAllNodes(ClusterConfig clusterConfig,
-      Map<String, InstanceConfig> instanceConfigMap, Set<String> activeInstances,
-      Map<String, Set<AssignableReplica>> allocatedReplicas) {
+      Map<String, InstanceConfig> instanceConfigMap, Set<String> activeInstances) {
     return activeInstances.stream().map(
         instanceName -> new AssignableNode(clusterConfig, instanceConfigMap.get(instanceName),
-            instanceName, allocatedReplicas.getOrDefault(instanceName, Collections.emptySet())))
+            instanceName))
         .collect(Collectors.toSet());
   }
 
@@ -188,11 +190,12 @@ public class ClusterModelProvider {
    *
    * @param dataProvider The cluster status cache that contains the current cluster status.
    * @param resourceMap  All the valid resources that are managed by the rebalancer.
+   * @param assignableNodes All the active assignable nodes.
    * @return A map of assignable replica set, <ResourceName, replica set>.
    */
   private static Map<String, Set<AssignableReplica>> parseAllReplicas(
       ResourceControllerDataProvider dataProvider, Map<String, Resource> resourceMap,
-      int instanceCount) {
+      Set<AssignableNode> assignableNodes) {
     Map<String, Set<AssignableReplica>> totalReplicaMap = new HashMap<>();
     ClusterConfig clusterConfig = dataProvider.getClusterConfig();
 
@@ -211,8 +214,11 @@ public class ClusterModelProvider {
                 is.getStateModelDefRef(), resourceName));
       }
 
+      int activeFaultZoneCount =
+          assignableNodes.stream().map(node -> node.getFaultZone()).collect(Collectors.toSet())
+              .size();
       Map<String, Integer> stateCountMap =
-          def.getStateCountMap(instanceCount, is.getReplicaCount(instanceCount));
+          def.getStateCountMap(activeFaultZoneCount, is.getReplicaCount(assignableNodes.size()));
 
       for (String partition : is.getPartitionSet()) {
         for (Map.Entry<String, Integer> entry : stateCountMap.entrySet()) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
index 76f1141..08143c6 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelTestHelper.java
@@ -43,8 +43,7 @@ public class ClusterModelTestHelper extends AbstractTestClusterModel {
     Set<AssignableNode> nodeSet = new HashSet<>();
     testCache.getInstanceConfigMap().values().stream()
             .forEach(config -> nodeSet.add(new AssignableNode(testCache.getClusterConfig(),
-                    testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName(),
-                    Collections.emptyList())));
+                    testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName())));
     return nodeSet;
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
index 92a6998..6975901 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -64,7 +64,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     expectedCapacityMap.put("item3", 30);
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
+    assignableNode.assignNewBatch(assignmentSet);
     Assert.assertEquals(assignableNode.getAssignedPartitionsMap(), expectedAssignment);
     Assert.assertEquals(assignableNode.getAssignedReplicaCount(), 4);
     Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
@@ -167,8 +168,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     ResourceControllerDataProvider testCache = setupClusterDataCache();
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
-        Collections.emptyList());
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
     AssignableReplica removingReplica = new AssignableReplica(testCache.getClusterConfig(),
         testCache.getResourceConfig(_resourceNames.get(1)), _partitionNames.get(2) + "non-exist",
         "MASTER", 1);
@@ -183,7 +183,8 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
+    assignableNode.assignNewBatch(assignmentSet);
     AssignableReplica duplicateReplica = new AssignableReplica(testCache.getClusterConfig(),
         testCache.getResourceConfig(_resourceNames.get(0)), _partitionNames.get(0), "SLAVE", 2);
     assignableNode.assign(duplicateReplica);
@@ -206,8 +207,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
 
     new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
-        Collections.emptyList());
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
   }
 
   @Test
@@ -227,8 +227,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
 
     AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
-        Collections.emptyList());
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
 
     Assert.assertEquals(assignableNode.getFaultZone(), "2/");
 
@@ -245,8 +244,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
 
     assignableNode = new AssignableNode(testCache.getClusterConfig(),
-        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
-        Collections.emptyList());
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId);
 
     Assert.assertEquals(assignableNode.getFaultZone(), "2/testInstance/");
   }
@@ -259,8 +257,7 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
 
     AssignableNode assignableNode =
-        new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId,
-            Collections.emptyList());
+        new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId);
     Assert.assertEquals(assignableNode.getMaxCapacity(), _capacityDataMap);
   }
 
@@ -274,7 +271,6 @@ public class TestAssignableNode extends AbstractTestClusterModel {
     InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
     testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
 
-    new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId,
-        Collections.emptyList());
+    new AssignableNode(testClusterConfig, testInstanceConfig, _testInstanceId);
   }
 }
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
index a45b729..5112413 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
@@ -43,8 +43,7 @@ public class TestClusterModel extends AbstractTestClusterModel {
     Set<AssignableNode> nodeSet = new HashSet<>();
     testCache.getInstanceConfigMap().values().stream().forEach(config -> nodeSet.add(
         new AssignableNode(testCache.getClusterConfig(),
-            testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName(),
-            Collections.emptyList())));
+            testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName())));
     return nodeSet;
   }
 
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index 1ec92a9..ad608b6 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -19,6 +19,14 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 import org.apache.helix.HelixConstants;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.rebalancer.waged.WagedRebalancer;
@@ -34,14 +42,6 @@ import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.when;
 
@@ -111,7 +111,18 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     Assert.assertEquals(
         clusterModel.getAssignableNodes().values().stream().map(AssignableNode::getInstanceName)
             .collect(Collectors.toSet()), _instances);
-    // Shall have 2 resources and 12 replicas
+    // Shall have 2 resources and 4 replicas, since all nodes are in the same fault zone.
+    Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
+    Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
+        .allMatch(replicaSet -> replicaSet.size() == 4));
+
+    // Adjust instance fault zone, so they have different fault zones.
+    testCache.getInstanceConfigMap().values().stream()
+        .forEach(config -> config.setZoneId(config.getInstanceName()));
+    clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
+            .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
+        _instances, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
+    // Shall have 2 resources and 12 replicas after fault zone adjusted.
     Assert.assertEquals(clusterModel.getAssignableReplicaMap().size(), 2);
     Assert.assertTrue(clusterModel.getAssignableReplicaMap().values().stream()
         .allMatch(replicaSet -> replicaSet.size() == 12));
@@ -197,10 +208,10 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
         _instances, Collections.singletonMap(HelixConstants.ChangeType.RESOURCE_CONFIG,
             Collections.singleton(changedResourceName)), Collections.emptyMap(),
         bestPossibleAssignment);
-    // There should be no existing assignment for all the resource except for resource2.
+    // There should be no existing assignment for all the resource except for resource2
     Assert.assertEquals(clusterModel.getContext().getAssignmentForFaultZoneMap().size(), 1);
     Map<String, Set<String>> resourceAssignmentMap =
-        clusterModel.getContext().getAssignmentForFaultZoneMap().get(_testFaultZoneId);
+        clusterModel.getContext().getAssignmentForFaultZoneMap().get(_testInstanceId);
     // Should be only resource2 in the map
     Assert.assertEquals(resourceAssignmentMap.size(), 1);
     for (String resource : _resourceNames) {


[helix] 26/37: Add soft constraint: ResourcePartitionAntiAffinityConstraint (#464)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e3cb1a64a1243f9afd9ab2670c1af37603c81cfc
Author: Yi Wang <i3...@gmail.com>
AuthorDate: Tue Sep 17 10:09:27 2019 -0700

    Add soft constraint: ResourcePartitionAntiAffinityConstraint (#464)
    
    If the resource of the partition overall has a light load on the instance, the score is higher compared to case when the resource is heavily loaded on the instance
---
 .../ResourcePartitionAntiAffinityConstraint.java   | 52 +++++++++++++++++
 ...estResourcePartitionAntiAffinityConstraint.java | 67 ++++++++++++++++++++++
 2 files changed, 119 insertions(+)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourcePartitionAntiAffinityConstraint.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourcePartitionAntiAffinityConstraint.java
new file mode 100644
index 0000000..a2f9099
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/constraints/ResourcePartitionAntiAffinityConstraint.java
@@ -0,0 +1,52 @@
+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;
+
+/**
+ * This constraint exists to make partitions belonging to the same resource be assigned as far from
+ * each other as possible. This is because it is undesirable to have many partitions belonging to
+ * the same resource be assigned to the same node to minimize the impact of node failure scenarios.
+ * The score is higher the fewer the partitions are on the node belonging to the same resource.
+ */
+class ResourcePartitionAntiAffinityConstraint extends SoftConstraint {
+  private static final float MAX_SCORE = 1f;
+  private static final float MIN_SCORE = 0f;
+
+  ResourcePartitionAntiAffinityConstraint() {
+    super(MAX_SCORE, MIN_SCORE);
+  }
+
+  @Override
+  protected float getAssignmentScore(AssignableNode node, AssignableReplica replica,
+      ClusterContext clusterContext) {
+    String resource = replica.getResourceName();
+    int curPartitionCountForResource = node.getAssignedPartitionsByResource(resource).size();
+    int doubleMaxPartitionCountForResource =
+        2 * clusterContext.getEstimatedMaxPartitionByResource(resource);
+    // The score measures the twice the max allowed count versus current counts
+    // The returned value is a measurement of remaining quota ratio, in the case of exceeding allowed counts, return 0
+    return Math.max(((float) doubleMaxPartitionCountForResource - curPartitionCountForResource)
+        / doubleMaxPartitionCountForResource, 0);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourcePartitionAntiAffinityConstraint.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourcePartitionAntiAffinityConstraint.java
new file mode 100644
index 0000000..c6830cf
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/constraints/TestResourcePartitionAntiAffinityConstraint.java
@@ -0,0 +1,67 @@
+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 static org.mockito.Mockito.when;
+
+import java.util.Collections;
+
+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;
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class TestResourcePartitionAntiAffinityConstraint {
+  private static final String TEST_PARTITION = "TestPartition";
+  private static final String TEST_RESOURCE = "TestResource";
+  private final AssignableReplica _testReplica = Mockito.mock(AssignableReplica.class);
+  private final AssignableNode _testNode = Mockito.mock(AssignableNode.class);
+  private final ClusterContext _clusterContext = Mockito.mock(ClusterContext.class);
+  private final SoftConstraint _constraint = new ResourcePartitionAntiAffinityConstraint();
+
+  @Test
+  public void testGetAssignmentScore() {
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testNode.getAssignedPartitionsByResource(TEST_RESOURCE)).thenReturn(
+        ImmutableSet.of(TEST_PARTITION + "1", TEST_PARTITION + "2", TEST_PARTITION + "3"));
+    when(_clusterContext.getEstimatedMaxPartitionByResource(TEST_RESOURCE)).thenReturn(10);
+
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 0.85f);
+    Assert.assertEquals(normalizedScore, 0.85f);
+  }
+
+  @Test
+  public void testGetAssignmentScoreMaxScore() {
+    when(_testReplica.getResourceName()).thenReturn(TEST_RESOURCE);
+    when(_testNode.getAssignedPartitionsByResource(TEST_RESOURCE)).thenReturn(Collections.emptySet());
+    when(_clusterContext.getEstimatedMaxPartitionByResource(TEST_RESOURCE)).thenReturn(10);
+
+    float score = _constraint.getAssignmentScore(_testNode, _testReplica, _clusterContext);
+    float normalizedScore = _constraint.getAssignmentNormalizedScore(_testNode, _testReplica, _clusterContext);
+    Assert.assertEquals(score, 1f);
+    Assert.assertEquals(normalizedScore, 1f);
+  }
+}


[helix] 13/37: Modify the expected change type from CONFIG to CLUSTER_CONFIG in the WAGED rebalancer. (#438)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2780fd99d0347af4eef08c293db7741c422c8af8
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Thu Aug 29 10:48:38 2019 -0700

    Modify the expected change type from CONFIG to CLUSTER_CONFIG in the WAGED rebalancer. (#438)
    
    CONFIG is for generic configuration items. That will be too generic for the rebalancer.
    Modify to check for CLUSTER_CONFIG to avoid confusion.
---
 .../helix/controller/rebalancer/waged/model/ClusterModelProvider.java   | 2 +-
 .../controller/rebalancer/waged/model/TestClusterModelProvider.java     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
index c4f7d02..e0a5e35 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModelProvider.java
@@ -112,7 +112,7 @@ public class ClusterModelProvider {
       Map<String, ResourceAssignment> bestPossibleAssignment,
       Map<String, Set<AssignableReplica>> allocatedReplicas) {
     Set<AssignableReplica> toBeAssignedReplicas = new HashSet<>();
-    if (clusterChanges.containsKey(HelixConstants.ChangeType.CONFIG)
+    if (clusterChanges.containsKey(HelixConstants.ChangeType.CLUSTER_CONFIG)
         || clusterChanges.containsKey(HelixConstants.ChangeType.INSTANCE_CONFIG)) {
       // If the cluster topology has been modified, need to reassign all replicas
       toBeAssignedReplicas
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
index 1221b6f..638182f 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModelProvider.java
@@ -177,7 +177,7 @@ public class TestClusterModelProvider extends AbstractTestClusterModel {
     // 5. test with best possible assignment but cluster topology is changed
     clusterModel = ClusterModelProvider.generateClusterModel(testCache, _resourceNames.stream()
             .collect(Collectors.toMap(resource -> resource, resource -> new Resource(resource))),
-        _instances, Collections.singletonMap(HelixConstants.ChangeType.CONFIG,
+        _instances, Collections.singletonMap(HelixConstants.ChangeType.CLUSTER_CONFIG,
             Collections.emptySet()), Collections.emptyMap(), bestPossibleAssignment);
     // There should be no existing assignment since the topology change invalidates all existing assignment
     Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()


[helix] 04/37: Change the rebalancer assignment record to be ResourceAssignment instead of IdealState. (#398)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 60dc54a7c5267b6abd03cf8c80338e481fd7e6bf
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Fri Aug 9 15:35:54 2019 -0700

    Change the rebalancer assignment record to be ResourceAssignment instead of IdealState. (#398)
    
    ResourceAssignment fit the usage better. And there will be no unnecessary information to be recorded or read during the rebalance calculation.
---
 .../controller/rebalancer/waged/RebalanceAlgorithm.java |  3 ++-
 .../constraints/ConstraintsRebalanceAlgorithm.java      |  3 ++-
 .../controller/rebalancer/waged/model/ClusterModel.java | 17 +++++++++--------
 3 files changed, 13 insertions(+), 10 deletions(-)

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
index 0e6c891..ae258ca 100644
--- 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
@@ -22,6 +22,7 @@ package org.apache.helix.controller.rebalancer.waged;
 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 org.apache.helix.model.ResourceAssignment;
 
 import java.util.Map;
 
@@ -40,6 +41,6 @@ public interface RebalanceAlgorithm {
    *                       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, ResourceAssignment> rebalance(ClusterModel clusterModel,
       Map<String, Map<HardConstraint.FailureReason, Integer>> failureReasons);
 }
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
index 292d903..a75854a 100644
--- 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
@@ -22,6 +22,7 @@ package org.apache.helix.controller.rebalancer.waged.constraints;
 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.apache.helix.model.ResourceAssignment;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +43,7 @@ public class ConstraintsRebalanceAlgorithm implements RebalanceAlgorithm {
   }
 
   @Override
-  public Map<String, IdealState> rebalance(ClusterModel clusterModel,
+  public Map<String, ResourceAssignment> 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/model/ClusterModel.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/waged/model/ClusterModel.java
index 2908939..1be527a 100644
--- 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
@@ -21,6 +21,7 @@ package org.apache.helix.controller.rebalancer.waged.model;
 
 import org.apache.helix.HelixException;
 import org.apache.helix.model.IdealState;
+import org.apache.helix.model.ResourceAssignment;
 
 import java.util.Collections;
 import java.util.Map;
@@ -40,10 +41,10 @@ public class ClusterModel {
   private final Map<String, AssignableNode> _assignableNodeMap;
 
   // Records about the previous assignment
-  // <ResourceName, IdealState contains the baseline assignment>
-  private final Map<String, IdealState> _baselineAssignment;
-  // <ResourceName, IdealState contains the best possible assignment>
-  private final Map<String, IdealState> _bestPossibleAssignment;
+  // <ResourceName, ResourceAssignment contains the baseline assignment>
+  private final Map<String, ResourceAssignment> _baselineAssignment;
+  // <ResourceName, ResourceAssignment contains the best possible assignment>
+  private final Map<String, ResourceAssignment> _bestPossibleAssignment;
 
   /**
    * @param clusterContext         The initialized cluster context.
@@ -54,8 +55,8 @@ public class ClusterModel {
    * @param bestPossibleAssignment The current best possible assignment.
    */
   ClusterModel(ClusterContext clusterContext, Set<AssignableReplica> assignableReplicas,
-      Set<AssignableNode> assignableNodes, Map<String, IdealState> baselineAssignment,
-      Map<String, IdealState> bestPossibleAssignment) {
+      Set<AssignableNode> assignableNodes, Map<String, ResourceAssignment> baselineAssignment,
+      Map<String, ResourceAssignment> bestPossibleAssignment) {
     _clusterContext = clusterContext;
 
     // Save all the to be assigned replication
@@ -87,11 +88,11 @@ public class ClusterModel {
     return _assignableReplicaMap;
   }
 
-  public Map<String, IdealState> getBaseline() {
+  public Map<String, ResourceAssignment> getBaseline() {
     return _baselineAssignment;
   }
 
-  public Map<String, IdealState> getBestPossibleAssignment() {
+  public Map<String, ResourceAssignment> getBestPossibleAssignment() {
     return _bestPossibleAssignment;
   }
 


[helix] 33/37: Add delayed rebalance and user-defined preference list features to the WAGED rebalancer. (#456)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 88ff70502dc7b1d017a601c3ef02e1936c47f1ea
Author: Jiajun Wang <18...@users.noreply.github.com>
AuthorDate: Tue Oct 1 12:08:56 2019 -0700

    Add delayed rebalance and user-defined preference list features to the WAGED rebalancer. (#456)
    
    - Add delayed rebalance and user-defined preference list features to the WAGED rebalancer.
    - Refine the delayed rebalance usage in the waged rebalancer.
    - Add the delayed rebalance scheduling logic.
    - Add the necessary tests. And fix TestMixedModeAutoRebalance and all delayed rebalance tests.
---
 .../rebalancer/DelayedAutoRebalancer.java          | 203 ++--------------
 .../rebalancer/util/DelayedRebalanceUtil.java      | 267 +++++++++++++++++++++
 .../rebalancer/waged/WagedRebalancer.java          | 159 ++++++++++--
 .../StrictMatchExternalViewVerifier.java           |   6 +-
 .../java/org/apache/helix/common/ZkTestBase.java   |   4 +-
 .../rebalancer/waged/TestWagedRebalancer.java      |  35 ++-
 .../TestDelayedAutoRebalance.java                  |  57 +++--
 ...stDelayedAutoRebalanceWithDisabledInstance.java |  33 +--
 .../TestDelayedAutoRebalanceWithRackaware.java     |   5 +-
 .../rebalancer/TestMixedModeAutoRebalance.java     | 101 +++++---
 .../rebalancer/TestZeroReplicaAvoidance.java       |  74 ++++--
 .../WagedRebalancer/TestDelayedWagedRebalance.java | 102 ++++++++
 ...tDelayedWagedRebalanceWithDisabledInstance.java | 103 ++++++++
 .../TestDelayedWagedRebalanceWithRackaware.java    | 102 ++++++++
 .../TestMixedModeWagedRebalance.java               |  66 +++++
 .../WagedRebalancer/TestWagedRebalance.java        |  23 +-
 .../TestWagedRebalanceFaultZone.java               |  10 +-
 .../apache/helix/tools/TestClusterVerifier.java    |  45 +++-
 18 files changed, 1059 insertions(+), 336 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
index 65b3f84..1073d6d 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/DelayedAutoRebalancer.java
@@ -32,11 +32,10 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
-import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
+import org.apache.helix.controller.rebalancer.util.DelayedRebalanceUtil;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.ResourceAssignment;
@@ -50,7 +49,6 @@ import org.slf4j.LoggerFactory;
  */
 public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceControllerDataProvider> {
   private static final Logger LOG = LoggerFactory.getLogger(DelayedAutoRebalancer.class);
-  private static RebalanceScheduler _rebalanceScheduler = new RebalanceScheduler();
 
   @Override
   public IdealState computeNewIdealState(String resourceName,
@@ -79,7 +77,8 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
 
     ClusterConfig clusterConfig = clusterData.getClusterConfig();
     ResourceConfig resourceConfig = clusterData.getResourceConfig(resourceName);
-    boolean delayRebalanceEnabled = isDelayRebalanceEnabled(currentIdealState, clusterConfig);
+    boolean delayRebalanceEnabled =
+        DelayedRebalanceUtil.isDelayRebalanceEnabled(currentIdealState, clusterConfig);
 
     if (resourceConfig != null) {
       userDefinedPreferenceList = resourceConfig.getPreferenceLists();
@@ -110,16 +109,18 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
 
     Set<String> activeNodes = liveEnabledNodes;
     if (delayRebalanceEnabled) {
-      long delay = getRebalanceDelay(currentIdealState, clusterConfig);
-      activeNodes = getActiveInstances(allNodes, currentIdealState, liveEnabledNodes,
-          clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
-          clusterData.getInstanceConfigMap(), delay, clusterConfig);
+      long delay = DelayedRebalanceUtil.getRebalanceDelay(currentIdealState, clusterConfig);
+      activeNodes = DelayedRebalanceUtil
+          .getActiveNodes(allNodes, currentIdealState, liveEnabledNodes,
+              clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
+              clusterData.getInstanceConfigMap(), delay, clusterConfig);
 
       Set<String> offlineOrDisabledInstances = new HashSet<>(activeNodes);
       offlineOrDisabledInstances.removeAll(liveEnabledNodes);
-      setRebalanceScheduler(currentIdealState, offlineOrDisabledInstances,
-          clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
-          clusterData.getInstanceConfigMap(), delay, clusterConfig);
+      DelayedRebalanceUtil.setRebalanceScheduler(currentIdealState.getResourceName(), true,
+          offlineOrDisabledInstances, clusterData.getInstanceOfflineTimeMap(),
+          clusterData.getLiveInstances().keySet(), clusterData.getInstanceConfigMap(), delay,
+          clusterConfig, _manager);
     }
 
     if (allNodes.isEmpty() || activeNodes.isEmpty()) {
@@ -162,16 +163,16 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
         .computePartitionAssignment(allNodeList, liveEnabledNodeList, currentMapping, clusterData);
     ZNRecord finalMapping = newIdealMapping;
 
-    if (isDelayRebalanceEnabled(currentIdealState, clusterConfig)) {
+    if (DelayedRebalanceUtil.isDelayRebalanceEnabled(currentIdealState, clusterConfig)) {
       List<String> activeNodeList = new ArrayList<>(activeNodes);
       Collections.sort(activeNodeList);
-      int minActiveReplicas = getMinActiveReplica(currentIdealState, replicaCount);
+      int minActiveReplicas =
+          DelayedRebalanceUtil.getMinActiveReplica(currentIdealState, replicaCount);
 
       ZNRecord newActiveMapping = _rebalanceStrategy
           .computePartitionAssignment(allNodeList, activeNodeList, currentMapping, clusterData);
-      finalMapping =
-          getFinalDelayedMapping(currentIdealState, newIdealMapping, newActiveMapping, liveEnabledNodes,
-              replicaCount, minActiveReplicas);
+      finalMapping = getFinalDelayedMapping(currentIdealState, newIdealMapping, newActiveMapping,
+          liveEnabledNodes, replicaCount, minActiveReplicas);
     }
 
     finalMapping.getListFields().putAll(userDefinedPreferenceList);
@@ -202,162 +203,15 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
     return newIdealState;
   }
 
-  /* get all active instances (live instances plus offline-yet-active instances */
-  private Set<String> getActiveInstances(Set<String> allNodes, IdealState idealState,
-      Set<String> liveEnabledNodes, Map<String, Long> instanceOfflineTimeMap, Set<String> liveNodes,
-      Map<String, InstanceConfig> instanceConfigMap, long delay, ClusterConfig clusterConfig) {
-    Set<String> activeInstances = new HashSet<>(liveEnabledNodes);
-
-    if (!isDelayRebalanceEnabled(idealState, clusterConfig)) {
-      return activeInstances;
-    }
-
-    Set<String> offlineOrDisabledInstances = new HashSet<>(allNodes);
-    offlineOrDisabledInstances.removeAll(liveEnabledNodes);
-
-    long currentTime = System.currentTimeMillis();
-    for (String ins : offlineOrDisabledInstances) {
-      long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
-          instanceConfigMap.get(ins), clusterConfig);
-      InstanceConfig instanceConfig = instanceConfigMap.get(ins);
-      if (inactiveTime > currentTime && instanceConfig != null && instanceConfig
-          .isDelayRebalanceEnabled()) {
-        activeInstances.add(ins);
-      }
-    }
-
-    return activeInstances;
-  }
-
-  /* Set a rebalance scheduler for the closest future rebalance time. */
-  private void setRebalanceScheduler(IdealState idealState, Set<String> offlineOrDisabledInstances,
-      Map<String, Long> instanceOfflineTimeMap, Set<String> liveNodes,
-      Map<String, InstanceConfig> instanceConfigMap,  long delay,
-      ClusterConfig clusterConfig) {
-    String resourceName = idealState.getResourceName();
-    if (!isDelayRebalanceEnabled(idealState, clusterConfig)) {
-      _rebalanceScheduler.removeScheduledRebalance(resourceName);
-      return;
-    }
-
-    long currentTime = System.currentTimeMillis();
-    long nextRebalanceTime = Long.MAX_VALUE;
-    // calculate the closest future rebalance time
-    for (String ins : offlineOrDisabledInstances) {
-      long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
-          instanceConfigMap.get(ins), clusterConfig);
-      if (inactiveTime != -1 && inactiveTime > currentTime && inactiveTime < nextRebalanceTime) {
-        nextRebalanceTime = inactiveTime;
-      }
-    }
-
-    if (nextRebalanceTime == Long.MAX_VALUE) {
-      long startTime = _rebalanceScheduler.removeScheduledRebalance(resourceName);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(String
-            .format("Remove exist rebalance timer for resource %s at %d\n", resourceName, startTime));
-      }
-    } else {
-      long currentScheduledTime = _rebalanceScheduler.getRebalanceTime(resourceName);
-      if (currentScheduledTime < 0 || currentScheduledTime > nextRebalanceTime) {
-        _rebalanceScheduler.scheduleRebalance(_manager, resourceName, nextRebalanceTime);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(String
-              .format("Set next rebalance time for resource %s at time %d\n", resourceName,
-                  nextRebalanceTime));
-        }
-      }
-    }
-  }
-
-  /**
-   * The time when an offline or disabled instance should be treated as inactive. return -1 if it is
-   * inactive now.
-   *
-   * @return
-   */
-  private long getInactiveTime(String instance, Set<String> liveInstances, Long offlineTime,
-      long delay, InstanceConfig instanceConfig, ClusterConfig clusterConfig) {
-    long inactiveTime = Long.MAX_VALUE;
-
-    // check the time instance went offline.
-    if (!liveInstances.contains(instance)) {
-      if (offlineTime != null && offlineTime > 0 && offlineTime + delay < inactiveTime) {
-        inactiveTime = offlineTime + delay;
-      }
-    }
-
-    // check the time instance got disabled.
-    if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
-        && clusterConfig.getDisabledInstances().containsKey(instance))) {
-      long disabledTime = instanceConfig.getInstanceEnabledTime();
-      if (clusterConfig.getDisabledInstances() != null && clusterConfig.getDisabledInstances()
-          .containsKey(instance)) {
-        // Update batch disable time
-        long batchDisableTime = Long.parseLong(clusterConfig.getDisabledInstances().get(instance));
-        if (disabledTime == -1 || disabledTime > batchDisableTime) {
-          disabledTime = batchDisableTime;
-        }
-      }
-      if (disabledTime > 0 && disabledTime + delay < inactiveTime) {
-        inactiveTime = disabledTime + delay;
-      }
-    }
-
-    if (inactiveTime == Long.MAX_VALUE) {
-      return -1;
-    }
-
-    return inactiveTime;
-  }
-
-  private long getRebalanceDelay(IdealState idealState, ClusterConfig clusterConfig) {
-    long delayTime = idealState.getRebalanceDelay();
-    if (delayTime < 0) {
-      delayTime = clusterConfig.getRebalanceDelayTime();
-    }
-    return delayTime;
-  }
-
-  private boolean isDelayRebalanceEnabled(IdealState idealState, ClusterConfig clusterConfig) {
-    long delay = getRebalanceDelay(idealState, clusterConfig);
-    return (delay > 0 && idealState.isDelayRebalanceEnabled() && clusterConfig
-        . isDelayRebalaceEnabled());
-  }
-
   private ZNRecord getFinalDelayedMapping(IdealState idealState, ZNRecord newIdealMapping,
       ZNRecord newActiveMapping, Set<String> liveInstances, int numReplica, int minActiveReplica) {
     if (minActiveReplica >= numReplica) {
       return newIdealMapping;
     }
     ZNRecord finalMapping = new ZNRecord(idealState.getResourceName());
-    for (String partition : newIdealMapping.getListFields().keySet()) {
-      List<String> idealList = newIdealMapping.getListField(partition);
-      List<String> activeList = newActiveMapping.getListField(partition);
-
-      List<String> liveList = new ArrayList<>();
-      int activeReplica = 0;
-      for (String ins : activeList) {
-        if (liveInstances.contains(ins)) {
-          activeReplica++;
-          liveList.add(ins);
-        }
-      }
-
-      if (activeReplica >= minActiveReplica) {
-        finalMapping.setListField(partition, activeList);
-      } else {
-        List<String> candidates = new ArrayList<String>(idealList);
-        candidates.removeAll(activeList);
-        for (String liveIns : candidates) {
-          liveList.add(liveIns);
-          if (liveList.size() >= minActiveReplica) {
-            break;
-          }
-        }
-        finalMapping.setListField(partition, liveList);
-      }
-    }
+    finalMapping.setListFields(DelayedRebalanceUtil
+        .getFinalDelayedMapping(newIdealMapping.getListFields(), newActiveMapping.getListFields(),
+            liveInstances, minActiveReplica));
     return finalMapping;
   }
 
@@ -391,10 +245,11 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
     Set<String> liveNodes = cache.getLiveInstances().keySet();
 
     ClusterConfig clusterConfig = cache.getClusterConfig();
-    long delayTime = getRebalanceDelay(idealState, clusterConfig);
-    Set<String> activeNodes = getActiveInstances(allNodes, idealState, liveNodes,
-        cache.getInstanceOfflineTimeMap(), cache.getLiveInstances().keySet(),
-        cache.getInstanceConfigMap(), delayTime, clusterConfig);
+    long delayTime = DelayedRebalanceUtil.getRebalanceDelay(idealState, clusterConfig);
+    Set<String> activeNodes = DelayedRebalanceUtil
+        .getActiveNodes(allNodes, idealState, liveNodes, cache.getInstanceOfflineTimeMap(),
+            cache.getLiveInstances().keySet(), cache.getInstanceConfigMap(), delayTime,
+            clusterConfig);
 
     String stateModelDefName = idealState.getStateModelDefRef();
     StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
@@ -419,14 +274,6 @@ public class DelayedAutoRebalancer extends AbstractRebalancer<ResourceController
     return partitionMapping;
   }
 
-  private int getMinActiveReplica(IdealState idealState, int replicaCount) {
-    int minActiveReplicas = idealState.getMinActiveReplicas();
-    if (minActiveReplicas < 0) {
-      minActiveReplicas = replicaCount;
-    }
-    return minActiveReplicas;
-  }
-
   /**
    * compute best state for resource in AUTO ideal state mode
    * @param liveInstances
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/DelayedRebalanceUtil.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/DelayedRebalanceUtil.java
new file mode 100644
index 0000000..1342860
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/util/DelayedRebalanceUtil.java
@@ -0,0 +1,267 @@
+package org.apache.helix.controller.rebalancer.util;
+
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.InstanceConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The util for supporting delayed rebalance logic.
+ */
+public class DelayedRebalanceUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(DelayedRebalanceUtil.class);
+
+  private static RebalanceScheduler REBALANCE_SCHEDULER = new RebalanceScheduler();
+
+  /**
+   * @return true if delay rebalance is configured and enabled in the ClusterConfig configurations.
+   */
+  public static boolean isDelayRebalanceEnabled(ClusterConfig clusterConfig) {
+    long delay = clusterConfig.getRebalanceDelayTime();
+    return (delay > 0 && clusterConfig.isDelayRebalaceEnabled());
+  }
+
+  /**
+   * @return true if delay rebalance is configured and enabled in Resource IdealState and the
+   * ClusterConfig configurations.
+   */
+  public static boolean isDelayRebalanceEnabled(IdealState idealState,
+      ClusterConfig clusterConfig) {
+    long delay = getRebalanceDelay(idealState, clusterConfig);
+    return (delay > 0 && idealState.isDelayRebalanceEnabled() && clusterConfig
+        .isDelayRebalaceEnabled());
+  }
+
+  /**
+   * @return the rebalance delay based on Resource IdealState and the ClusterConfig configurations.
+   */
+  public static long getRebalanceDelay(IdealState idealState, ClusterConfig clusterConfig) {
+    long delayTime = idealState.getRebalanceDelay();
+    if (delayTime < 0) {
+      delayTime = clusterConfig.getRebalanceDelayTime();
+    }
+    return delayTime;
+  }
+
+  /**
+   * @return all active nodes (live nodes plus offline-yet-active nodes) while considering cluster
+   * delay rebalance configurations.
+   */
+  public static Set<String> getActiveNodes(Set<String> allNodes, Set<String> liveEnabledNodes,
+      Map<String, Long> instanceOfflineTimeMap, Set<String> liveNodes,
+      Map<String, InstanceConfig> instanceConfigMap, ClusterConfig clusterConfig) {
+    if (!isDelayRebalanceEnabled(clusterConfig)) {
+      return new HashSet<>(liveEnabledNodes);
+    }
+    return getActiveNodes(allNodes, liveEnabledNodes, instanceOfflineTimeMap, liveNodes,
+        instanceConfigMap, clusterConfig.getRebalanceDelayTime(), clusterConfig);
+  }
+
+  /**
+   * @return all active nodes (live nodes plus offline-yet-active nodes) while considering cluster
+   * and the resource delay rebalance configurations.
+   */
+  public static Set<String> getActiveNodes(Set<String> allNodes, IdealState idealState,
+      Set<String> liveEnabledNodes, Map<String, Long> instanceOfflineTimeMap, Set<String> liveNodes,
+      Map<String, InstanceConfig> instanceConfigMap, long delay, ClusterConfig clusterConfig) {
+    if (!isDelayRebalanceEnabled(idealState, clusterConfig)) {
+      return new HashSet<>(liveEnabledNodes);
+    }
+    return getActiveNodes(allNodes, liveEnabledNodes, instanceOfflineTimeMap, liveNodes,
+        instanceConfigMap, delay, clusterConfig);
+  }
+
+  private static Set<String> getActiveNodes(Set<String> allNodes, Set<String> liveEnabledNodes,
+      Map<String, Long> instanceOfflineTimeMap, Set<String> liveNodes,
+      Map<String, InstanceConfig> instanceConfigMap, long delay, ClusterConfig clusterConfig) {
+    Set<String> activeNodes = new HashSet<>(liveEnabledNodes);
+    Set<String> offlineOrDisabledInstances = new HashSet<>(allNodes);
+    offlineOrDisabledInstances.removeAll(liveEnabledNodes);
+    long currentTime = System.currentTimeMillis();
+    for (String ins : offlineOrDisabledInstances) {
+      long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
+          instanceConfigMap.get(ins), clusterConfig);
+      InstanceConfig instanceConfig = instanceConfigMap.get(ins);
+      if (inactiveTime > currentTime && instanceConfig != null && instanceConfig
+          .isDelayRebalanceEnabled()) {
+        activeNodes.add(ins);
+      }
+    }
+    return activeNodes;
+  }
+
+  /**
+   * @return The time when an offline or disabled instance should be treated as inactive.
+   * Return -1 if it is inactive now.
+   */
+  private static long getInactiveTime(String instance, Set<String> liveInstances, Long offlineTime,
+      long delay, InstanceConfig instanceConfig, ClusterConfig clusterConfig) {
+    long inactiveTime = Long.MAX_VALUE;
+
+    // check the time instance went offline.
+    if (!liveInstances.contains(instance)) {
+      if (offlineTime != null && offlineTime > 0 && offlineTime + delay < inactiveTime) {
+        inactiveTime = offlineTime + delay;
+      }
+    }
+
+    // check the time instance got disabled.
+    if (!instanceConfig.getInstanceEnabled() || (clusterConfig.getDisabledInstances() != null
+        && clusterConfig.getDisabledInstances().containsKey(instance))) {
+      long disabledTime = instanceConfig.getInstanceEnabledTime();
+      if (clusterConfig.getDisabledInstances() != null && clusterConfig.getDisabledInstances()
+          .containsKey(instance)) {
+        // Update batch disable time
+        long batchDisableTime = Long.parseLong(clusterConfig.getDisabledInstances().get(instance));
+        if (disabledTime == -1 || disabledTime > batchDisableTime) {
+          disabledTime = batchDisableTime;
+        }
+      }
+      if (disabledTime > 0 && disabledTime + delay < inactiveTime) {
+        inactiveTime = disabledTime + delay;
+      }
+    }
+
+    if (inactiveTime == Long.MAX_VALUE) {
+      return -1;
+    }
+
+    return inactiveTime;
+  }
+
+  /**
+   * Merge the new ideal preference list with the delayed mapping that is calculated based on the
+   * delayed rebalance configurations.
+   * The method will prioritize the "active" preference list so as to avoid unnecessary transient
+   * state transitions.
+   *
+   * @param newIdealPreferenceList  the ideal mapping that was calculated based on the current
+   *                                instance status
+   * @param newDelayedPreferenceList the delayed mapping that was calculated based on the delayed
+   *                                 instance status
+   * @param liveEnabledInstances    list of all the nodes that are both alive and enabled.
+   * @param minActiveReplica        the minimum replica count to ensure a valid mapping.
+   *                                If the active list does not have enough replica assignment,
+   *                                this method will fill the list with the new ideal mapping until
+   *                                the replica count satisfies the minimum requirement.
+   * @return the merged state mapping.
+   */
+  public static Map<String, List<String>> getFinalDelayedMapping(
+      Map<String, List<String>> newIdealPreferenceList,
+      Map<String, List<String>> newDelayedPreferenceList, Set<String> liveEnabledInstances,
+      int minActiveReplica) {
+    Map<String, List<String>> finalPreferenceList = new HashMap<>();
+    for (String partition : newIdealPreferenceList.keySet()) {
+      List<String> idealList = newIdealPreferenceList.get(partition);
+      List<String> delayedIdealList = newDelayedPreferenceList.get(partition);
+
+      List<String> liveList = new ArrayList<>();
+      for (String ins : delayedIdealList) {
+        if (liveEnabledInstances.contains(ins)) {
+          liveList.add(ins);
+        }
+      }
+
+      if (liveList.size() >= minActiveReplica) {
+        finalPreferenceList.put(partition, delayedIdealList);
+      } else {
+        List<String> candidates = new ArrayList<>(idealList);
+        candidates.removeAll(delayedIdealList);
+        for (String liveIns : candidates) {
+          liveList.add(liveIns);
+          if (liveList.size() >= minActiveReplica) {
+            break;
+          }
+        }
+        finalPreferenceList.put(partition, liveList);
+      }
+    }
+    return finalPreferenceList;
+  }
+
+  /**
+   * Get the minimum active replica count threshold that allows delayed rebalance.
+   *
+   * @param idealState      the resource Ideal State
+   * @param replicaCount the expected active replica count.
+   * @return the expected minimum active replica count that is required
+   */
+  public static int getMinActiveReplica(IdealState idealState, int replicaCount) {
+    int minActiveReplicas = idealState.getMinActiveReplicas();
+    if (minActiveReplicas < 0) {
+      minActiveReplicas = replicaCount;
+    }
+    return minActiveReplicas;
+  }
+
+  /**
+   * Set a rebalance scheduler for the closest future rebalance time.
+   */
+  public static void setRebalanceScheduler(String resourceName, boolean isDelayedRebalanceEnabled,
+      Set<String> offlineOrDisabledInstances, Map<String, Long> instanceOfflineTimeMap,
+      Set<String> liveNodes, Map<String, InstanceConfig> instanceConfigMap, long delay,
+      ClusterConfig clusterConfig, HelixManager manager) {
+    if (!isDelayedRebalanceEnabled) {
+      REBALANCE_SCHEDULER.removeScheduledRebalance(resourceName);
+      return;
+    }
+
+    long currentTime = System.currentTimeMillis();
+    long nextRebalanceTime = Long.MAX_VALUE;
+    // calculate the closest future rebalance time
+    for (String ins : offlineOrDisabledInstances) {
+      long inactiveTime = getInactiveTime(ins, liveNodes, instanceOfflineTimeMap.get(ins), delay,
+          instanceConfigMap.get(ins), clusterConfig);
+      if (inactiveTime != -1 && inactiveTime > currentTime && inactiveTime < nextRebalanceTime) {
+        nextRebalanceTime = inactiveTime;
+      }
+    }
+
+    if (nextRebalanceTime == Long.MAX_VALUE) {
+      long startTime = REBALANCE_SCHEDULER.removeScheduledRebalance(resourceName);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String
+            .format("Remove exist rebalance timer for resource %s at %d\n", resourceName,
+                startTime));
+      }
+    } else {
+      long currentScheduledTime = REBALANCE_SCHEDULER.getRebalanceTime(resourceName);
+      if (currentScheduledTime < 0 || currentScheduledTime > nextRebalanceTime) {
+        REBALANCE_SCHEDULER.scheduleRebalance(manager, resourceName, nextRebalanceTime);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String
+              .format("Set next rebalance time for resource %s at time %d\n", resourceName,
+                  nextRebalanceTime));
+        }
+      }
+    }
+  }
+}
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
index 1861e10..d211884 100644
--- 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
@@ -19,6 +19,7 @@ package org.apache.helix.controller.rebalancer.waged;
  * under the License.
  */
 
+import com.google.common.collect.ImmutableSet;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -27,7 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
@@ -36,6 +36,7 @@ import org.apache.helix.controller.changedetector.ResourceChangeDetector;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
 import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
+import org.apache.helix.controller.rebalancer.util.DelayedRebalanceUtil;
 import org.apache.helix.controller.rebalancer.waged.constraints.ConstraintBasedAlgorithmFactory;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModel;
 import org.apache.helix.controller.rebalancer.waged.model.ClusterModelProvider;
@@ -46,12 +47,10 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.ResourceAssignment;
+import org.apache.helix.model.ResourceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableSet;
-
 /**
  * Weight-Aware Globally-Even Distribute Rebalancer.
  * @see <a
@@ -73,6 +72,7 @@ public class WagedRebalancer {
   // Make it static to avoid unnecessary reinitialization.
   private static final ThreadLocal<ResourceChangeDetector> CHANGE_DETECTOR_THREAD_LOCAL =
       new ThreadLocal<>();
+  private final HelixManager _manager;
   private final MappingCalculator<ResourceControllerDataProvider> _mappingCalculator;
   private final AssignmentMetadataStore _assignmentMetadataStore;
   private final RebalanceAlgorithm _rebalanceAlgorithm;
@@ -97,11 +97,18 @@ public class WagedRebalancer {
         // Mapping calculator will translate the best possible assignment into the applicable state
         // mapping based on the current states.
         // TODO abstract and separate the main mapping calculator logic from DelayedAutoRebalancer
-        new DelayedAutoRebalancer());
+        new DelayedAutoRebalancer(),
+        // Helix Manager is required for the rebalancer scheduler
+        helixManager);
   }
 
-  private WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
+  protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
       RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator) {
+    this(assignmentMetadataStore, algorithm, mappingCalculator, null);
+  }
+
+  private WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
+      RebalanceAlgorithm algorithm, MappingCalculator mappingCalculator, HelixManager manager) {
     if (assignmentMetadataStore == null) {
       LOG.warn("Assignment Metadata Store is not configured properly."
           + " The rebalancer will not access the assignment store during the rebalance.");
@@ -109,12 +116,7 @@ public class WagedRebalancer {
     _assignmentMetadataStore = assignmentMetadataStore;
     _rebalanceAlgorithm = algorithm;
     _mappingCalculator = mappingCalculator;
-  }
-
-  @VisibleForTesting
-  protected WagedRebalancer(AssignmentMetadataStore assignmentMetadataStore,
-      RebalanceAlgorithm algorithm) {
-    this(assignmentMetadataStore, algorithm, new DelayedAutoRebalancer());
+    _manager = manager;
   }
 
   // Release all the resources.
@@ -196,29 +198,59 @@ public class WagedRebalancer {
       clusterChanges.putIfAbsent(HelixConstants.ChangeType.CLUSTER_CONFIG, Collections.emptySet());
     }
 
+    Set<String> activeNodes = DelayedRebalanceUtil
+        .getActiveNodes(clusterData.getAllInstances(), clusterData.getEnabledLiveInstances(),
+            clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
+            clusterData.getInstanceConfigMap(), clusterData.getClusterConfig());
+
+    // Schedule (or unschedule) delayed rebalance according to the delayed rebalance config.
+    delayedRebalanceSchedule(clusterData, activeNodes, resourceMap.keySet());
+
     Map<String, ResourceAssignment> newAssignment =
-        partialRebalance(clusterData, clusterChanges, resourceMap, currentStateOutput);
+        partialRebalance(clusterData, clusterChanges, resourceMap, activeNodes,
+            currentStateOutput);
 
+    // <ResourceName, <State, Priority>>
+    Map<String, Map<String, Integer>> resourceStatePriorityMap = new HashMap<>();
     // Convert the assignments into IdealState for the following state mapping calculation.
-    Map<String, IdealState> finalIdealState = new HashMap<>();
+    Map<String, IdealState> finalIdealStateMap = new HashMap<>();
     for (String resourceName : newAssignment.keySet()) {
-      IdealState newIdeaState;
+      IdealState newIdealState;
       try {
         IdealState currentIdealState = clusterData.getIdealState(resourceName);
         Map<String, Integer> statePriorityMap = clusterData
             .getStateModelDef(currentIdealState.getStateModelDefRef()).getStatePriorityMap();
+        // Keep the priority map for the rebalance overwrite logic later.
+        resourceStatePriorityMap.put(resourceName, statePriorityMap);
         // Create a new IdealState instance contains the new calculated assignment in the preference
         // list.
-        newIdeaState = generateIdealStateWithAssignment(resourceName, currentIdealState,
+        newIdealState = generateIdealStateWithAssignment(resourceName, currentIdealState,
             newAssignment.get(resourceName), statePriorityMap);
       } catch (Exception ex) {
         throw new HelixRebalanceException(
             "Fail to calculate the new IdealState for resource: " + resourceName,
             HelixRebalanceException.Type.INVALID_CLUSTER_STATUS, ex);
       }
-      finalIdealState.put(resourceName, newIdeaState);
+      finalIdealStateMap.put(resourceName, newIdealState);
+    }
+
+    // The additional rebalance overwrite is required since the calculated mapping may contains
+    // some delayed rebalanced assignments.
+    if (!activeNodes.equals(clusterData.getEnabledLiveInstances())) {
+      applyRebalanceOverwrite(finalIdealStateMap, clusterData, resourceMap, clusterChanges,
+          resourceStatePriorityMap,
+          getBaselineAssignment(_assignmentMetadataStore, currentStateOutput,
+              resourceMap.keySet()));
     }
-    return finalIdealState;
+    // Replace the assignment if user-defined preference list is configured.
+    // Note the user-defined list is intentionally applied to the final mapping after calculation.
+    // This is to avoid persisting it into the assignment store, which impacts the long term
+    // assignment evenness and partition movements.
+    finalIdealStateMap.entrySet().stream().forEach(
+        idealStateEntry -> applyUserDefinedPreferenceList(
+            clusterData.getResourceConfig(idealStateEntry.getKey()), idealStateEntry.getValue()));
+
+    return finalIdealStateMap;
   }
 
   // TODO make the Baseline calculation async if complicated algorithm is used for the Baseline
@@ -253,7 +285,8 @@ public class WagedRebalancer {
   private Map<String, ResourceAssignment> partialRebalance(
       ResourceControllerDataProvider clusterData,
       Map<HelixConstants.ChangeType, Set<String>> clusterChanges, Map<String, Resource> resourceMap,
-      final CurrentStateOutput currentStateOutput) throws HelixRebalanceException {
+      Set<String> activeNodes, final CurrentStateOutput currentStateOutput)
+      throws HelixRebalanceException {
     LOG.info("Start calculating the new best possible assignment.");
     Map<String, ResourceAssignment> currentBaseline =
         getBaselineAssignment(_assignmentMetadataStore, currentStateOutput, resourceMap.keySet());
@@ -261,8 +294,8 @@ public class WagedRebalancer {
         getBestPossibleAssignment(_assignmentMetadataStore, currentStateOutput,
             resourceMap.keySet());
     Map<String, ResourceAssignment> newAssignment =
-        calculateAssignment(clusterData, clusterChanges, resourceMap,
-            clusterData.getEnabledLiveInstances(), currentBaseline, currentBestPossibleAssignment);
+        calculateAssignment(clusterData, clusterChanges, resourceMap, activeNodes, currentBaseline,
+            currentBestPossibleAssignment);
 
     if (_assignmentMetadataStore != null) {
       try {
@@ -458,4 +491,88 @@ public class WagedRebalancer {
     }
     return currentStateAssignment;
   }
+
+  /**
+   * Schedule rebalance according to the delayed rebalance logic.
+   * @param clusterData the current cluster data cache
+   * @param delayedActiveNodes the active nodes set that is calculated with the delay time window
+   * @param resourceSet the rebalanced resourceSet
+   */
+  private void delayedRebalanceSchedule(ResourceControllerDataProvider clusterData,
+      Set<String> delayedActiveNodes, Set<String> resourceSet) {
+    if (_manager != null) {
+      // Schedule for the next delayed rebalance in case no cluster change event happens.
+      ClusterConfig clusterConfig = clusterData.getClusterConfig();
+      boolean delayedRebalanceEnabled = DelayedRebalanceUtil.isDelayRebalanceEnabled(clusterConfig);
+      Set<String> offlineOrDisabledInstances = new HashSet<>(delayedActiveNodes);
+      offlineOrDisabledInstances.removeAll(clusterData.getEnabledLiveInstances());
+      for (String resource : resourceSet) {
+        DelayedRebalanceUtil
+            .setRebalanceScheduler(resource, delayedRebalanceEnabled, offlineOrDisabledInstances,
+                clusterData.getInstanceOfflineTimeMap(), clusterData.getLiveInstances().keySet(),
+                clusterData.getInstanceConfigMap(), clusterConfig.getRebalanceDelayTime(),
+                clusterConfig, _manager);
+      }
+    } else {
+      LOG.warn("Skip scheduling a delayed rebalancer since HelixManager is not specified.");
+    }
+  }
+
+  /**
+   * Update the rebalanced ideal states according to the real active nodes.
+   * Since the rebalancing might be done with the delayed logic, the rebalanced ideal states
+   * might include inactive nodes.
+   * This overwrite will adjust the final mapping, so as to ensure the result is completely valid.
+   * @param idealStateMap            the calculated ideal states.
+   * @param clusterData              the cluster data cache.
+   * @param resourceMap              the rebalanaced resource map.
+   * @param clusterChanges           the detected cluster changes that triggeres the rebalance.
+   * @param resourceStatePriorityMap the state priority map for each resource.
+   * @param baseline                 the baseline assignment
+   */
+  private void applyRebalanceOverwrite(Map<String, IdealState> idealStateMap,
+      ResourceControllerDataProvider clusterData, Map<String, Resource> resourceMap,
+      Map<HelixConstants.ChangeType, Set<String>> clusterChanges,
+      Map<String, Map<String, Integer>> resourceStatePriorityMap,
+      Map<String, ResourceAssignment> baseline)
+      throws HelixRebalanceException {
+    Set<String> enabledLiveInstances = clusterData.getEnabledLiveInstances();
+    // Note that the calculation used the baseline as the input only. This is for minimizing unnecessary partition movement.
+    Map<String, ResourceAssignment> activeAssignment =
+        calculateAssignment(clusterData, clusterChanges, resourceMap, enabledLiveInstances,
+            Collections.emptyMap(), baseline);
+    for (String resourceName : idealStateMap.keySet()) {
+      IdealState is = idealStateMap.get(resourceName);
+      if (!activeAssignment.containsKey(resourceName)) {
+        throw new HelixRebalanceException(
+            "Failed to calculate the complete partition assignment with all active nodes. Cannot find the resource assignment for "
+                + resourceName, HelixRebalanceException.Type.FAILED_TO_CALCULATE);
+      }
+      IdealState currentIdealState = clusterData.getIdealState(resourceName);
+      IdealState newActiveIdealState =
+          generateIdealStateWithAssignment(resourceName, currentIdealState,
+              activeAssignment.get(resourceName), resourceStatePriorityMap.get(resourceName));
+
+      int numReplia = currentIdealState.getReplicaCount(enabledLiveInstances.size());
+      int minActiveReplica = DelayedRebalanceUtil.getMinActiveReplica(currentIdealState, numReplia);
+      Map<String, List<String>> finalPreferenceLists = DelayedRebalanceUtil
+          .getFinalDelayedMapping(newActiveIdealState.getPreferenceLists(), is.getPreferenceLists(),
+              enabledLiveInstances, Math.min(minActiveReplica, numReplia));
+
+      is.setPreferenceLists(finalPreferenceLists);
+    }
+  }
+
+  private void applyUserDefinedPreferenceList(ResourceConfig resourceConfig,
+      IdealState idealState) {
+    if (resourceConfig != null) {
+      Map<String, List<String>> userDefinedPreferenceList = resourceConfig.getPreferenceLists();
+      if (!userDefinedPreferenceList.isEmpty()) {
+        LOG.info("Using user defined preference list for partitions.");
+        for (String partition : userDefinedPreferenceList.keySet()) {
+          idealState.setPreferenceList(partition, userDefinedPreferenceList.get(partition));
+        }
+      }
+    }
+  }
 }
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
index 85f0397..f3bca9e 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterVerifiers/StrictMatchExternalViewVerifier.java
@@ -28,6 +28,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import org.apache.helix.ConfigAccessor;
 import org.apache.helix.HelixException;
 import org.apache.helix.PropertyKey;
@@ -250,11 +251,12 @@ public class StrictMatchExternalViewVerifier extends ZkHelixClusterVerifier {
             + "is enabled."));
       }
       for (String partition : idealState.getPartitionSet()) {
-        if (idealState.getPreferenceList(partition) == null || idealState.getPreferenceList(partition).isEmpty()) {
+        if (idealState.getInstanceStateMap(partition) == null || idealState
+            .getInstanceStateMap(partition).isEmpty()) {
           return false;
         }
       }
-      idealPartitionState = computeIdealPartitionState(dataCache, idealState);
+      idealPartitionState = idealState.getRecord().getMapFields();
       break;
     case SEMI_AUTO:
     case USER_DEFINED:
diff --git a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
index b9284b9..e166e13 100644
--- a/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
+++ b/helix-core/src/test/java/org/apache/helix/common/ZkTestBase.java
@@ -353,9 +353,9 @@ public class ZkTestBase {
   }
 
   protected IdealState createResourceWithWagedRebalance(String clusterName, String db,
-      String stateModel, int numPartition, int replica, int minActiveReplica, long delay) {
+      String stateModel, int numPartition, int replica, int minActiveReplica) {
     return createResource(clusterName, db, stateModel, numPartition, replica, minActiveReplica,
-        delay, WagedRebalancer.class.getName(), null);
+        -1, WagedRebalancer.class.getName(), null);
   }
 
   private IdealState createResource(String clusterName, String db, String stateModel,
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
index e7368be..96b6523 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/TestWagedRebalancer.java
@@ -26,10 +26,10 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
-
 import org.apache.helix.HelixConstants;
 import org.apache.helix.HelixRebalanceException;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.waged.constraints.MockRebalanceAlgorithm;
 import org.apache.helix.controller.rebalancer.waged.model.AbstractTestClusterModel;
@@ -112,7 +112,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   @Test
   public void testRebalance() throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -132,9 +133,11 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   }
 
   @Test(dependsOnMethods = "testRebalance")
-  public void testPartialRebalance() throws IOException, HelixRebalanceException {
+  public void testPartialRebalance()
+      throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -159,7 +162,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   @Test(dependsOnMethods = "testRebalance")
   public void testRebalanceWithCurrentState() throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     // Generate the input for the rebalancer.
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
@@ -216,9 +220,11 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
   }
 
   @Test(dependsOnMethods = "testRebalance", expectedExceptions = HelixRebalanceException.class, expectedExceptionsMessageRegExp = "Input contains invalid resource\\(s\\) that cannot be rebalanced by the WAGED rebalancer. \\[Resource1\\] Failure Type: INVALID_INPUT")
-  public void testNonCompatibleConfiguration() throws IOException, HelixRebalanceException {
+  public void testNonCompatibleConfiguration()
+      throws IOException, HelixRebalanceException {
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     String nonCompatibleResourceName = _resourceNames.get(0);
@@ -237,9 +243,11 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
 
   // TODO test with invalid capacity configuration which will fail the cluster model constructing.
   @Test(dependsOnMethods = "testRebalance")
-  public void testInvalidClusterStatus() throws IOException {
+  public void testInvalidClusterStatus()
+      throws IOException {
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     String invalidResource = _resourceNames.get(0);
@@ -264,7 +272,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     AssignmentMetadataStore metadataStore = Mockito.mock(AssignmentMetadataStore.class);
     when(metadataStore.getBaseline())
         .thenThrow(new RuntimeException("Mock Error. Metadata store fails."));
-    WagedRebalancer rebalancer = new WagedRebalancer(metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     // The input resource Map shall contain all the valid resources.
@@ -288,7 +297,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
         HelixRebalanceException.Type.FAILED_TO_CALCULATE));
 
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, badAlgorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, badAlgorithm, new DelayedAutoRebalancer());
 
     ResourceControllerDataProvider clusterData = setupClusterDataCache();
     Map<String, Resource> resourceMap = clusterData.getIdealStates().keySet().stream().collect(
@@ -312,7 +322,8 @@ public class TestWagedRebalancer extends AbstractTestClusterModel {
     // won't propagate any existing assignment from the cluster model.
 
     _metadataStore.clearMetadataStore();
-    WagedRebalancer rebalancer = new WagedRebalancer(_metadataStore, _algorithm);
+    WagedRebalancer rebalancer =
+        new WagedRebalancer(_metadataStore, _algorithm, new DelayedAutoRebalancer());
 
     // 1. rebalance with baseline calculation done
     // Generate the input for the rebalancer.
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
index 0105a51..7d4965e 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalance.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
-import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
@@ -44,19 +43,22 @@ import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public class TestDelayedAutoRebalance extends ZkTestBase {
-  final int NUM_NODE = 5;
+  static final int NUM_NODE = 5;
   protected static final int START_PORT = 12918;
-  protected static final int _PARTITIONS = 5;
+  protected static final int PARTITIONS = 5;
+  // TODO: remove this wait time once we have a better way to determine if the rebalance has been
+  // TODO: done as a reaction of the test operations.
+  protected static final int DEFAULT_REBALANCE_PROCESSING_WAIT_TIME = 1000;
 
   protected final String CLASS_NAME = getShortClassName();
   protected final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
   protected ClusterControllerManager _controller;
 
-  List<MockParticipantManager> _participants = new ArrayList<>();
-  int _replica = 3;
-  int _minActiveReplica = _replica - 1;
-  ZkHelixClusterVerifier _clusterVerifier;
-  List<String> _testDBs = new ArrayList<String>();
+  protected List<MockParticipantManager> _participants = new ArrayList<>();
+  protected int _replica = 3;
+  protected int _minActiveReplica = _replica - 1;
+  protected ZkHelixClusterVerifier _clusterVerifier;
+  protected List<String> _testDBs = new ArrayList<>();
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -80,8 +82,7 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    _clusterVerifier = getClusterVerifier();
 
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
   }
@@ -123,7 +124,8 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
 
     // bring down another node, the minimal active replica for each partition should be maintained.
     _participants.get(3).syncStop();
-    Thread.sleep(500);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
     for (String db : _testDBs) {
       ExternalView ev =
           _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
@@ -141,10 +143,11 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
     long delay = 4000;
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(delay);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, delay);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
     validateDelayedMovements(externalViewsBefore);
 
-    Thread.sleep(delay + 200);
+    Thread.sleep(delay + DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
     // after delay time, it should maintain required number of replicas.
     for (String db : _testDBs) {
@@ -157,7 +160,8 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
 
   @Test (dependsOnMethods = {"testMinimalActiveReplicaMaintain"})
   public void testDisableDelayRebalanceInResource() throws Exception {
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(1000000);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, 1000000);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
     validateDelayedMovements(externalViewsBefore);
 
     // disable delay rebalance for one db, partition should be moved immediately
@@ -166,7 +170,7 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
         CLUSTER_NAME, testDb);
     idealState.setDelayRebalanceEnabled(false);
     _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
-
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     // once delay rebalance is disabled, it should maintain required number of replicas for that db.
@@ -190,13 +194,13 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
   @Test (dependsOnMethods = {"testDisableDelayRebalanceInResource"})
   public void testDisableDelayRebalanceInCluster() throws Exception {
     enableDelayRebalanceInCluster(_gZkClient, CLUSTER_NAME, true);
-
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(1000000);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, 1000000);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
     validateDelayedMovements(externalViewsBefore);
 
     // disable delay rebalance for the entire cluster.
     enableDelayRebalanceInCluster(_gZkClient, CLUSTER_NAME, false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
     for (String db : _testDBs) {
       ExternalView ev =
@@ -210,13 +214,14 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
 
   @Test (dependsOnMethods = {"testDisableDelayRebalanceInCluster"})
   public void testDisableDelayRebalanceInInstance() throws Exception {
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(1000000);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, 1000000);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
     validateDelayedMovements(externalViewsBefore);
 
     String disabledInstanceName = _participants.get(0).getInstanceName();
     enableDelayRebalanceInInstance(_gZkClient, CLUSTER_NAME, disabledInstanceName, false);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
-
     for (String db : _testDBs) {
       IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       Map<String, List<String>> preferenceLists = is.getPreferenceLists();
@@ -234,7 +239,7 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
       _gSetupTool.dropResourceFromCluster(CLUSTER_NAME, db);
     }
     _testDBs.clear();
-    Thread.sleep(50);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
   }
 
   @BeforeMethod
@@ -249,17 +254,21 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
     }
   }
 
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    return new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+  }
+
   // create test DBs, wait it converged and return externalviews
   protected Map<String, ExternalView> createTestDBs(long delayTime) throws InterruptedException {
     Map<String, ExternalView> externalViews = new HashMap<String, ExternalView>();
     int i = 0;
     for (String stateModel : TestStateModels) {
       String db = "Test-DB-" + i++;
-      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
+      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
           _minActiveReplica, delayTime, CrushRebalanceStrategy.class.getName());
       _testDBs.add(db);
     }
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
     for (String db : _testDBs) {
       ExternalView ev =
@@ -302,7 +311,7 @@ public class TestDelayedAutoRebalance extends ZkTestBase {
   private void validateDelayedMovements(Map<String, ExternalView> externalViewsBefore)
       throws InterruptedException {
     _participants.get(0).syncStop();
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
index 746bdf3..145148f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithDisabledInstance.java
@@ -21,7 +21,6 @@ package org.apache.helix.integration.rebalancer.DelayedAutoRebalancer;
 
 import java.util.Map;
 import org.apache.helix.ConfigAccessor;
-import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
 import org.apache.helix.integration.manager.MockParticipantManager;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.IdealState;
@@ -56,7 +55,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     String instance = _participants.get(0).getInstanceName();
     enableInstance(instance, false);
 
-    Thread.sleep(300);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -79,7 +78,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     String instance = _participants.get(0).getInstanceName();
     enableInstance(instance, false);
 
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -106,7 +105,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable one node, no partition should be moved.
     enableInstance(_participants.get(0).getInstanceName(), false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -120,7 +119,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable another node, the minimal active replica for each partition should be maintained.
     enableInstance(_participants.get(3).getInstanceName(), false);
-    Thread.sleep(1000);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -143,7 +142,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable one node, no partition should be moved.
     enableInstance(_participants.get(0).getInstanceName(), false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -157,7 +156,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // bring down another node, the minimal active replica for each partition should be maintained.
     _participants.get(3).syncStop();
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -178,11 +177,12 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
     long delay = 10000;
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(delay);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, delay);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
 
     // disable one node, no partition should be moved.
     enableInstance(_participants.get(0).getInstanceName(), false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
     for (String db : _testDBs) {
       ExternalView ev =
@@ -193,7 +193,8 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
           _participants.get(0).getInstanceName(), true);
     }
 
-    Thread.sleep(delay + 500);
+    Thread.sleep(delay + DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
     // after delay time, it should maintain required number of replicas.
     for (String db : _testDBs) {
       ExternalView ev =
@@ -210,7 +211,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable one node, no partition should be moved.
     enableInstance(_participants.get(0).getInstanceName(), false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -228,7 +229,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
         CLUSTER_NAME, testDb);
     idealState.setDelayRebalanceEnabled(false);
     _gSetupTool.getClusterManagementTool().setResourceIdealState(CLUSTER_NAME, testDb, idealState);
-    Thread.sleep(2000);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     // once delay rebalance is disabled, it should maintain required number of replicas for that db.
@@ -253,12 +254,12 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
   @Override
   public void testDisableDelayRebalanceInCluster() throws Exception {
     enableDelayRebalanceInCluster(_gZkClient, CLUSTER_NAME, true);
-
-    Map<String, ExternalView> externalViewsBefore = createTestDBs(1000000);
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, 1000000);
+    Map<String, ExternalView> externalViewsBefore = createTestDBs(-1);
 
     // disable one node, no partition should be moved.
     enableInstance(_participants.get(0).getInstanceName(), false);
-    Thread.sleep(100);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
 
     for (String db : _testDBs) {
@@ -272,7 +273,7 @@ public class TestDelayedAutoRebalanceWithDisabledInstance extends TestDelayedAut
 
     // disable delay rebalance for the entire cluster.
     enableDelayRebalanceInCluster(_gZkClient, CLUSTER_NAME, false);
-    Thread.sleep(2000);
+    Thread.sleep(DEFAULT_REBALANCE_PROCESSING_WAIT_TIME);
     Assert.assertTrue(_clusterVerifier.verifyByPolling());
     for (String db : _testDBs) {
       ExternalView ev =
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
index f768684..f85f07f 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/DelayedAutoRebalancer/TestDelayedAutoRebalanceWithRackaware.java
@@ -29,7 +29,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 public class TestDelayedAutoRebalanceWithRackaware extends TestDelayedAutoRebalance {
-  final int NUM_NODE = 9;
+  static final int NUM_NODE = 9;
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -58,8 +58,7 @@ public class TestDelayedAutoRebalanceWithRackaware extends TestDelayedAutoRebala
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    _clusterVerifier = getClusterVerifier();
   }
 
   @Override
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
index 76560e9..33dab8d 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestMixedModeAutoRebalance.java
@@ -25,16 +25,15 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import org.apache.helix.ConfigAccessor;
-import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.NotificationContext;
+import org.apache.helix.TestHelper;
 import org.apache.helix.common.ZkTestBase;
 import org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy;
 import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy;
-import org.apache.helix.controller.rebalancer.util.RebalanceScheduler;
 import org.apache.helix.integration.manager.ClusterControllerManager;
 import org.apache.helix.integration.manager.MockParticipantManager;
-import org.apache.helix.manager.zk.ZKHelixDataAccessor;
 import org.apache.helix.mock.participant.MockMSModelFactory;
 import org.apache.helix.mock.participant.MockMSStateModel;
 import org.apache.helix.mock.participant.MockTransition;
@@ -49,6 +48,7 @@ import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -60,13 +60,13 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
 
   private final String CLASS_NAME = getShortClassName();
   private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
-  private ClusterControllerManager _controller;
+  protected static final String DB_NAME = "Test-DB";
 
+  private ClusterControllerManager _controller;
   private List<MockParticipantManager> _participants = new ArrayList<>();
   private int _replica = 3;
   private ZkHelixClusterVerifier _clusterVerifier;
   private ConfigAccessor _configAccessor;
-  private HelixDataAccessor _dataAccessor;
 
   @BeforeClass
   public void beforeClass() throws Exception {
@@ -90,13 +90,11 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
 
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    _clusterVerifier = getClusterVerifier();
 
     enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
 
     _configAccessor = new ConfigAccessor(_gZkClient);
-    _dataAccessor = new ZKHelixDataAccessor(CLUSTER_NAME, _baseAccessor);
   }
 
   @DataProvider(name = "stateModels")
@@ -112,19 +110,28 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
     };
   }
 
-  @Test(dataProvider = "stateModels")
-  public void testUserDefinedPreferenceListsInFullAuto(
-      String stateModel, boolean delayEnabled, String rebalanceStrateyName) throws Exception {
-    String db = "Test-DB-" + stateModel;
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    return new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+  }
+
+  protected void createResource(String stateModel, int numPartition, int replica,
+      boolean delayEnabled, String rebalanceStrategy) {
     if (delayEnabled) {
-      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
-          _replica - 1, 200, rebalanceStrateyName);
+      createResourceWithDelayedRebalance(CLUSTER_NAME, DB_NAME, stateModel, numPartition, replica,
+          replica - 1, 200, rebalanceStrategy);
     } else {
-      createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, _PARTITIONS, _replica,
-          _replica, 0, rebalanceStrateyName);
+      createResourceWithDelayedRebalance(CLUSTER_NAME, DB_NAME, stateModel, numPartition, replica,
+          replica, 0, rebalanceStrategy);
     }
+  }
+
+  @Test(dataProvider = "stateModels")
+  public void testUserDefinedPreferenceListsInFullAuto(String stateModel, boolean delayEnabled,
+      String rebalanceStrateyName) throws Exception {
+    createResource(stateModel, _PARTITIONS, _replica, delayEnabled,
+        rebalanceStrateyName);
     IdealState idealState =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB_NAME);
     Map<String, List<String>> userDefinedPreferenceLists = idealState.getPreferenceLists();
     List<String> userDefinedPartitions = new ArrayList<>();
     for (String partition : userDefinedPreferenceLists.keySet()) {
@@ -138,33 +145,34 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
     }
 
     ResourceConfig resourceConfig =
-        new ResourceConfig.Builder(db).setPreferenceLists(userDefinedPreferenceLists).build();
-    _configAccessor.setResourceConfig(CLUSTER_NAME, db, resourceConfig);
+        new ResourceConfig.Builder(DB_NAME).setPreferenceLists(userDefinedPreferenceLists).build();
+    _configAccessor.setResourceConfig(CLUSTER_NAME, DB_NAME, resourceConfig);
 
-    Assert.assertTrue(_clusterVerifier.verify(1000));
-    verifyUserDefinedPreferenceLists(db, userDefinedPreferenceLists, userDefinedPartitions);
+    Assert.assertTrue(_clusterVerifier.verify(3000));
+    verifyUserDefinedPreferenceLists(DB_NAME, userDefinedPreferenceLists, userDefinedPartitions);
 
     while (userDefinedPartitions.size() > 0) {
-      IdealState originIS = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+      IdealState originIS = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME,
+          DB_NAME);
       Set<String> nonUserDefinedPartitions = new HashSet<>(originIS.getPartitionSet());
       nonUserDefinedPartitions.removeAll(userDefinedPartitions);
 
-      removePartitionFromUserDefinedList(db, userDefinedPartitions);
-      Assert.assertTrue(_clusterVerifier.verify(1000));
-      verifyUserDefinedPreferenceLists(db, userDefinedPreferenceLists, userDefinedPartitions);
-      verifyNonUserDefinedAssignment(db, originIS, nonUserDefinedPartitions);
+      removePartitionFromUserDefinedList(DB_NAME, userDefinedPartitions);
+      // TODO: Remove wait once we enable the BestPossibleExternalViewVerifier for the WAGED rebalancer.
+      Thread.sleep(1000);
+      Assert.assertTrue(_clusterVerifier.verify(3000));
+      verifyUserDefinedPreferenceLists(DB_NAME, userDefinedPreferenceLists, userDefinedPartitions);
+      verifyNonUserDefinedAssignment(DB_NAME, originIS, nonUserDefinedPartitions);
     }
   }
 
   @Test
   public void testUserDefinedPreferenceListsInFullAutoWithErrors() throws Exception {
-    String db = "Test-DB-1";
-    createResourceWithDelayedRebalance(CLUSTER_NAME, db,
-        BuiltInStateModelDefinitions.MasterSlave.name(), 5, _replica, _replica, 0,
-        CrushRebalanceStrategy.class.getName());
+    createResource(BuiltInStateModelDefinitions.MasterSlave.name(), 5, _replica,
+        false, CrushRebalanceStrategy.class.getName());
 
     IdealState idealState =
-        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+        _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, DB_NAME);
     Map<String, List<String>> userDefinedPreferenceLists = idealState.getPreferenceLists();
 
     List<String> newNodes = new ArrayList<>();
@@ -187,13 +195,28 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
     }
 
     ResourceConfig resourceConfig =
-        new ResourceConfig.Builder(db).setPreferenceLists(userDefinedPreferenceLists).build();
-    _configAccessor.setResourceConfig(CLUSTER_NAME, db, resourceConfig);
+        new ResourceConfig.Builder(DB_NAME).setPreferenceLists(userDefinedPreferenceLists).build();
+    _configAccessor.setResourceConfig(CLUSTER_NAME, DB_NAME, resourceConfig);
+
+    TestHelper.verify(() -> {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, DB_NAME);
+      if (ev != null) {
+        for (String partition : ev.getPartitionSet()) {
+          Map<String, String> stateMap = ev.getStateMap(partition);
+          if (stateMap.values().contains("ERROR")) {
+            return true;
+          }
+        }
+      }
+      return false;
+    }, 2000);
+    Assert.assertTrue(_clusterVerifier.verify(3000));
 
-    Thread.sleep(1000);
     ExternalView ev =
-        _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
-    IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
+        _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, DB_NAME);
+    IdealState is = _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME,
+        DB_NAME);
     validateMinActiveAndTopStateReplica(is, ev, _replica, NUM_NODE);
   }
 
@@ -238,6 +261,12 @@ public class TestMixedModeAutoRebalance extends ZkTestBase {
     _configAccessor.setResourceConfig(CLUSTER_NAME, db, resourceConfig);
   }
 
+  @AfterMethod
+  public void afterMethod() {
+    _gSetupTool.getClusterManagementTool().dropResource(CLUSTER_NAME, DB_NAME);
+    getClusterVerifier().verify(5000);
+  }
+
   @AfterClass
   public void afterClass() throws Exception {
     /**
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
index ab4a263..7090cbf 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/TestZeroReplicaAvoidance.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.helix.ExternalViewChangeListener;
 import org.apache.helix.HelixManager;
 import org.apache.helix.HelixManagerFactory;
@@ -39,10 +40,11 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Message;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
 import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
 import org.testng.Assert;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 public class TestZeroReplicaAvoidance extends ZkTestBase
@@ -53,16 +55,13 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
   private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
 
   private List<MockParticipantManager> _participants = new ArrayList<>();
-  private ZkHelixClusterVerifier _clusterVerifier;
   private boolean _testSuccess = true;
   private boolean _startListen = false;
 
   private ClusterControllerManager _controller;
 
-  @BeforeClass
-  public void beforeClass() throws Exception {
-    System.out.println("START " + CLASS_NAME + " at " + new Date(System.currentTimeMillis()));
-
+  @BeforeMethod
+  public void beforeMethod() {
     _gSetupTool.addCluster(CLUSTER_NAME, true);
     for (int i = 0; i < NUM_NODE; i++) {
       String storageNodeName = PARTICIPANT_PREFIX + "_" + (START_PORT + i);
@@ -77,13 +76,11 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
     String controllerName = CONTROLLER_PREFIX + "_0";
     _controller = new ClusterControllerManager(ZK_ADDR, CLUSTER_NAME, controllerName);
     _controller.syncStart();
-
-    _clusterVerifier =
-        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
   }
 
-  @AfterClass
-  public void afterClass() {
+  @AfterMethod
+  public void afterMethod() {
+    _startListen = false;
     if (_controller != null && _controller.isConnected()) {
       _controller.syncStop();
     }
@@ -92,6 +89,7 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
         participant.syncStop();
       }
     }
+    _participants.clear();
     deleteCluster(CLUSTER_NAME);
   }
 
@@ -102,7 +100,8 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
   };
 
   @Test
-  public void test() throws Exception {
+  public void testDelayedRebalancer() throws Exception {
+    System.out.println("START testDelayedRebalancer at " + new Date(System.currentTimeMillis()));
     HelixManager manager =
         HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, null, InstanceType.SPECTATOR, ZK_ADDR);
     manager.connect();
@@ -123,7 +122,51 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
       createResourceWithDelayedRebalance(CLUSTER_NAME, db, stateModel, partition, replica, replica,
           0);
     }
-    Assert.assertTrue(_clusterVerifier.verifyByPolling(50000L, 100L));
+    ZkHelixClusterVerifier clusterVerifier =
+        new BestPossibleExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    Assert.assertTrue(clusterVerifier.verifyByPolling(50000L, 100L));
+
+    _startListen = true;
+    DelayedTransition.setDelay(5);
+
+    // add the other half of nodes.
+    for (; i < NUM_NODE; i++) {
+      _participants.get(i).syncStart();
+    }
+    Assert.assertTrue(clusterVerifier.verify(70000L));
+    Assert.assertTrue(_testSuccess);
+
+    if (manager.isConnected()) {
+      manager.disconnect();
+    }
+    System.out.println("END testDelayedRebalancer at " + new Date(System.currentTimeMillis()));
+  }
+
+  @Test
+  public void testWagedRebalancer() throws Exception {
+    System.out.println("START testWagedRebalancer at " + new Date(System.currentTimeMillis()));
+    HelixManager manager =
+        HelixManagerFactory.getZKHelixManager(CLUSTER_NAME, null, InstanceType.SPECTATOR, ZK_ADDR);
+    manager.connect();
+    manager.addExternalViewChangeListener(this);
+    manager.addIdealStateChangeListener(this);
+    enablePersistBestPossibleAssignment(_gZkClient, CLUSTER_NAME, true);
+
+    // Start half number of nodes.
+    int i = 0;
+    for (; i < NUM_NODE / 2; i++) {
+      _participants.get(i).syncStart();
+    }
+
+    int replica = 3;
+    int partition = 30;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + stateModel;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, partition, replica, replica);
+    }
+    ZkHelixClusterVerifier clusterVerifier =
+        new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR).build();
+    Assert.assertTrue(clusterVerifier.verifyByPolling(50000L, 100L));
 
     _startListen = true;
     DelayedTransition.setDelay(5);
@@ -132,12 +175,13 @@ public class TestZeroReplicaAvoidance extends ZkTestBase
     for (; i < NUM_NODE; i++) {
       _participants.get(i).syncStart();
     }
-    Assert.assertTrue(_clusterVerifier.verify(70000L));
+    Assert.assertTrue(clusterVerifier.verify(70000L));
     Assert.assertTrue(_testSuccess);
 
     if (manager.isConnected()) {
       manager.disconnect();
     }
+    System.out.println("END testWagedRebalancer at " + new Date(System.currentTimeMillis()));
   }
 
   /**
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalance.java
new file mode 100644
index 0000000..8587f40
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalance.java
@@ -0,0 +1,102 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.integration.rebalancer.DelayedAutoRebalancer.TestDelayedAutoRebalance;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ * Inherit TestDelayedAutoRebalance to ensure the test logic is the same.
+ */
+public class TestDelayedWagedRebalance extends TestDelayedAutoRebalance {
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    Set<String> dbNames = new HashSet<>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      dbNames.add("Test-DB-" + i++);
+    }
+    return new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setResources(dbNames)
+        .setZkAddr(ZK_ADDR).build();
+  }
+
+  // create test DBs, wait it converged and return externalviews
+  protected Map<String, ExternalView> createTestDBs(long delayTime) throws InterruptedException {
+    Map<String, ExternalView> externalViews = new HashMap<String, ExternalView>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _minActiveReplica);
+      _testDBs.add(db);
+    }
+    Thread.sleep(100);
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+    for (String db : _testDBs) {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      externalViews.put(db, ev);
+    }
+    return externalViews;
+  }
+
+  @Test
+  public void testDelayedPartitionMovement() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test
+  public void testDisableDelayRebalanceInResource() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovement" })
+  public void testDelayedPartitionMovementWithClusterConfigedDelay() throws Exception {
+    super.testDelayedPartitionMovementWithClusterConfigedDelay();
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovementWithClusterConfigedDelay" })
+  public void testMinimalActiveReplicaMaintain() throws Exception {
+    super.testMinimalActiveReplicaMaintain();
+  }
+
+  @Test(dependsOnMethods = { "testMinimalActiveReplicaMaintain" })
+  public void testPartitionMovementAfterDelayTime() throws Exception {
+    super.testPartitionMovementAfterDelayTime();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInResource" })
+  public void testDisableDelayRebalanceInCluster() throws Exception {
+    super.testDisableDelayRebalanceInCluster();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInCluster" })
+  public void testDisableDelayRebalanceInInstance() throws Exception {
+    super.testDisableDelayRebalanceInInstance();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithDisabledInstance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithDisabledInstance.java
new file mode 100644
index 0000000..fab254c
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithDisabledInstance.java
@@ -0,0 +1,103 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.integration.rebalancer.DelayedAutoRebalancer.TestDelayedAutoRebalanceWithDisabledInstance;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ * Inherit TestDelayedAutoRebalanceWithDisabledInstance to ensure the test logic is the same.
+ */
+public class TestDelayedWagedRebalanceWithDisabledInstance
+    extends TestDelayedAutoRebalanceWithDisabledInstance {
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    Set<String> dbNames = new HashSet<>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      dbNames.add("Test-DB-" + i++);
+    }
+    return new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setResources(dbNames)
+        .setZkAddr(ZK_ADDR).build();
+  }
+
+  // create test DBs, wait it converged and return externalviews
+  protected Map<String, ExternalView> createTestDBs(long delayTime) throws InterruptedException {
+    Map<String, ExternalView> externalViews = new HashMap<String, ExternalView>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _minActiveReplica);
+      _testDBs.add(db);
+    }
+    Thread.sleep(100);
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+    for (String db : _testDBs) {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      externalViews.put(db, ev);
+    }
+    return externalViews;
+  }
+
+  @Test
+  public void testDelayedPartitionMovement() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test
+  public void testDisableDelayRebalanceInResource() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovement" })
+  public void testDelayedPartitionMovementWithClusterConfigedDelay() throws Exception {
+    super.testDelayedPartitionMovementWithClusterConfigedDelay();
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovementWithClusterConfigedDelay" })
+  public void testMinimalActiveReplicaMaintain() throws Exception {
+    super.testMinimalActiveReplicaMaintain();
+  }
+
+  @Test(dependsOnMethods = { "testMinimalActiveReplicaMaintain" })
+  public void testPartitionMovementAfterDelayTime() throws Exception {
+    super.testPartitionMovementAfterDelayTime();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInResource" })
+  public void testDisableDelayRebalanceInCluster() throws Exception {
+    super.testDisableDelayRebalanceInCluster();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInCluster" })
+  public void testDisableDelayRebalanceInInstance() throws Exception {
+    super.testDisableDelayRebalanceInInstance();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithRackaware.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithRackaware.java
new file mode 100644
index 0000000..4791e6e
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestDelayedWagedRebalanceWithRackaware.java
@@ -0,0 +1,102 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.integration.rebalancer.DelayedAutoRebalancer.TestDelayedAutoRebalanceWithRackaware;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ * Inherit TestDelayedAutoRebalanceWithRackaware to ensure the test logic is the same.
+ */
+public class TestDelayedWagedRebalanceWithRackaware extends TestDelayedAutoRebalanceWithRackaware {
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    Set<String> dbNames = new HashSet<>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      dbNames.add("Test-DB-" + i++);
+    }
+    return new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setResources(dbNames)
+        .setZkAddr(ZK_ADDR).build();
+  }
+
+  // create test DBs, wait it converged and return externalviews
+  protected Map<String, ExternalView> createTestDBs(long delayTime) throws InterruptedException {
+    Map<String, ExternalView> externalViews = new HashMap<String, ExternalView>();
+    int i = 0;
+    for (String stateModel : TestStateModels) {
+      String db = "Test-DB-" + i++;
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _minActiveReplica);
+      _testDBs.add(db);
+    }
+    Thread.sleep(100);
+    Assert.assertTrue(_clusterVerifier.verifyByPolling());
+    for (String db : _testDBs) {
+      ExternalView ev =
+          _gSetupTool.getClusterManagementTool().getResourceExternalView(CLUSTER_NAME, db);
+      externalViews.put(db, ev);
+    }
+    return externalViews;
+  }
+
+  @Test
+  public void testDelayedPartitionMovement() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test
+  public void testDisableDelayRebalanceInResource() {
+    // Waged Rebalancer takes cluster level delay config only. Skip this test.
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovement" })
+  public void testDelayedPartitionMovementWithClusterConfigedDelay() throws Exception {
+    super.testDelayedPartitionMovementWithClusterConfigedDelay();
+  }
+
+  @Test(dependsOnMethods = { "testDelayedPartitionMovementWithClusterConfigedDelay" })
+  public void testMinimalActiveReplicaMaintain() throws Exception {
+    super.testMinimalActiveReplicaMaintain();
+  }
+
+  @Test(dependsOnMethods = { "testMinimalActiveReplicaMaintain" })
+  public void testPartitionMovementAfterDelayTime() throws Exception {
+    super.testPartitionMovementAfterDelayTime();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInResource" })
+  public void testDisableDelayRebalanceInCluster() throws Exception {
+    super.testDisableDelayRebalanceInCluster();
+  }
+
+  @Test(dependsOnMethods = { "testDisableDelayRebalanceInCluster" })
+  public void testDisableDelayRebalanceInInstance() throws Exception {
+    super.testDisableDelayRebalanceInInstance();
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestMixedModeWagedRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestMixedModeWagedRebalance.java
new file mode 100644
index 0000000..7087dfc
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestMixedModeWagedRebalance.java
@@ -0,0 +1,66 @@
+package org.apache.helix.integration.rebalancer.WagedRebalancer;
+
+/*
+ * 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.Collections;
+import org.apache.helix.integration.rebalancer.TestMixedModeAutoRebalance;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.apache.helix.tools.ClusterVerifiers.ZkHelixClusterVerifier;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.DataProvider;
+
+public class TestMixedModeWagedRebalance extends TestMixedModeAutoRebalance {
+  private final String CLASS_NAME = getShortClassName();
+  private final String CLUSTER_NAME = CLUSTER_PREFIX + "_" + CLASS_NAME;
+
+  @DataProvider(name = "stateModels")
+  public static Object[][] stateModels() {
+    return new Object[][] { { BuiltInStateModelDefinitions.MasterSlave.name(), true, null },
+        { BuiltInStateModelDefinitions.OnlineOffline.name(), true, null },
+        { BuiltInStateModelDefinitions.LeaderStandby.name(), true, null },
+        { BuiltInStateModelDefinitions.MasterSlave.name(), false, null },
+        { BuiltInStateModelDefinitions.OnlineOffline.name(), false, null },
+        { BuiltInStateModelDefinitions.LeaderStandby.name(), false, null }
+    };
+  }
+
+  protected ZkHelixClusterVerifier getClusterVerifier() {
+    return new StrictMatchExternalViewVerifier.Builder(CLUSTER_NAME).setZkAddr(ZK_ADDR)
+        .setResources(Collections.singleton(DB_NAME)).build();
+  }
+
+  protected void createResource(String stateModel, int numPartition,
+      int replica, boolean delayEnabled, String rebalanceStrategy) {
+    if (delayEnabled) {
+      setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, 200);
+      createResourceWithWagedRebalance(CLUSTER_NAME, DB_NAME, stateModel, numPartition, replica,
+          replica - 1);
+    } else {
+      createResourceWithWagedRebalance(CLUSTER_NAME, DB_NAME, stateModel, numPartition, replica, replica);
+    }
+  }
+
+  @AfterMethod
+  public void afterMethod() {
+    super.afterMethod();
+    setDelayTimeInCluster(_gZkClient, CLUSTER_NAME, -1);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java
index fb5375c..37c1229 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalance.java
@@ -109,8 +109,7 @@ public class TestWagedRebalance extends ZkTestBase {
     int i = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + i++;
-      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
-          -1);
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -123,7 +122,7 @@ public class TestWagedRebalance extends ZkTestBase {
     for (String stateModel : _testModels) {
       String moreDB = "More-Test-DB-" + i++;
       createResourceWithWagedRebalance(CLUSTER_NAME, moreDB, stateModel, PARTITIONS, _replica,
-          _replica, -1);
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, moreDB, _replica);
       _allDBs.add(moreDB);
 
@@ -151,7 +150,7 @@ public class TestWagedRebalance extends ZkTestBase {
     for (String tag : tags) {
       String db = "Test-DB-" + i++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db,
-          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica);
       IdealState is =
           _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       is.setInstanceGroupTag(tag);
@@ -167,7 +166,7 @@ public class TestWagedRebalance extends ZkTestBase {
   public void testChangeIdealState() throws InterruptedException {
     String dbName = "Test-DB";
     createResourceWithWagedRebalance(CLUSTER_NAME, dbName,
-        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica);
     _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
     _allDBs.add(dbName);
     Thread.sleep(300);
@@ -201,7 +200,7 @@ public class TestWagedRebalance extends ZkTestBase {
   public void testDisableInstance() throws InterruptedException {
     String dbName = "Test-DB";
     createResourceWithWagedRebalance(CLUSTER_NAME, dbName,
-        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+        BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica);
     _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, dbName, _replica);
     _allDBs.add(dbName);
     Thread.sleep(300);
@@ -256,8 +255,8 @@ public class TestWagedRebalance extends ZkTestBase {
     int j = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + j++;
-      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
-          -1);
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -295,8 +294,8 @@ public class TestWagedRebalance extends ZkTestBase {
     int j = 0;
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + j++;
-      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica, _replica,
-          -1);
+      createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -334,7 +333,7 @@ public class TestWagedRebalance extends ZkTestBase {
             IdealState.RebalanceMode.FULL_AUTO + "", CrushEdRebalanceStrategy.class.getName());
       } else {
         createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-            _replica, -1);
+            _replica);
       }
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
@@ -357,7 +356,7 @@ public class TestWagedRebalance extends ZkTestBase {
       for (String stateModel : _testModels) {
         String db = "Test-DB-" + i++;
         createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-            _replica, -1);
+            _replica);
         if (i == 1) {
           // The limited resource has additional limitation, so even the other resources can be assigned
           // later, this resource will still be blocked by the max partition limitation.
diff --git a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java
index 0b020db..84c6ac4 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/rebalancer/WagedRebalancer/TestWagedRebalanceFaultZone.java
@@ -112,7 +112,7 @@ public class TestWagedRebalanceFaultZone extends ZkTestBase {
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + i++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-          _replica, -1);
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -128,7 +128,7 @@ public class TestWagedRebalanceFaultZone extends ZkTestBase {
     for (String tag : tags) {
       String db = "Test-DB-" + i++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db,
-          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica, -1);
+          BuiltInStateModelDefinitions.MasterSlave.name(), PARTITIONS, _replica, _replica);
       IdealState is =
           _gSetupTool.getClusterManagementTool().getResourceIdealState(CLUSTER_NAME, db);
       is.setInstanceGroupTag(tag);
@@ -156,7 +156,7 @@ public class TestWagedRebalanceFaultZone extends ZkTestBase {
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + j++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-          _replica, -1);
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -198,7 +198,7 @@ public class TestWagedRebalanceFaultZone extends ZkTestBase {
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + j++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-          _replica, -1);
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
@@ -230,7 +230,7 @@ public class TestWagedRebalanceFaultZone extends ZkTestBase {
     for (String stateModel : _testModels) {
       String db = "Test-DB-" + i++;
       createResourceWithWagedRebalance(CLUSTER_NAME, db, stateModel, PARTITIONS, _replica,
-          _replica, -1);
+          _replica);
       _gSetupTool.rebalanceStorageCluster(CLUSTER_NAME, db, _replica);
       _allDBs.add(db);
     }
diff --git a/helix-core/src/test/java/org/apache/helix/tools/TestClusterVerifier.java b/helix-core/src/test/java/org/apache/helix/tools/TestClusterVerifier.java
index ca6b6b6..e1ecdee 100644
--- a/helix-core/src/test/java/org/apache/helix/tools/TestClusterVerifier.java
+++ b/helix-core/src/test/java/org/apache/helix/tools/TestClusterVerifier.java
@@ -125,19 +125,42 @@ public class TestClusterVerifier extends ZkUnitTestBase {
         new BestPossibleExternalViewVerifier.Builder(_clusterName).setZkClient(_gZkClient).build();
     Assert.assertTrue(bestPossibleVerifier.verify(10000));
 
+    // Disable partition for 1 instance, then Full-Auto ExternalView should not match IdealState.
+    _admin.enablePartition(false, _clusterName, _participants[0].getInstanceName(),
+        FULL_AUTO_RESOURCES[0], Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
+    Thread.sleep(1000);
+    Assert.assertTrue(bestPossibleVerifier.verify(3000));
+
+    // Enable the partition back
+    _admin.enablePartition(true, _clusterName, _participants[0].getInstanceName(),
+        FULL_AUTO_RESOURCES[0], Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
+    Thread.sleep(1000);
+    Assert.assertTrue(bestPossibleVerifier.verify(10000));
+
+    // Make 1 instance non-live
+    _participants[0].syncStop();
+    Thread.sleep(1000);
+    Assert.assertTrue(bestPossibleVerifier.verify(10000));
+
+    // Recover the participant before next test
+    String id = _participants[0].getInstanceName();
+    _participants[0] = new MockParticipantManager(ZK_ADDR, _clusterName, id);
+    _participants[0].syncStart();
+
     HelixClusterVerifier strictMatchVerifier =
-        new StrictMatchExternalViewVerifier.Builder(_clusterName).setZkClient(_gZkClient).build();
+        new StrictMatchExternalViewVerifier.Builder(_clusterName)
+            .setResources(Sets.newHashSet(RESOURCES)).setZkClient(_gZkClient).build();
     Assert.assertTrue(strictMatchVerifier.verify(10000));
 
     // Disable partition for 1 instance, then Full-Auto ExternalView should not match IdealState.
-    _admin.enablePartition(false, _clusterName, _participants[0].getInstanceName(), FULL_AUTO_RESOURCES[0],
-        Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
+    _admin.enablePartition(false, _clusterName, _participants[0].getInstanceName(),
+        FULL_AUTO_RESOURCES[0], Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
     Thread.sleep(1000);
-    Assert.assertFalse(strictMatchVerifier.verify(3000));
+    Assert.assertTrue(strictMatchVerifier.verify(3000));
 
     // Enable the partition back
-    _admin.enablePartition(true, _clusterName, _participants[0].getInstanceName(), FULL_AUTO_RESOURCES[0],
-        Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
+    _admin.enablePartition(true, _clusterName, _participants[0].getInstanceName(),
+        FULL_AUTO_RESOURCES[0], Lists.newArrayList(FULL_AUTO_RESOURCES[0] + "_0"));
     Thread.sleep(1000);
     Assert.assertTrue(strictMatchVerifier.verify(10000));
 
@@ -148,14 +171,16 @@ public class TestClusterVerifier extends ZkUnitTestBase {
     // Semi-Auto ExternalView should not match IdealState
     for (String resource : SEMI_AUTO_RESOURCES) {
       System.out.println("Un-verify resource: " + resource);
-      strictMatchVerifier = new StrictMatchExternalViewVerifier.Builder(_clusterName)
-          .setZkClient(_gZkClient).setResources(Sets.newHashSet(resource)).build();
+      strictMatchVerifier =
+          new StrictMatchExternalViewVerifier.Builder(_clusterName).setZkClient(_gZkClient)
+              .setResources(Sets.newHashSet(resource)).build();
       Assert.assertFalse(strictMatchVerifier.verify(3000));
     }
 
     // Full-Auto still match, because preference list wouldn't contain non-live instances
-    strictMatchVerifier = new StrictMatchExternalViewVerifier.Builder(_clusterName)
-        .setZkClient(_gZkClient).setResources(Sets.newHashSet(FULL_AUTO_RESOURCES)).build();
+    strictMatchVerifier =
+        new StrictMatchExternalViewVerifier.Builder(_clusterName).setZkClient(_gZkClient)
+            .setResources(Sets.newHashSet(FULL_AUTO_RESOURCES)).build();
     Assert.assertTrue(strictMatchVerifier.verify(10000));
   }
 


[helix] 03/37: Implement the WAGED rebalancer cluster model (#362)

Posted by ji...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 730054b90f9ac73e06168201728c89c57b69cc58
Author: jiajunwang <18...@users.noreply.github.com>
AuthorDate: Fri Aug 2 21:21:49 2019 -0700

    Implement the WAGED rebalancer cluster model (#362)
    
    * Introduce the cluster model classes to support the WAGED rebalancer.
    
    Implement the cluster model classes with the minimum necessary information to support rebalance.
    Additional field/logics might be added later once the detailed rebalance logic is implemented.
    
    Also add related tests.
---
 .../rebalancer/waged/ClusterDataProvider.java      |   2 +-
 .../rebalancer/waged/model/AssignableNode.java     | 291 ++++++++++++++++++++-
 .../rebalancer/waged/model/AssignableReplica.java  | 118 ++++++++-
 .../rebalancer/waged/model/ClusterContext.java     |  99 ++++++-
 .../rebalancer/waged/model/ClusterModel.java       | 132 +++++++++-
 .../apache/helix/model/StateModelDefinition.java   |   4 +-
 .../waged/model/AbstractTestClusterModel.java      | 176 +++++++++++++
 .../rebalancer/waged/model/TestAssignableNode.java | 203 ++++++++++++++
 .../waged/model/TestAssignableReplica.java         |  99 +++++++
 .../rebalancer/waged/model/TestClusterContext.java |  90 +++++++
 .../rebalancer/waged/model/TestClusterModel.java   | 114 ++++++++
 11 files changed, 1311 insertions(+), 17 deletions(-)

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
index 419be42..feae1dc 100644
--- 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
@@ -48,6 +48,6 @@ public class ClusterDataProvider {
       Set<String> activeInstances, Map<ClusterDataDetector.ChangeType, Set<String>> clusterChanges,
       Map<String, IdealState> baselineAssignment, Map<String, IdealState> bestPossibleAssignment) {
     // TODO finish the implementation.
-    return new ClusterModel();
+    return null;
   }
 }
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
index ae037f4..989323e 100644
--- 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
@@ -19,10 +19,291 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixException;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.max;
+
 /**
- * A placeholder before we have the implementation.
- *
- * This class represents a potential allocation of the replication.
- * Note that AssignableNode is not thread safe.
+ * This class represents a possible allocation of the replication.
+ * Note that any usage updates to the AssignableNode are not thread safe.
  */
-public class AssignableNode { }
+public class AssignableNode {
+  private static final Logger LOG = LoggerFactory.getLogger(AssignableNode.class.getName());
+
+  // basic node information
+  private final String _instanceName;
+  private Set<String> _instanceTags;
+  private String _faultZone;
+  private Map<String, List<String>> _disabledPartitionsMap;
+  private Map<String, Integer> _maxCapacity;
+  private int _maxPartition; // maximum number of the partitions that can be assigned to the node.
+
+  // proposed assignment tracking
+  // <resource name, partition name>
+  private Map<String, Set<String>> _currentAssignments;
+  // <resource name, top state partition name>
+  private Map<String, Set<String>> _currentTopStateAssignments;
+  // <capacity key, capacity value>
+  private Map<String, Integer> _currentCapacity;
+  // The maximum capacity utilization (0.0 - 1.0) across all the capacity categories.
+  private float _highestCapacityUtilization;
+
+  AssignableNode(ClusterConfig clusterConfig, InstanceConfig instanceConfig, String instanceName,
+      Collection<AssignableReplica> existingAssignment) {
+    _instanceName = instanceName;
+    refresh(clusterConfig, instanceConfig, existingAssignment);
+  }
+
+  private void reset() {
+    _currentAssignments = new HashMap<>();
+    _currentTopStateAssignments = new HashMap<>();
+    _currentCapacity = new HashMap<>();
+    _highestCapacityUtilization = 0;
+  }
+
+  /**
+   * Update the node with a ClusterDataCache. This resets the current assignment and recalculates currentCapacity.
+   * NOTE: While this is required to be used in the constructor, this can also be used when the clusterCache needs to be
+   * refreshed. This is under the assumption that the capacity mappings of InstanceConfig and ResourceConfig could
+   * subject to change. If the assumption is no longer true, this function should become private.
+   *
+   * @param clusterConfig  - the Cluster Config of the cluster where the node is located
+   * @param instanceConfig - the Instance Config of the node
+   * @param existingAssignment - all the existing replicas that are current assigned to the node
+   */
+  private void refresh(ClusterConfig clusterConfig, InstanceConfig instanceConfig,
+      Collection<AssignableReplica> existingAssignment) {
+    reset();
+
+    _currentCapacity.putAll(instanceConfig.getInstanceCapacityMap());
+    _faultZone = computeFaultZone(clusterConfig, instanceConfig);
+    _instanceTags = new HashSet<>(instanceConfig.getTags());
+    _disabledPartitionsMap = instanceConfig.getDisabledPartitionsMap();
+    _maxCapacity = instanceConfig.getInstanceCapacityMap();
+    _maxPartition = clusterConfig.getMaxPartitionsPerInstance();
+
+    assignNewBatch(existingAssignment);
+  }
+
+  /**
+   * Assign a replica to the node.
+   *
+   * @param assignableReplica - the replica to be assigned
+   */
+  void assign(AssignableReplica assignableReplica) {
+    if (!addToAssignmentRecord(assignableReplica, _currentAssignments)) {
+      throw new HelixException(String
+          .format("Resource %s already has a replica from partition %s on node %s",
+              assignableReplica.getResourceName(), assignableReplica.getPartitionName(),
+              getInstanceName()));
+    } else {
+      if (assignableReplica.isReplicaTopState()) {
+        addToAssignmentRecord(assignableReplica, _currentTopStateAssignments);
+      }
+      assignableReplica.getCapacity().entrySet().stream().forEach(
+          capacity -> updateCapacityAndUtilization(capacity.getKey(), capacity.getValue()));
+    }
+  }
+
+  /**
+   * Release a replica from the node.
+   * If the replication is not on this node, the assignable node is not updated.
+   *
+   * @param assignableReplica - the replica to be released
+   */
+  void release(AssignableReplica assignableReplica) throws IllegalArgumentException {
+    String resourceName = assignableReplica.getResourceName();
+    String partitionName = assignableReplica.getPartitionName();
+
+    // Check if the release is necessary
+    if (!_currentAssignments.containsKey(resourceName)) {
+      LOG.warn("Resource {} is not on node {}. Ignore the release call.", resourceName,
+          getInstanceName());
+      return;
+    }
+    Set<String> partitions = _currentAssignments.get(resourceName);
+    if (!partitions.contains(partitionName)) {
+      LOG.warn(String
+          .format("Resource %s does not have a replica from partition %s on node %s", resourceName,
+              partitionName, getInstanceName()));
+      return;
+    }
+
+    partitions.remove(assignableReplica.getPartitionName());
+    if (assignableReplica.isReplicaTopState()) {
+      _currentTopStateAssignments.get(resourceName).remove(partitionName);
+    }
+    // Recalculate utilization because of release
+    _highestCapacityUtilization = 0;
+    assignableReplica.getCapacity().entrySet().stream()
+        .forEach(entry -> updateCapacityAndUtilization(entry.getKey(), -1 * entry.getValue()));
+  }
+
+  public Map<String, Set<String>> getCurrentAssignmentsMap() {
+    return _currentAssignments;
+  }
+
+  public Set<String> getCurrentAssignmentsByResource(String resource) {
+    return _currentAssignments.getOrDefault(resource, Collections.emptySet());
+  }
+
+  public Set<String> getCurrentTopStateAssignmentsByResource(String resource) {
+    return _currentTopStateAssignments.getOrDefault(resource, Collections.emptySet());
+  }
+
+  public int getTopStateAssignmentTotalSize() {
+    return _currentTopStateAssignments.values().stream().mapToInt(Set::size).sum();
+  }
+
+  public int getCurrentAssignmentCount() {
+    return _currentAssignments.values().stream().mapToInt(Set::size).sum();
+  }
+
+  public Map<String, Integer> getCurrentCapacity() {
+    return _currentCapacity;
+  }
+
+  public float getHighestCapacityUtilization() {
+    return _highestCapacityUtilization;
+  }
+
+  public String getInstanceName() {
+    return _instanceName;
+  }
+
+  public Set<String> getInstanceTags() {
+    return _instanceTags;
+  }
+
+  public String getFaultZone() {
+    return _faultZone;
+  }
+
+  public Map<String, List<String>> getDisabledPartitionsMap() {
+    return _disabledPartitionsMap;
+  }
+
+  public Map<String, Integer> getMaxCapacity() {
+    return _maxCapacity;
+  }
+
+  public int getMaxPartition() {
+    return _maxPartition;
+  }
+
+  /**
+   * Computes the fault zone id based on the domain and fault zone type when topology is enabled. For example, when
+   * the domain is "zone=2, instance=testInstance" and the fault zone type is "zone", this function returns "2".
+   * If cannot find the fault zone id, this function leaves the fault zone id as the instance name.
+   * TODO merge this logic with Topology.java tree building logic.
+   * For now, the WAGED rebalancer has a more strict topology def requirement.
+   * Any missing field will cause an invalid topology config exception.
+   */
+  private String computeFaultZone(ClusterConfig clusterConfig, InstanceConfig instanceConfig) {
+    if (clusterConfig.isTopologyAwareEnabled()) {
+      String topologyStr = clusterConfig.getTopology();
+      String faultZoneType = clusterConfig.getFaultZoneType();
+      if (topologyStr == null || faultZoneType == null) {
+        throw new HelixException("Fault zone or cluster topology information is not configured.");
+      }
+
+      String[] topologyDef = topologyStr.trim().split("/");
+      if (topologyDef.length == 0 || Arrays.stream(topologyDef)
+          .noneMatch(type -> type.equals(faultZoneType))) {
+        throw new HelixException(
+            "The configured topology definition is empty or does not contain the fault zone type.");
+      }
+
+      Map<String, String> domainAsMap = instanceConfig.getDomainAsMap();
+      if (domainAsMap == null) {
+        throw new HelixException(
+            String.format("The domain configuration of node %s is not configured", _instanceName));
+      } else {
+        StringBuilder faultZoneStringBuilder = new StringBuilder();
+        for (String key : topologyDef) {
+          if (!key.isEmpty()) {
+            if (domainAsMap.containsKey(key)) {
+              faultZoneStringBuilder.append(domainAsMap.get(key));
+              faultZoneStringBuilder.append('/');
+            } else {
+              throw new HelixException(String.format(
+                  "The domain configuration of node %s is not complete. Type %s is not found.",
+                  _instanceName, key));
+            }
+            if (key.equals(faultZoneType)) {
+              break;
+            }
+          }
+        }
+        return faultZoneStringBuilder.toString();
+      }
+    } else {
+      // For backward compatibility
+      String zoneId = instanceConfig.getZoneId();
+      return zoneId == null ? instanceConfig.getInstanceName() : zoneId;
+    }
+  }
+
+  /**
+   * This function should only be used to assign a set of new partitions that are not allocated on this node.
+   * Using this function avoids the overhead of updating capacity repeatedly.
+   */
+  private void assignNewBatch(Collection<AssignableReplica> replicas) {
+    Map<String, Integer> totalPartitionCapacity = new HashMap<>();
+    for (AssignableReplica replica : replicas) {
+      addToAssignmentRecord(replica, _currentAssignments);
+      if (replica.isReplicaTopState()) {
+        addToAssignmentRecord(replica, _currentTopStateAssignments);
+      }
+      // increment the capacity requirement according to partition's capacity configuration.
+      for (Map.Entry<String, Integer> capacity : replica.getCapacity().entrySet()) {
+        totalPartitionCapacity.compute(capacity.getKey(),
+            (key, totalValue) -> (totalValue == null) ?
+                capacity.getValue() :
+                totalValue + capacity.getValue());
+      }
+    }
+
+    // Update the global state after all single replications' calculation is done.
+    for (String key : totalPartitionCapacity.keySet()) {
+      updateCapacityAndUtilization(key, totalPartitionCapacity.get(key));
+    }
+  }
+
+  private boolean addToAssignmentRecord(AssignableReplica replica,
+      Map<String, Set<String>> currentAssignments) {
+    return currentAssignments.computeIfAbsent(replica.getResourceName(), k -> new HashSet<>())
+        .add(replica.getPartitionName());
+  }
+
+  private void updateCapacityAndUtilization(String capacityKey, int valueToSubtract) {
+    if (_currentCapacity.containsKey(capacityKey)) {
+      int newCapacity = _currentCapacity.get(capacityKey) - valueToSubtract;
+      _currentCapacity.put(capacityKey, newCapacity);
+      // For the purpose of constraint calculation, the max utilization cannot be larger than 100%.
+      float utilization = Math.min(
+          (float) (_maxCapacity.get(capacityKey) - newCapacity) / _maxCapacity.get(capacityKey), 1);
+      _highestCapacityUtilization = max(_highestCapacityUtilization, utilization);
+    }
+    // else if the capacityKey does not exist in the capacity map, this method essentially becomes
+    // a NOP; in other words, this node will be treated as if it has unlimited capacity.
+  }
+
+  @Override
+  public int hashCode() {
+    return _instanceName.hashCode();
+  }
+}
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
index a6a7e4a..0082a2d 100644
--- 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
@@ -19,9 +19,121 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.model.StateModelDefinition;
+
+import java.io.IOException;
+import java.util.Map;
+
 /**
- * A placeholder before we have the implementation.
- *
  * This class represents a partition replication that needs to be allocated.
  */
-public class AssignableReplica { }
+public class AssignableReplica implements Comparable<AssignableReplica> {
+  private final String _partitionName;
+  private final String _resourceName;
+  private final String _resourceInstanceGroupTag;
+  private final int _resourceMaxPartitionsPerInstance;
+  private final Map<String, Integer> _capacityUsage;
+  // The priority of the replica's state
+  private final int _statePriority;
+  // The state of the replica
+  private final String _replicaState;
+
+  /**
+   * @param resourceConfig The resource config for the resource which contains the replication.
+   * @param partitionName  The replication's partition name.
+   * @param replicaState   The state of the replication.
+   * @param statePriority  The priority of the replication's state.
+   */
+  AssignableReplica(ResourceConfig resourceConfig, String partitionName, String replicaState,
+      int statePriority) {
+    _partitionName = partitionName;
+    _replicaState = replicaState;
+    _statePriority = statePriority;
+    _resourceName = resourceConfig.getResourceName();
+    _capacityUsage = fetchCapacityUsage(partitionName, resourceConfig);
+    _resourceInstanceGroupTag = resourceConfig.getInstanceGroupTag();
+    _resourceMaxPartitionsPerInstance = resourceConfig.getMaxPartitionsPerInstance();
+  }
+
+  public Map<String, Integer> getCapacity() {
+    return _capacityUsage;
+  }
+
+  public String getPartitionName() {
+    return _partitionName;
+  }
+
+  public String getReplicaState() {
+    return _replicaState;
+  }
+
+  public boolean isReplicaTopState() {
+    return _statePriority == StateModelDefinition.TOP_STATE_PRIORITY;
+  }
+
+  public int getStatePriority() {
+    return _statePriority;
+  }
+
+  public String getResourceName() {
+    return _resourceName;
+  }
+
+  public String getResourceInstanceGroupTag() {
+    return _resourceInstanceGroupTag;
+  }
+
+  public int getResourceMaxPartitionsPerInstance() {
+    return _resourceMaxPartitionsPerInstance;
+  }
+
+  @Override
+  public String toString() {
+    return generateReplicaKey(_resourceName, _partitionName, _replicaState);
+  }
+
+  @Override
+  public int compareTo(AssignableReplica replica) {
+    if (!_resourceName.equals(replica._resourceName)) {
+      return _resourceName.compareTo(replica._resourceName);
+    }
+    if (!_partitionName.equals(replica._partitionName)) {
+      return _partitionName.compareTo(replica._partitionName);
+    }
+    if (!_replicaState.equals(replica._replicaState)) {
+      return _replicaState.compareTo(replica._replicaState);
+    }
+    return 0;
+  }
+
+  public static String generateReplicaKey(String resourceName, String partitionName, String state) {
+    return String.format("%s-%s-%s", resourceName, partitionName, state);
+  }
+
+  /**
+   * Parse the resource config for the partition weight.
+   */
+  private Map<String, Integer> fetchCapacityUsage(String partitionName,
+      ResourceConfig resourceConfig) {
+    Map<String, Map<String, Integer>> capacityMap;
+    try {
+      capacityMap = resourceConfig.getPartitionCapacityMap();
+    } catch (IOException ex) {
+      throw new IllegalArgumentException(
+          "Invalid partition capacity configuration of resource: " + resourceConfig
+              .getResourceName(), ex);
+    }
+
+    Map<String, Integer> partitionCapacity = capacityMap.get(partitionName);
+    if (partitionCapacity == null) {
+      partitionCapacity = capacityMap.get(ResourceConfig.DEFAULT_PARTITION_KEY);
+    }
+    if (partitionCapacity == null) {
+      throw new IllegalArgumentException(String.format(
+          "The capacity usage of the specified partition %s is not configured in the Resource Config %s. No default partition capacity is configured neither.",
+          partitionName, resourceConfig.getResourceName()));
+    }
+    return partitionCapacity;
+  }
+}
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
index adca7d1..c163e4c 100644
--- 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
@@ -19,9 +19,100 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixException;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 /**
- * A placeholder before we have the implementation.
- *
- * This class tracks the global rebalance-related status of a Helix managed cluster.
+ * This class tracks the rebalance-related global cluster status.
  */
-public class ClusterContext { }
+public class ClusterContext {
+  private final static float ERROR_MARGIN_FOR_ESTIMATED_MAX_COUNT = 1.1f;
+
+  // This estimation helps to ensure global partition count evenness
+  private final int _estimatedMaxPartitionCount;
+  // This estimation helps to ensure global top state replica count evenness
+  private final int _estimatedMaxTopStateCount;
+  // This estimation helps to ensure per-resource partition count evenness
+  private final Map<String, Integer> _estimatedMaxPartitionByResource = new HashMap<>();
+
+  // map{zoneName : map{resourceName : set(partitionNames)}}
+  private Map<String, Map<String, Set<String>>> _assignmentForFaultZoneMap = new HashMap<>();
+
+  /**
+   * Construct the cluster context based on the current instance status.
+   *
+   * @param replicaSet    All the partition replicas that are managed by the rebalancer
+   * @param instanceCount The count of all the active instances that can be used to host partitions.
+   */
+  ClusterContext(Set<AssignableReplica> replicaSet, int instanceCount) {
+    int totalReplicas = 0;
+    int totalTopStateReplicas = 0;
+
+    for (Map.Entry<String, List<AssignableReplica>> entry : replicaSet.stream()
+        .collect(Collectors.groupingBy(AssignableReplica::getResourceName)).entrySet()) {
+      int replicas = entry.getValue().size();
+      totalReplicas += replicas;
+
+      int replicaCnt = Math.max(1, estimateAvgReplicaCount(replicas, instanceCount));
+      _estimatedMaxPartitionByResource.put(entry.getKey(), replicaCnt);
+
+      totalTopStateReplicas +=
+          entry.getValue().stream().filter(AssignableReplica::isReplicaTopState).count();
+    }
+
+    _estimatedMaxPartitionCount = estimateAvgReplicaCount(totalReplicas, instanceCount);
+    _estimatedMaxTopStateCount = estimateAvgReplicaCount(totalTopStateReplicas, instanceCount);
+  }
+
+  public Map<String, Map<String, Set<String>>> getAssignmentForFaultZoneMap() {
+    return _assignmentForFaultZoneMap;
+  }
+
+  public int getEstimatedMaxPartitionCount() {
+    return _estimatedMaxPartitionCount;
+  }
+
+  public int getEstimatedMaxPartitionByResource(String resourceName) {
+    return _estimatedMaxPartitionByResource.get(resourceName);
+  }
+
+  public int getEstimatedMaxTopStateCount() {
+    return _estimatedMaxTopStateCount;
+  }
+
+  public Set<String> getPartitionsForResourceAndFaultZone(String resourceName, String faultZoneId) {
+    return _assignmentForFaultZoneMap.getOrDefault(faultZoneId, Collections.emptyMap())
+        .getOrDefault(resourceName, Collections.emptySet());
+  }
+
+  void addPartitionToFaultZone(String faultZoneId, String resourceName, String partition) {
+    if (!_assignmentForFaultZoneMap.computeIfAbsent(faultZoneId, k -> new HashMap<>())
+        .computeIfAbsent(resourceName, k -> new HashSet<>()).add(partition)) {
+      throw new HelixException(String
+          .format("Resource %s already has a replica from partition %s in fault zone %s",
+              resourceName, partition, faultZoneId));
+    }
+  }
+
+  boolean removePartitionFromFaultZone(String faultZoneId, String resourceName, String partition) {
+    return _assignmentForFaultZoneMap.getOrDefault(faultZoneId, Collections.emptyMap())
+        .getOrDefault(resourceName, Collections.emptySet()).remove(partition);
+  }
+
+  void setAssignmentForFaultZoneMap(
+      Map<String, Map<String, Set<String>>> assignmentForFaultZoneMap) {
+    _assignmentForFaultZoneMap = assignmentForFaultZoneMap;
+  }
+
+  private int estimateAvgReplicaCount(int replicaCount, int instanceCount) {
+    return (int) Math
+        .ceil((float) replicaCount / instanceCount * ERROR_MARGIN_FOR_ESTIMATED_MAX_COUNT);
+  }
+}
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
index 06eebf7..2908939 100644
--- 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
@@ -19,9 +19,135 @@ package org.apache.helix.controller.rebalancer.waged.model;
  * under the License.
  */
 
+import org.apache.helix.HelixException;
+import org.apache.helix.model.IdealState;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 /**
- * A placeholder before we have the implementation.
- *
  * This class wraps the required input for the rebalance algorithm.
  */
-public class ClusterModel { }
+public class ClusterModel {
+  private final ClusterContext _clusterContext;
+  // Map to track all the assignable replications. <Resource Name, Set<Replicas>>
+  private final Map<String, Set<AssignableReplica>> _assignableReplicaMap;
+  // The index to find the replication information with a certain state. <Resource, <Key(resource_partition_state), Replica>>
+  // Note that the identical replicas are deduped in the index.
+  private final Map<String, Map<String, AssignableReplica>> _assignableReplicaIndex;
+  private final Map<String, AssignableNode> _assignableNodeMap;
+
+  // Records about the previous assignment
+  // <ResourceName, IdealState contains the baseline assignment>
+  private final Map<String, IdealState> _baselineAssignment;
+  // <ResourceName, IdealState contains the best possible assignment>
+  private final Map<String, IdealState> _bestPossibleAssignment;
+
+  /**
+   * @param clusterContext         The initialized cluster context.
+   * @param assignableReplicas     The replicas to be assigned.
+   *                               Note that the replicas in this list shall not be included while initializing the context and assignable nodes.
+   * @param assignableNodes        The active instances.
+   * @param baselineAssignment     The recorded baseline assignment.
+   * @param bestPossibleAssignment The current best possible assignment.
+   */
+  ClusterModel(ClusterContext clusterContext, Set<AssignableReplica> assignableReplicas,
+      Set<AssignableNode> assignableNodes, Map<String, IdealState> baselineAssignment,
+      Map<String, IdealState> bestPossibleAssignment) {
+    _clusterContext = clusterContext;
+
+    // Save all the to be assigned replication
+    _assignableReplicaMap = assignableReplicas.stream()
+        .collect(Collectors.groupingBy(AssignableReplica::getResourceName, Collectors.toSet()));
+
+    // Index all the replicas to be assigned. Dedup the replica if two instances have the same resource/partition/state
+    _assignableReplicaIndex = assignableReplicas.stream().collect(Collectors
+        .groupingBy(AssignableReplica::getResourceName, Collectors
+            .toMap(AssignableReplica::toString, replica -> replica,
+                (oldValue, newValue) -> oldValue)));
+
+    _assignableNodeMap = assignableNodes.stream()
+        .collect(Collectors.toMap(AssignableNode::getInstanceName, node -> node));
+
+    _baselineAssignment = baselineAssignment;
+    _bestPossibleAssignment = bestPossibleAssignment;
+  }
+
+  public ClusterContext getContext() {
+    return _clusterContext;
+  }
+
+  public Map<String, AssignableNode> getAssignableNodes() {
+    return _assignableNodeMap;
+  }
+
+  public Map<String, Set<AssignableReplica>> getAssignableReplicaMap() {
+    return _assignableReplicaMap;
+  }
+
+  public Map<String, IdealState> getBaseline() {
+    return _baselineAssignment;
+  }
+
+  public Map<String, IdealState> getBestPossibleAssignment() {
+    return _bestPossibleAssignment;
+  }
+
+  /**
+   * Assign the given replica to the specified instance and record the assignment in the cluster model.
+   * The cluster usage information will be updated accordingly.
+   *
+   * @param resourceName
+   * @param partitionName
+   * @param state
+   * @param instanceName
+   */
+  public void assign(String resourceName, String partitionName, String state, String instanceName) {
+    AssignableNode node = locateAssignableNode(instanceName);
+    AssignableReplica replica = locateAssignableReplica(resourceName, partitionName, state);
+
+    node.assign(replica);
+    _clusterContext.addPartitionToFaultZone(node.getFaultZone(), resourceName, partitionName);
+  }
+
+  /**
+   * Revert the proposed assignment from the cluster model.
+   * The cluster usage information will be updated accordingly.
+   *
+   * @param resourceName
+   * @param partitionName
+   * @param state
+   * @param instanceName
+   */
+  public void release(String resourceName, String partitionName, String state,
+      String instanceName) {
+    AssignableNode node = locateAssignableNode(instanceName);
+    AssignableReplica replica = locateAssignableReplica(resourceName, partitionName, state);
+
+    node.release(replica);
+    _clusterContext.removePartitionFromFaultZone(node.getFaultZone(), resourceName, partitionName);
+  }
+
+  private AssignableNode locateAssignableNode(String instanceName) {
+    AssignableNode node = _assignableNodeMap.get(instanceName);
+    if (node == null) {
+      throw new HelixException("Cannot find the instance: " + instanceName);
+    }
+    return node;
+  }
+
+  private AssignableReplica locateAssignableReplica(String resourceName, String partitionName,
+      String state) {
+    AssignableReplica sampleReplica =
+        _assignableReplicaIndex.getOrDefault(resourceName, Collections.emptyMap())
+            .get(AssignableReplica.generateReplicaKey(resourceName, partitionName, state));
+    if (sampleReplica == null) {
+      throw new HelixException(String
+          .format("Cannot find the replication with resource name %s, partition name %s, state %s.",
+              resourceName, partitionName, state));
+    }
+    return sampleReplica;
+  }
+}
diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
index ae59522..0a40331 100644
--- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
+++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
@@ -46,6 +46,8 @@ public class StateModelDefinition extends HelixProperty {
     STATE_PRIORITY_LIST
   }
 
+  public static final int TOP_STATE_PRIORITY = 1;
+
   /**
    * state model's initial state
    */
@@ -98,7 +100,7 @@ public class StateModelDefinition extends HelixProperty {
     _stateTransitionTable = new HashMap<>();
     _statesCountMap = new HashMap<>();
     if (_statesPriorityList != null) {
-      int priority = 1;
+      int priority = TOP_STATE_PRIORITY;
       for (String state : _statesPriorityList) {
         Map<String, String> metaData = record.getMapField(state + ".meta");
         if (metaData != null) {
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
new file mode 100644
index 0000000..0e2b43a
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/AbstractTestClusterModel.java
@@ -0,0 +1,176 @@
+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.
+ */
+
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.InstanceConfig;
+import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.ResourceConfig;
+import org.mockito.Mockito;
+import org.testng.annotations.BeforeClass;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Mockito.when;
+
+public abstract class AbstractTestClusterModel {
+  protected String _testInstanceId;
+  protected List<String> _resourceNames;
+  protected List<String> _partitionNames;
+  protected Map<String, Integer> _capacityDataMap;
+  protected Map<String, List<String>> _disabledPartitionsMap;
+  protected List<String> _testInstanceTags;
+  protected String _testFaultZoneId;
+
+  @BeforeClass
+  public void initialize() {
+    _testInstanceId = "testInstanceId";
+    _resourceNames = new ArrayList<>();
+    _resourceNames.add("Resource1");
+    _resourceNames.add("Resource2");
+    _partitionNames = new ArrayList<>();
+    _partitionNames.add("Partition1");
+    _partitionNames.add("Partition2");
+    _partitionNames.add("Partition3");
+    _partitionNames.add("Partition4");
+    _capacityDataMap = new HashMap<>();
+    _capacityDataMap.put("item1", 20);
+    _capacityDataMap.put("item2", 40);
+    _capacityDataMap.put("item3", 30);
+    List<String> disabledPartitions = new ArrayList<>();
+    disabledPartitions.add("TestPartition");
+    _disabledPartitionsMap = new HashMap<>();
+    _disabledPartitionsMap.put("TestResource", disabledPartitions);
+    _testInstanceTags = new ArrayList<>();
+    _testInstanceTags.add("TestTag");
+    _testFaultZoneId = "testZone";
+  }
+
+  protected ResourceControllerDataProvider setupClusterDataCache() throws IOException {
+    ResourceControllerDataProvider testCache = Mockito.mock(ResourceControllerDataProvider.class);
+
+    // 1. Set up the default instance information with capacity configuration.
+    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceId");
+    testInstanceConfig.setInstanceCapacityMap(_capacityDataMap);
+    testInstanceConfig.addTag(_testInstanceTags.get(0));
+    testInstanceConfig.setInstanceEnabledForPartition("TestResource", "TestPartition", false);
+    testInstanceConfig.setInstanceEnabled(true);
+    testInstanceConfig.setZoneId(_testFaultZoneId);
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    instanceConfigMap.put(_testInstanceId, testInstanceConfig);
+    when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+
+    // 2. Set up the basic cluster configuration.
+    ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
+    testClusterConfig.setMaxPartitionsPerInstance(5);
+    testClusterConfig.setDisabledInstances(Collections.emptyMap());
+    testClusterConfig.setTopologyAwareEnabled(false);
+    when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
+
+    // 3. Mock the live instance node for the default instance.
+    LiveInstance testLiveInstance = new LiveInstance(_testInstanceId);
+    testLiveInstance.setSessionId("testSessionId");
+    Map<String, LiveInstance> liveInstanceMap = new HashMap<>();
+    liveInstanceMap.put(_testInstanceId, testLiveInstance);
+    when(testCache.getLiveInstances()).thenReturn(liveInstanceMap);
+
+    // 4. Mock two resources, each with 2 partitions on the default instance.
+    // The instance will have the following partitions assigned
+    // Resource 1:
+    //          partition 1 - MASTER
+    //          partition 2 - SLAVE
+    // Resource 2:
+    //          partition 3 - MASTER
+    //          partition 4 - SLAVE
+    CurrentState testCurrentStateResource1 = Mockito.mock(CurrentState.class);
+    Map<String, String> partitionStateMap1 = new HashMap<>();
+    partitionStateMap1.put(_partitionNames.get(0), "MASTER");
+    partitionStateMap1.put(_partitionNames.get(1), "SLAVE");
+    when(testCurrentStateResource1.getResourceName()).thenReturn(_resourceNames.get(0));
+    when(testCurrentStateResource1.getPartitionStateMap()).thenReturn(partitionStateMap1);
+    when(testCurrentStateResource1.getStateModelDefRef()).thenReturn("MasterSlave");
+    when(testCurrentStateResource1.getState(_partitionNames.get(0))).thenReturn("MASTER");
+    when(testCurrentStateResource1.getState(_partitionNames.get(1))).thenReturn("SLAVE");
+    CurrentState testCurrentStateResource2 = Mockito.mock(CurrentState.class);
+    Map<String, String> partitionStateMap2 = new HashMap<>();
+    partitionStateMap2.put(_partitionNames.get(2), "MASTER");
+    partitionStateMap2.put(_partitionNames.get(3), "SLAVE");
+    when(testCurrentStateResource2.getResourceName()).thenReturn(_resourceNames.get(1));
+    when(testCurrentStateResource2.getPartitionStateMap()).thenReturn(partitionStateMap2);
+    when(testCurrentStateResource2.getStateModelDefRef()).thenReturn("MasterSlave");
+    when(testCurrentStateResource2.getState(_partitionNames.get(2))).thenReturn("MASTER");
+    when(testCurrentStateResource2.getState(_partitionNames.get(3))).thenReturn("SLAVE");
+    Map<String, CurrentState> currentStatemap = new HashMap<>();
+    currentStatemap.put(_resourceNames.get(0), testCurrentStateResource1);
+    currentStatemap.put(_resourceNames.get(1), testCurrentStateResource2);
+    when(testCache.getCurrentState(_testInstanceId, "testSessionId")).thenReturn(currentStatemap);
+
+    // 5. Set up the resource config for the two resources with the partition weight.
+    Map<String, Integer> capacityDataMapResource1 = new HashMap<>();
+    capacityDataMapResource1.put("item1", 3);
+    capacityDataMapResource1.put("item2", 6);
+    ResourceConfig testResourceConfigResource1 = new ResourceConfig("Resource1");
+    testResourceConfigResource1.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMapResource1));
+    when(testCache.getResourceConfig("Resource1")).thenReturn(testResourceConfigResource1);
+    Map<String, Integer> capacityDataMapResource2 = new HashMap<>();
+    capacityDataMapResource2.put("item1", 5);
+    capacityDataMapResource2.put("item2", 10);
+    ResourceConfig testResourceConfigResource2 = new ResourceConfig("Resource2");
+    testResourceConfigResource2.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMapResource2));
+    when(testCache.getResourceConfig("Resource2")).thenReturn(testResourceConfigResource2);
+
+    // 6. Define mock state model
+    for (BuiltInStateModelDefinitions bsmd : BuiltInStateModelDefinitions.values()) {
+      when(testCache.getStateModelDef(bsmd.name())).thenReturn(bsmd.getStateModelDefinition());
+    }
+
+    return testCache;
+  }
+
+  /**
+   * Generate the replica objects according to the provider information.
+   */
+  protected Set<AssignableReplica> generateReplicas(ResourceControllerDataProvider dataProvider) {
+    // Create assignable replica based on the current state.
+    Map<String, CurrentState> currentStatemap =
+        dataProvider.getCurrentState(_testInstanceId, "testSessionId");
+    Set<AssignableReplica> assignmentSet = new HashSet<>();
+    for (CurrentState cs : currentStatemap.values()) {
+      ResourceConfig resourceConfig = dataProvider.getResourceConfig(cs.getResourceName());
+      // Construct one AssignableReplica for each partition in the current state.
+      cs.getPartitionStateMap().entrySet().stream().forEach(entry -> assignmentSet.add(
+          new AssignableReplica(resourceConfig, entry.getKey(), entry.getValue(),
+              entry.getValue().equals("MASTER") ? 1 : 2)));
+    }
+    return assignmentSet;
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
new file mode 100644
index 0000000..d7fcce9
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableNode.java
@@ -0,0 +1,203 @@
+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.
+ */
+
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.InstanceConfig;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Mockito.when;
+
+public class TestAssignableNode extends AbstractTestClusterModel {
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+  }
+
+  @Test
+  public void testNormalUsage() throws IOException {
+    // Test 1 - initialize based on the data cache and check with the expected result
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
+
+    Set<String> expectedAssignmentSet1 = new HashSet<>(_partitionNames.subList(0, 2));
+    Set<String> expectedAssignmentSet2 = new HashSet<>(_partitionNames.subList(2, 4));
+    Map<String, Set<String>> expectedAssignment = new HashMap<>();
+    expectedAssignment.put("Resource1", expectedAssignmentSet1);
+    expectedAssignment.put("Resource2", expectedAssignmentSet2);
+    Map<String, Integer> expectedCapacityMap = new HashMap<>();
+    expectedCapacityMap.put("item1", 4);
+    expectedCapacityMap.put("item2", 8);
+    expectedCapacityMap.put("item3", 30);
+
+    AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
+    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
+    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 4);
+    Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
+    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxPartition(), 5);
+    Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
+    Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
+    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
+    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+
+    // Test 2 - release assignment from the AssignableNode
+    AssignableReplica removingReplica =
+        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
+            _partitionNames.get(2), "MASTER", 1);
+    expectedAssignment.get(_resourceNames.get(1)).remove(_partitionNames.get(2));
+    expectedCapacityMap.put("item1", 9);
+    expectedCapacityMap.put("item2", 18);
+
+    assignableNode.release(removingReplica);
+
+    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
+    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 3);
+    Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 11.0 / 20.0, 0.005);
+    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxPartition(), 5);
+    Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
+    Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
+    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
+    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+
+    // Test 3 - add assignment to the AssignableNode
+    AssignableReplica addingReplica =
+        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
+            _partitionNames.get(2), "SLAVE", 2);
+    expectedAssignment.get(_resourceNames.get(1)).add(_partitionNames.get(2));
+    expectedCapacityMap.put("item1", 4);
+    expectedCapacityMap.put("item2", 8);
+
+    assignableNode.assign(addingReplica);
+
+    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().equals(expectedAssignment));
+    Assert.assertEquals(assignableNode.getCurrentAssignmentCount(), 4);
+    Assert.assertEquals(assignableNode.getHighestCapacityUtilization(), 16.0 / 20.0, 0.005);
+    Assert.assertTrue(assignableNode.getMaxCapacity().equals(_capacityDataMap));
+    Assert.assertEquals(assignableNode.getMaxPartition(), 5);
+    Assert.assertEquals(assignableNode.getInstanceTags(), _testInstanceTags);
+    Assert.assertEquals(assignableNode.getFaultZone(), _testFaultZoneId);
+    Assert.assertTrue(assignableNode.getDisabledPartitionsMap().equals(_disabledPartitionsMap));
+    Assert.assertTrue(assignableNode.getCurrentCapacity().equals(expectedCapacityMap));
+  }
+
+  @Test
+  public void testReleaseNoPartition() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+
+    AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
+        Collections.emptyList());
+    AssignableReplica removingReplica =
+        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(1)),
+            _partitionNames.get(2) + "non-exist", "MASTER", 1);
+
+    // Release shall pass.
+    assignableNode.release(removingReplica);
+  }
+
+  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "Resource Resource1 already has a replica from partition Partition1 on node testInstanceId")
+  public void testAssignDuplicateReplica() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
+
+    AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId, assignmentSet);
+    AssignableReplica duplicateReplica =
+        new AssignableReplica(testCache.getResourceConfig(_resourceNames.get(0)),
+            _partitionNames.get(0), "SLAVE", 2);
+    assignableNode.assign(duplicateReplica);
+  }
+
+  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "The domain configuration of node testInstanceId is not complete. Type DOES_NOT_EXIST is not found.")
+  public void testParseFaultZoneNotFound() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+
+    ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
+    testClusterConfig.setFaultZoneType("DOES_NOT_EXIST");
+    testClusterConfig.setTopologyAwareEnabled(true);
+    testClusterConfig.setTopology("/DOES_NOT_EXIST/");
+    when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
+
+    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
+    testInstanceConfig.setDomain("zone=2, instance=testInstance");
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    instanceConfigMap.put(_testInstanceId, testInstanceConfig);
+    when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+
+    new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
+        Collections.emptyList());
+  }
+
+  @Test
+  public void testParseFaultZone() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+
+    ClusterConfig testClusterConfig = new ClusterConfig("testClusterConfigId");
+    testClusterConfig.setFaultZoneType("zone");
+    testClusterConfig.setTopologyAwareEnabled(true);
+    testClusterConfig.setTopology("/zone/instance");
+    when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
+
+    InstanceConfig testInstanceConfig = new InstanceConfig("testInstanceConfigId");
+    testInstanceConfig.setDomain("zone=2, instance=testInstance");
+    Map<String, InstanceConfig> instanceConfigMap = new HashMap<>();
+    instanceConfigMap.put(_testInstanceId, testInstanceConfig);
+    when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+
+    AssignableNode assignableNode = new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
+        Collections.emptyList());
+
+    Assert.assertEquals(assignableNode.getFaultZone(), "2/");
+
+    testClusterConfig = new ClusterConfig("testClusterConfigId");
+    testClusterConfig.setFaultZoneType("instance");
+    testClusterConfig.setTopologyAwareEnabled(true);
+    testClusterConfig.setTopology("/zone/instance");
+    when(testCache.getClusterConfig()).thenReturn(testClusterConfig);
+
+    testInstanceConfig = new InstanceConfig("testInstanceConfigId");
+    testInstanceConfig.setDomain("zone=2, instance=testInstance");
+    instanceConfigMap = new HashMap<>();
+    instanceConfigMap.put(_testInstanceId, testInstanceConfig);
+    when(testCache.getInstanceConfigMap()).thenReturn(instanceConfigMap);
+
+    assignableNode = new AssignableNode(testCache.getClusterConfig(),
+        testCache.getInstanceConfigMap().get(_testInstanceId), _testInstanceId,
+        Collections.emptyList());
+
+    Assert.assertEquals(assignableNode.getFaultZone(), "2/testInstance/");
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java
new file mode 100644
index 0000000..d069ced
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestAssignableReplica.java
@@ -0,0 +1,99 @@
+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.
+ */
+
+import org.apache.helix.model.ResourceConfig;
+import org.apache.helix.model.StateModelDefinition;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestAssignableReplica {
+  String resourceName = "Resource";
+  String partitionNamePrefix = "partition";
+  String masterState = "Master";
+  int masterPriority = StateModelDefinition.TOP_STATE_PRIORITY;
+  String slaveState = "Slave";
+  int slavePriority = 2;
+
+  @Test
+  public void testConstructRepliaWithResourceConfig() throws IOException {
+    // Init assignable replica with a basic config object
+    Map<String, Integer> capacityDataMapResource1 = new HashMap<>();
+    capacityDataMapResource1.put("item1", 3);
+    capacityDataMapResource1.put("item2", 6);
+    ResourceConfig testResourceConfigResource = new ResourceConfig(resourceName);
+    testResourceConfigResource.setPartitionCapacityMap(
+        Collections.singletonMap(ResourceConfig.DEFAULT_PARTITION_KEY, capacityDataMapResource1));
+
+    String partitionName = partitionNamePrefix + 1;
+    AssignableReplica replica =
+        new AssignableReplica(testResourceConfigResource, partitionName, masterState,
+            masterPriority);
+    Assert.assertEquals(replica.getResourceName(), resourceName);
+    Assert.assertEquals(replica.getPartitionName(), partitionName);
+    Assert.assertEquals(replica.getReplicaState(), masterState);
+    Assert.assertEquals(replica.getStatePriority(), masterPriority);
+    Assert.assertTrue(replica.isReplicaTopState());
+    Assert.assertEquals(replica.getCapacity(), capacityDataMapResource1);
+    Assert.assertEquals(replica.getResourceInstanceGroupTag(), null);
+    Assert.assertEquals(replica.getResourceMaxPartitionsPerInstance(), Integer.MAX_VALUE);
+
+    // Modify the config and initialize more replicas.
+    // 1. update capacity
+    Map<String, Integer> capacityDataMapResource2 = new HashMap<>();
+    capacityDataMapResource2.put("item1", 5);
+    capacityDataMapResource2.put("item2", 10);
+    Map<String, Map<String, Integer>> capacityMap =
+        testResourceConfigResource.getPartitionCapacityMap();
+    String partitionName2 = partitionNamePrefix + 2;
+    capacityMap.put(partitionName2, capacityDataMapResource2);
+    testResourceConfigResource.setPartitionCapacityMap(capacityMap);
+    // 2. update instance group tag and max partitions per instance
+    String group = "DEFAULT";
+    int maxPartition = 10;
+    testResourceConfigResource.getRecord()
+        .setSimpleField(ResourceConfig.ResourceConfigProperty.INSTANCE_GROUP_TAG.toString(), group);
+    testResourceConfigResource.getRecord()
+        .setIntField(ResourceConfig.ResourceConfigProperty.MAX_PARTITIONS_PER_INSTANCE.name(),
+            maxPartition);
+
+    replica = new AssignableReplica(testResourceConfigResource, partitionName, masterState,
+        masterPriority);
+    Assert.assertEquals(replica.getCapacity(), capacityDataMapResource1);
+    Assert.assertEquals(replica.getResourceInstanceGroupTag(), group);
+    Assert.assertEquals(replica.getResourceMaxPartitionsPerInstance(), maxPartition);
+
+    replica = new AssignableReplica(testResourceConfigResource, partitionName2, slaveState,
+        slavePriority);
+    Assert.assertEquals(replica.getResourceName(), resourceName);
+    Assert.assertEquals(replica.getPartitionName(), partitionName2);
+    Assert.assertEquals(replica.getReplicaState(), slaveState);
+    Assert.assertEquals(replica.getStatePriority(), slavePriority);
+    Assert.assertFalse(replica.isReplicaTopState());
+    Assert.assertEquals(replica.getCapacity(), capacityDataMapResource2);
+    Assert.assertEquals(replica.getResourceInstanceGroupTag(), group);
+    Assert.assertEquals(replica.getResourceMaxPartitionsPerInstance(), maxPartition);
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java
new file mode 100644
index 0000000..8206f29
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterContext.java
@@ -0,0 +1,90 @@
+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.
+ */
+
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class TestClusterContext extends AbstractTestClusterModel {
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+  }
+
+  @Test
+  public void testNormalUsage() throws IOException {
+    // Test 1 - initialize the cluster context based on the data cache.
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
+
+    ClusterContext context = new ClusterContext(assignmentSet, 2);
+
+    // Note that we left some margin for the max estimation.
+    Assert.assertEquals(context.getEstimatedMaxPartitionCount(), 3);
+    Assert.assertEquals(context.getEstimatedMaxTopStateCount(), 2);
+    Assert.assertEquals(context.getAssignmentForFaultZoneMap(), Collections.emptyMap());
+    for (String resourceName : _resourceNames) {
+      Assert.assertEquals(context.getEstimatedMaxPartitionByResource(resourceName), 2);
+      Assert.assertEquals(
+          context.getPartitionsForResourceAndFaultZone(_testFaultZoneId, resourceName),
+          Collections.emptySet());
+    }
+
+    // Assign
+    Map<String, Map<String, Set<String>>> expectedFaultZoneMap = Collections
+        .singletonMap(_testFaultZoneId, assignmentSet.stream().collect(Collectors
+            .groupingBy(AssignableReplica::getResourceName,
+                Collectors.mapping(AssignableReplica::getPartitionName, Collectors.toSet()))));
+
+    assignmentSet.stream().forEach(replica -> context
+        .addPartitionToFaultZone(_testFaultZoneId, replica.getResourceName(),
+            replica.getPartitionName()));
+    Assert.assertEquals(context.getAssignmentForFaultZoneMap(), expectedFaultZoneMap);
+
+    // release
+    expectedFaultZoneMap.get(_testFaultZoneId).get(_resourceNames.get(0))
+        .remove(_partitionNames.get(0));
+    Assert.assertTrue(context.removePartitionFromFaultZone(_testFaultZoneId, _resourceNames.get(0),
+        _partitionNames.get(0)));
+
+    Assert.assertEquals(context.getAssignmentForFaultZoneMap(), expectedFaultZoneMap);
+  }
+
+  @Test(expectedExceptions = HelixException.class, expectedExceptionsMessageRegExp = "Resource Resource1 already has a replica from partition Partition1 in fault zone testZone")
+  public void testDuplicateAssign() throws IOException {
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignmentSet = generateReplicas(testCache);
+    ClusterContext context = new ClusterContext(assignmentSet, 2);
+    context
+        .addPartitionToFaultZone(_testFaultZoneId, _resourceNames.get(0), _partitionNames.get(0));
+    // Insert again and trigger the error.
+    context
+        .addPartitionToFaultZone(_testFaultZoneId, _resourceNames.get(0), _partitionNames.get(0));
+  }
+}
diff --git a/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
new file mode 100644
index 0000000..c07bd98
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/controller/rebalancer/waged/model/TestClusterModel.java
@@ -0,0 +1,114 @@
+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.
+ */
+
+import org.apache.helix.HelixException;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+public class TestClusterModel extends AbstractTestClusterModel {
+  @BeforeClass
+  public void initialize() {
+    super.initialize();
+  }
+
+  /**
+   * Generate AssignableNodes according to the instances included in the cluster data cache.
+   */
+  Set<AssignableNode> generateNodes(ResourceControllerDataProvider testCache) {
+    Set<AssignableNode> nodeSet = new HashSet<>();
+    testCache.getInstanceConfigMap().values().stream().forEach(config -> nodeSet.add(
+        new AssignableNode(testCache.getClusterConfig(),
+            testCache.getInstanceConfigMap().get(_testInstanceId), config.getInstanceName(),
+            Collections.emptyList())));
+    return nodeSet;
+  }
+
+  @Test
+  public void testNormalUsage() throws IOException {
+    // Test 1 - initialize the cluster model based on the data cache.
+    ResourceControllerDataProvider testCache = setupClusterDataCache();
+    Set<AssignableReplica> assignableReplicas = generateReplicas(testCache);
+    Set<AssignableNode> assignableNodes = generateNodes(testCache);
+
+    ClusterContext context = new ClusterContext(assignableReplicas, 2);
+    ClusterModel clusterModel =
+        new ClusterModel(context, assignableReplicas, assignableNodes, Collections.emptyMap(),
+            Collections.emptyMap());
+
+    Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .allMatch(resourceMap -> resourceMap.values().isEmpty()));
+    Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
+        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+
+    // The initialization of the context, node and replication has been tested separately. So for
+    // cluster model, focus on testing the assignment and release.
+
+    // Assign
+    AssignableReplica replica = assignableReplicas.iterator().next();
+    AssignableNode assignableNode = assignableNodes.iterator().next();
+    clusterModel
+        .assign(replica.getResourceName(), replica.getPartitionName(), replica.getReplicaState(),
+            assignableNode.getInstanceName());
+
+    Assert.assertTrue(
+        clusterModel.getContext().getAssignmentForFaultZoneMap().get(assignableNode.getFaultZone())
+            .get(replica.getResourceName()).contains(replica.getPartitionName()));
+    Assert.assertTrue(assignableNode.getCurrentAssignmentsMap().get(replica.getResourceName())
+        .contains(replica.getPartitionName()));
+
+    // Assign a nonexist replication
+    try {
+      clusterModel.assign("NOT-EXIST", replica.getPartitionName(), replica.getReplicaState(),
+          assignableNode.getInstanceName());
+      Assert.fail("Assigning a non existing resource partition shall fail.");
+    } catch (HelixException ex) {
+      // expected
+    }
+
+    // Assign a non-exist replication
+    try {
+      clusterModel
+          .assign(replica.getResourceName(), replica.getPartitionName(), replica.getReplicaState(),
+              "NON-EXIST");
+      Assert.fail("Assigning a resource partition to a non existing instance shall fail.");
+    } catch (HelixException ex) {
+      // expected
+    }
+
+    // Release
+    clusterModel
+        .release(replica.getResourceName(), replica.getPartitionName(), replica.getReplicaState(),
+            assignableNode.getInstanceName());
+
+    Assert.assertTrue(clusterModel.getContext().getAssignmentForFaultZoneMap().values().stream()
+        .allMatch(resourceMap -> resourceMap.values().stream()
+            .allMatch(partitions -> partitions.isEmpty())));
+    Assert.assertFalse(clusterModel.getAssignableNodes().values().stream()
+        .anyMatch(node -> node.getCurrentAssignmentCount() != 0));
+  }
+}